You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2022/10/21 10:11:24 UTC

[cassandra] 02/05: Provides the Trie interface with MemtableTrie implementation

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

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

commit 30641ea7b6b8253651562aeb0102778a0f9a405b
Author: Branimir Lambov <br...@datastax.com>
AuthorDate: Mon Jan 11 16:02:12 2021 +0200

    Provides the Trie interface with MemtableTrie implementation
    
    also includes functionality to merge, intersect and iterate on tries.
    
    patch by Branimir Lambov; reviewed by Jason Rutherglen, Jacek Lewandowski, Andres de la Peña and Calib Rackliffe for CASSANDRA-17240
---
 .build/cassandra-deps-template.xml                 |    4 +
 .build/parent-pom-template.xml                     |    5 +
 .../config/CassandraRelevantProperties.java        |    1 +
 .../cassandra/db/tries/CollectionMergeTrie.java    |  363 +++++++
 .../cassandra/db/tries/MemtableReadTrie.java       |  859 +++++++++++++++++
 .../apache/cassandra/db/tries/MemtableTrie.java    | 1003 ++++++++++++++++++++
 .../org/apache/cassandra/db/tries/MemtableTrie.md  |  753 +++++++++++++++
 .../cassandra/db/tries/MemtableTrie.md.a1.svg      |  599 ++++++++++++
 .../cassandra/db/tries/MemtableTrie.md.g1.svg      |   76 ++
 .../cassandra/db/tries/MemtableTrie.md.g2.svg      |  116 +++
 .../cassandra/db/tries/MemtableTrie.md.g3.svg      |  253 +++++
 .../cassandra/db/tries/MemtableTrie.md.g4.svg      |  290 ++++++
 .../cassandra/db/tries/MemtableTrie.md.m1.svg      |  349 +++++++
 .../cassandra/db/tries/MemtableTrie.md.m2.svg      |  430 +++++++++
 .../cassandra/db/tries/MemtableTrie.md.m3.svg      |  500 ++++++++++
 .../cassandra/db/tries/MemtableTrie.md.p1.svg      |  405 ++++++++
 .../cassandra/db/tries/MemtableTrie.md.w1.svg      |  226 +++++
 .../cassandra/db/tries/MemtableTrie.md.w2.svg      |  326 +++++++
 .../cassandra/db/tries/MemtableTrie.md.w3.svg      |  269 ++++++
 .../cassandra/db/tries/MemtableTrie.md.wc1.svg     |  349 +++++++
 .../cassandra/db/tries/MemtableTrie.md.wc2.svg     |  314 ++++++
 .../org/apache/cassandra/db/tries/MergeTrie.java   |  167 ++++
 .../apache/cassandra/db/tries/SingletonTrie.java   |  109 +++
 .../org/apache/cassandra/db/tries/SlicedTrie.java  |  242 +++++
 src/java/org/apache/cassandra/db/tries/Trie.java   |  622 ++++++++++++
 src/java/org/apache/cassandra/db/tries/Trie.md     |  252 +++++
 .../org/apache/cassandra/db/tries/TrieDumper.java  |   93 ++
 .../cassandra/db/tries/TrieEntriesIterator.java    |   86 ++
 .../cassandra/db/tries/TrieEntriesWalker.java      |   62 ++
 .../cassandra/db/tries/TriePathReconstructor.java  |   56 ++
 .../org/apache/cassandra/db/tries/TrieToDot.java   |  123 +++
 .../cassandra/db/tries/TrieValuesIterator.java     |   57 ++
 .../test/microbench/tries/ComparisonReadBench.java |  517 ++++++++++
 .../microbench/tries/MemtableTrieReadBench.java    |  183 ++++
 .../microbench/tries/MemtableTrieUnionBench.java   |  146 +++
 .../microbench/tries/MemtableTrieWriteBench.java   |  121 +++
 .../db/tries/CollectionMergeTrieTest.java          |  207 ++++
 .../cassandra/db/tries/MemtableTrieApplyTest.java  |   28 +
 .../cassandra/db/tries/MemtableTriePutTest.java    |  123 +++
 .../cassandra/db/tries/MemtableTrieTestBase.java   |  629 ++++++++++++
 .../db/tries/MemtableTrieThreadedTest.java         |  161 ++++
 .../apache/cassandra/db/tries/MergeTrieTest.java   |   99 ++
 .../apache/cassandra/db/tries/SlicedTrieTest.java  |  527 ++++++++++
 .../apache/cassandra/db/tries/TrieToDotTest.java   |   41 +
 44 files changed, 12141 insertions(+)

diff --git a/.build/cassandra-deps-template.xml b/.build/cassandra-deps-template.xml
index 513fcbc5aa..96c6c5379b 100644
--- a/.build/cassandra-deps-template.xml
+++ b/.build/cassandra-deps-template.xml
@@ -340,5 +340,9 @@
       <groupId>com.github.seancfoley</groupId>
       <artifactId>ipaddress</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.agrona</groupId>
+      <artifactId>agrona</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/.build/parent-pom-template.xml b/.build/parent-pom-template.xml
index b288221e49..8aa4b9bee4 100644
--- a/.build/parent-pom-template.xml
+++ b/.build/parent-pom-template.xml
@@ -1024,6 +1024,11 @@
         <artifactId>ipaddress</artifactId>
         <version>5.3.3</version>
       </dependency>
+      <dependency>
+        <groupId>org.agrona</groupId>
+        <artifactId>agrona</artifactId>
+        <version>1.17.1</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 </project>
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
index d7c543da67..88d70f402b 100644
--- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
@@ -235,6 +235,7 @@ public enum CassandraRelevantProperties
 
     MEMTABLE_OVERHEAD_SIZE("cassandra.memtable.row_overhead_size", "-1"),
     MEMTABLE_OVERHEAD_COMPUTE_STEPS("cassandra.memtable_row_overhead_computation_step", "100000"),
+    MEMTABLE_TRIE_SIZE_LIMIT("cassandra.trie_size_limit_mb"),
     MIGRATION_DELAY("cassandra.migration_delay_ms", "60000"),
     /** Defines how often schema definitions are pulled from the other nodes */
     SCHEMA_PULL_INTERVAL_MS("cassandra.schema_pull_interval_ms", "60000"),
diff --git a/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java b/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java
new file mode 100644
index 0000000000..0336910494
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java
@@ -0,0 +1,363 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * A merged view of multiple tries.
+ *
+ * This is accomplished by walking the cursors in parallel; the merged cursor takes the position and features of the
+ * smallest and advances with it; when multiple cursors are equal, all of them are advanced. The ordered view of the
+ * cursors is maintained using a custom binary min-heap, built for efficiently reforming the heap when the top elements
+ * are advanced (see {@link CollectionMergeCursor}).
+ *
+ * Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors'
+ * positions by their depth descending and then incomingTransition ascending.
+ *
+ * See Trie.md for further details.
+ */
+class CollectionMergeTrie<T> extends Trie<T>
+{
+    private final CollectionMergeResolver<T> resolver;  // only called on more than one input
+    protected final Collection<? extends Trie<T>> inputs;
+
+    CollectionMergeTrie(Collection<? extends Trie<T>> inputs, CollectionMergeResolver<T> resolver)
+    {
+        this.resolver = resolver;
+        this.inputs = inputs;
+    }
+
+    @Override
+    protected Cursor<T> cursor()
+    {
+        return new CollectionMergeCursor<>(resolver, inputs);
+    }
+
+    /**
+     * Compare the positions of two cursors. One is before the other when
+     * - its depth is greater, or
+     * - its depth is equal, and the incoming transition is smaller.
+     */
+    static <T> boolean greaterCursor(Cursor<T> c1, Cursor<T> c2)
+    {
+        int c1depth = c1.depth();
+        int c2depth = c2.depth();
+        if (c1depth != c2depth)
+            return c1depth < c2depth;
+        return c1.incomingTransition() > c2.incomingTransition();
+    }
+
+    static <T> boolean equalCursor(Cursor<T> c1, Cursor<T> c2)
+    {
+        return c1.depth() == c2.depth() && c1.incomingTransition() == c2.incomingTransition();
+    }
+
+    /*
+     * The merge cursor is a variation of the idea of a merge iterator with one key observation: because we advance
+     * the source iterators together, we can compare them just by depth and incoming transition.
+     *
+     * The most straightforward way to implement merging of iterators is to use a {@code PriorityQueue},
+     * {@code poll} it to find the next item to consume, then {@code add} the iterator back after advancing.
+     * This is not very efficient as {@code poll} and {@code add} in all cases require at least
+     * {@code log(size)} comparisons and swaps (usually more than {@code 2*log(size)}) per consumed item, even
+     * if the input is suitable for fast iteration.
+     *
+     * The implementation below makes use of the fact that replacing the top element in a binary heap can be
+     * done much more efficiently than separately removing it and placing it back, especially in the cases where
+     * the top iterator is to be used again very soon (e.g. when there are large sections of the output where
+     * only a limited number of input iterators overlap, which is normally the case in many practically useful
+     * situations, e.g. levelled compaction).
+     *
+     * The implementation builds and maintains a binary heap of sources (stored in an array), where we do not
+     * add items after the initial construction. Instead we advance the smallest element (which is at the top
+     * of the heap) and push it down to find its place for its new position. Should this source be exhausted,
+     * we swap it with the last source in the heap and proceed by pushing that down in the heap.
+     *
+     * In the case where we have multiple sources with matching positions, the merging algorithm
+     * must be able to merge all equal values. To achieve this {@code content} walks the heap to
+     * find all equal cursors without advancing them, and separately {@code advance} advances
+     * all equal sources and restores the heap structure.
+     *
+     * The latter is done equivalently to the process of initial construction of a min-heap using back-to-front
+     * heapification as done in the classic heapsort algorithm. It only needs to heapify subheaps whose top item
+     * is advanced (i.e. one whose position matches the current), and we can do that recursively from
+     * bottom to top. Should a source be exhausted when advancing, it can be thrown away by swapping in the last
+     * source in the heap (note: we must be careful to advance that source too if required).
+     *
+     * To make it easier to advance efficienty in single-sourced branches of tries, we extract the current smallest
+     * cursor (the head) separately, and start any advance with comparing that to the heap's first. When the smallest
+     * cursor remains the same (e.g. in branches coming from a single source) this makes it possible to advance with
+     * just one comparison instead of two at the expense of increasing the number by one in the general case.
+     *
+     * Note: This is a simplification of the MergeIterator code from CASSANDRA-8915, without the leading ordered
+     * section and equalParent flag since comparisons of cursor positions are cheap.
+     */
+    static class CollectionMergeCursor<T> implements Cursor<T>
+    {
+        private final CollectionMergeResolver<T> resolver;
+
+        /**
+         * The smallest cursor, tracked separately to improve performance in single-source sections of the trie.
+         */
+        private Cursor<T> head;
+
+        /**
+         * Binary heap of the remaining cursors. The smallest element is at position 0.
+         * Every element i is smaller than or equal to its two children, i.e.
+         *     heap[i] <= heap[i*2 + 1] && heap[i] <= heap[i*2 + 2]
+         */
+        private final Cursor<T>[] heap;
+
+        /**
+         * A list used to collect contents during content() calls.
+         */
+        private final List<T> contents;
+
+        public CollectionMergeCursor(CollectionMergeResolver<T> resolver, Collection<? extends Trie<T>> inputs)
+        {
+            this.resolver = resolver;
+            int count = inputs.size();
+            // Get cursors for all inputs. Put one of them in head and the rest in the heap.
+            heap = new Cursor[count - 1];
+            contents = new ArrayList<>(count);
+            int i = -1;
+            for (Trie<T> trie : inputs)
+            {
+                Cursor<T> cursor = trie.cursor();
+                assert cursor.depth() == 0;
+                if (i >= 0)
+                    heap[i] = cursor;
+                else
+                    head = cursor;
+                ++i;
+            }
+            // The cursors are all currently positioned on the root and thus in valid heap order.
+        }
+
+        /**
+         * Interface for internal operations that can be applied to the equal top elements of the heap.
+         */
+        interface HeapOp<T>
+        {
+            void apply(CollectionMergeCursor<T> self, Cursor<T> cursor, int index);
+        }
+
+        /**
+         * Apply a non-interfering operation, i.e. one that does not change the cursor state, to all inputs in the heap
+         * that are on equal position to the head.
+         * For interfering operations like advancing the cursors, use {@link #advanceEqualAndRestoreHeap(AdvancingHeapOp)}.
+         */
+        private void applyToEqualOnHeap(HeapOp<T> action)
+        {
+            applyToEqualElementsInHeap(action, 0);
+        }
+
+        /**
+         * Interface for internal advancing operations that can be applied to the heap cursors. This interface provides
+         * the code to restore the heap structure after advancing the cursors.
+         */
+        interface AdvancingHeapOp<T> extends HeapOp<T>
+        {
+            void apply(Cursor<T> cursor);
+
+            default void apply(CollectionMergeCursor<T> self, Cursor<T> cursor, int index)
+            {
+                // Apply the operation, which should advance the position of the element.
+                apply(cursor);
+
+                // This method is called on the back path of the recursion. At this point the heaps at both children are
+                // advanced and well-formed.
+                // Place current node in its proper position.
+                self.heapifyDown(cursor, index);
+                // The heap rooted at index is now advanced and well-formed.
+            }
+        }
+
+
+        /**
+         * Advance the state of all inputs in the heap that are on equal position as the head and restore the heap
+         * invariant.
+         */
+        private void advanceEqualAndRestoreHeap(AdvancingHeapOp<T> action)
+        {
+            applyToEqualElementsInHeap(action, 0);
+        }
+
+        /**
+         * Apply an operation to all elements on the heap that are equal to the head. Descends recursively in the heap
+         * structure to all equal children and applies the operation on the way back.
+         *
+         * This operation can be something that does not change the cursor state (see {@link #content}) or an operation
+         * that advances the cursor to a new state, wrapped in a {@link AdvancingHeapOp} ({@link #advance} or
+         * {@link #skipChildren}). The latter interface takes care of pushing elements down in the heap after advancing
+         * and restores the subheap state on return from each level of the recursion.
+         */
+        private void applyToEqualElementsInHeap(HeapOp<T> action, int index)
+        {
+            if (index >= heap.length)
+                return;
+            Cursor<T> item = heap[index];
+            if (!equalCursor(item, head))
+                return;
+
+            // If the children are at the same position, they also need advancing and their subheap
+            // invariant to be restored.
+            applyToEqualElementsInHeap(action, index * 2 + 1);
+            applyToEqualElementsInHeap(action, index * 2 + 2);
+
+            // Apply the action. This is done on the reverse direction to give the action a chance to form proper
+            // subheaps and combine them on processing the parent.
+            action.apply(this, item, index);
+        }
+
+        /**
+         * Push the given state down in the heap from the given index until it finds its proper place among
+         * the subheap rooted at that position.
+         */
+        private void heapifyDown(Cursor<T> item, int index)
+        {
+            while (true)
+            {
+                int next = index * 2 + 1;
+                if (next >= heap.length)
+                    break;
+                // Select the smaller of the two children to push down to.
+                if (next + 1 < heap.length && greaterCursor(heap[next], heap[next + 1]))
+                    ++next;
+                // If the child is greater or equal, the invariant has been restored.
+                if (!greaterCursor(item, heap[next]))
+                    break;
+                heap[index] = heap[next];
+                index = next;
+            }
+            heap[index] = item;
+        }
+
+        /**
+         * Check if the head is greater than the top element in the heap, and if so, swap them and push down the new
+         * top until its proper place.
+         * @param headDepth the depth of the head cursor (as returned by e.g. advance).
+         * @return the new head element's depth
+         */
+        private int maybeSwapHead(int headDepth)
+        {
+            int heap0Depth = heap[0].depth();
+            if (headDepth > heap0Depth ||
+                (headDepth == heap0Depth && head.incomingTransition() <= heap[0].incomingTransition()))
+                return headDepth;   // head is still smallest
+
+            // otherwise we need to swap heap and heap[0]
+            Cursor<T> newHeap0 = head;
+            head = heap[0];
+            heapifyDown(newHeap0, 0);
+            return heap0Depth;
+        }
+
+        @Override
+        public int advance()
+        {
+            advanceEqualAndRestoreHeap(Cursor::advance);
+            return maybeSwapHead(head.advance());
+        }
+
+        @Override
+        public int advanceMultiple(TransitionsReceiver receiver)
+        {
+            // If the current position is present in just one cursor, we can safely descend multiple levels within
+            // its branch as no one of the other tries has content for it.
+            if (equalCursor(heap[0], head))
+                return advance();   // More than one source at current position, do single-step advance.
+
+            // If there are no children, i.e. the cursor ascends, we have to check if it's become larger than some
+            // other candidate.
+            return maybeSwapHead(head.advanceMultiple(receiver));
+        }
+
+        @Override
+        public int skipChildren()
+        {
+            advanceEqualAndRestoreHeap(Cursor::skipChildren);
+            return maybeSwapHead(head.skipChildren());
+        }
+
+        @Override
+        public int depth()
+        {
+            return head.depth();
+        }
+
+        @Override
+        public int incomingTransition()
+        {
+            return head.incomingTransition();
+        }
+
+        @Override
+        public T content()
+        {
+            applyToEqualOnHeap(CollectionMergeCursor::collectContent);
+            collectContent(head, -1);
+
+            T toReturn;
+            switch (contents.size())
+            {
+                case 0:
+                    toReturn = null;
+                    break;
+                case 1:
+                    toReturn = contents.get(0);
+                    break;
+                default:
+                    toReturn = resolver.resolve(contents);
+                    break;
+            }
+            contents.clear();
+            return toReturn;
+        }
+
+        private void collectContent(Cursor<T> item, int index)
+        {
+            T itemContent = item.content();
+            if (itemContent != null)
+                contents.add(itemContent);
+        }
+    }
+
+    /**
+     * Special instance for sources that are guaranteed distinct. The main difference is that we can form unordered
+     * value list by concatenating sources.
+     */
+    static class Distinct<T> extends CollectionMergeTrie<T>
+    {
+        Distinct(Collection<? extends Trie<T>> inputs)
+        {
+            super(inputs, throwingResolver());
+        }
+
+        @Override
+        public Iterable<T> valuesUnordered()
+        {
+            return Iterables.concat(Iterables.transform(inputs, Trie::valuesUnordered));
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableReadTrie.java b/src/java/org/apache/cassandra/db/tries/MemtableReadTrie.java
new file mode 100644
index 0000000000..073a99e16e
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableReadTrie.java
@@ -0,0 +1,859 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.Function;
+
+import org.agrona.concurrent.UnsafeBuffer;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Memtable trie, i.e. an in-memory trie built for fast modification and reads executing concurrently with writes from
+ * a single mutator thread.
+ *
+ * This class provides the read-only functionality, expanded in {@link MemtableTrie} to writes.
+ */
+public class MemtableReadTrie<T> extends Trie<T>
+{
+    /*
+    TRIE FORMAT AND NODE TYPES
+
+    The memtable trie uses five different types of nodes:
+     - "leaf" nodes, which have content and no children;
+     - single-transition "chain" nodes, which have exactly one child; while each node is a single transition, they are
+       called "chain" because multiple such transition are packed in a block.
+     - "sparse" nodes which have between two and six children;
+     - "split" nodes for anything above six children;
+     - "prefix" nodes that augment one of the other types (except leaf) with content.
+
+    The data for all nodes except leaf ones is stored in a contiguous 'node buffer' and laid out in blocks of 32 bytes.
+    A block only contains data for a single type of node, but there is no direct correspondence between block and node
+    in that:
+     - a single block can contain multiple "chain" nodes.
+     - a sparse node occupies exactly one block.
+     - a split node occupies a variable number of blocks.
+     - a prefix node can be placed in the same block as the node it augments, or in a separate block.
+
+    Nodes are referenced in that buffer by an integer position/pointer, the 'node pointer'. Note that node pointers are
+    not pointing at the beginning of blocks, and we call 'pointer offset' the offset of the node pointer to the block it
+    points into. The value of a 'node pointer' is used to decide what kind of node is pointed:
+
+     - If the pointer is negative, we have a leaf node. Since a leaf has no children, we need no data outside of its
+       content to represent it, and that content is stored in a 'content list', not in the nodes buffer. The content
+       of a particular leaf node is located at the ~pointer position in the content list (~ instead of - so that -1 can
+       correspond to position 0).
+
+     - If the 'pointer offset' is smaller than 28, we have a chain node with one transition. The transition character is
+       the byte at the position pointed in the 'node buffer', and the child is pointed by:
+       - the integer value at offset 28 of the block pointed if the 'pointer offset' is 27
+       - pointer + 1 (which is guaranteed to have offset smaller than 28, i.e. to be a chain node), otherwise
+       In other words, a chain block contains a sequence of characters that leads to the child whose address is at
+       offset 28. It may have between 1 and 28 characters depending on the pointer with which the block is entered.
+
+     - If the 'pointer offset' is 30, we have a sparse node. The data of a sparse node occupies a full block and is laid
+       out as:
+       - six pointers to children at offsets 0 to 24
+       - six transition characters at offsets 24 to 30
+       - an order word stored in the two bytes at offset 30
+       To enable in-place addition of children, the pointers and transition characters are not stored ordered.
+       Instead, we use an order encoding in the last 2 bytes of the node. The encoding is a base-6 number which
+       describes the order of the transitions (least significant digit being the smallest).
+       The node must have at least two transitions and the transition at position 0 is never the biggest (we can
+       enforce this by choosing for position 0 the smaller of the two transitions a sparse node starts with). This
+       allows iteration over the order word (which divides said word by 6 each step) to finish when the result becomes 0.
+
+     - If the 'pointer offset' is 28, the node is a split one. Split nodes are dense, meaning that there is a direct
+       mapping between a transition character and the address of the associated pointer, and new children can easily be
+       added in place.
+       Split nodes occupy multiple blocks, and a child is located by traversing 3 layers of pointers:
+       - the first pointer is within the top-level block (the one pointed by the pointer) and points to a "mid" block.
+         The top-level block has 4 such pointers to "mid" block, located between offset 16 and 32.
+       - the 2nd pointer is within the "mid" block and points to a "tail" block. A "mid" block has 8 such pointers
+         occupying the whole block.
+       - the 3rd pointer is with the "tail" block and is the actual child pointer. Like "mid" block, there are 8 such
+         pointers (so we finally address 4 * 8 * 8 = 256 children).
+       To find a child, we thus need to know the index of the pointer to follow within the top-level block, the index
+       of the one in the "mid" block and the index in the "tail" block. For that, we split the transition byte in a
+       sequence of 2-3-3 bits:
+       - the first 2 bits are the index in the top-level block;
+       - the next 3 bits, the index in the "mid" block;
+       - and the last 3 bits the index in the "tail" block.
+       This layout allows the node to use the smaller fixed-size blocks (instead of 256*4 bytes for the whole character
+       space) and also leaves some room in the head block (the 16 first bytes) for additional information (which we can
+       use to store prefix nodes containing things like deletion times).
+       One split node may need up to 1 + 4 + 4*8 blocks (1184 bytes) to store all its children.
+
+     - If the pointer offset is 31, we have a prefix node. These are two types:
+       -- Embedded prefix nodes occupy the free bytes in a chain or split node. The byte at offset 4 has the offset
+          within the 32-byte block for the augmented node.
+       -- Full prefix nodes have 0xFF at offset 4 and a pointer at 28, pointing to the augmented node.
+       Both types contain an index for content at offset 0. The augmented node cannot be a leaf or NONE -- in the former
+       case the leaf itself contains the content index, in the latter we use a leaf instead.
+       The term "node" when applied to these is a bit of a misnomer as they are not presented as separate nodes during
+       traversals. Instead, they augment a node, changing only its content. Internally we create a Node object for the
+       augmented node and wrap a PrefixNode around it, which changes the `content()` method and routes all other
+       calls to the augmented node's methods.
+
+     When building a trie we first allocate the content, then create a chain node leading to it. While we only have
+     single transitions leading to a chain node, we can expand that node (attaching a character and using pointer - 1)
+     instead of creating a new one. When a chain node already has a child and needs a new one added we change the type
+     (i.e. create a new node and remap the parent) to sparse with two children. When a six-child sparse node needs a new
+     child, we switch to split.
+
+     Blocks currently are not reused, because we do not yet have a mechanism to tell when readers are done with blocks
+     they are referencing. This currently causes a very low overhead (because we change data in place with the only
+     exception of nodes needing to change type) and is planned to be addressed later.
+
+     For further descriptions and examples of the mechanics of the trie, see MemtableTrie.md.
+     */
+
+    static final int BLOCK_SIZE = 32;
+
+    // Biggest block offset that can contain a pointer.
+    static final int LAST_POINTER_OFFSET = BLOCK_SIZE - 4;
+
+    /*
+     Block offsets used to identify node types (by comparing them to the node 'pointer offset').
+     */
+
+    // split node (dense, 2-3-3 transitions), laid out as 4 pointers to "mid" block, with has 8 pointers to "tail" block,
+    // which has 8 pointers to children
+    static final int SPLIT_OFFSET = BLOCK_SIZE - 4;
+    // sparse node, unordered list of up to 6 transition, laid out as 6 transition pointers followed by 6 transition
+    // bytes. The last two bytes contain an ordering of the transitions (in base-6) which is used for iteration. On
+    // update the pointer is set last, i.e. during reads the node may show that a transition exists and list a character
+    // for it, but pointer may still be null.
+    static final int SPARSE_OFFSET = BLOCK_SIZE - 2;
+    // min and max offset for a chain node. A block of chain node is laid out as a pointer at LAST_POINTER_OFFSET,
+    // preceded by characters that lead to it. Thus a full chain block contains BLOCK_SIZE-4 transitions/chain nodes.
+    static final int CHAIN_MIN_OFFSET = 0;
+    static final int CHAIN_MAX_OFFSET = BLOCK_SIZE - 5;
+    // Prefix node, an intermediate node augmenting its child node with content.
+    static final int PREFIX_OFFSET = BLOCK_SIZE - 1;
+
+    /*
+     Offsets and values for navigating in a block for particular node type. Those offsets are 'from the node pointer'
+     (not the block start) and can be thus negative since node pointers points towards the end of blocks.
+     */
+
+    // Limit for the starting cell / sublevel (2 bits -> 4 pointers).
+    static final int SPLIT_START_LEVEL_LIMIT = 4;
+    // Limit for the other sublevels (3 bits -> 8 pointers).
+    static final int SPLIT_OTHER_LEVEL_LIMIT = 8;
+    // Bitshift between levels.
+    static final int SPLIT_LEVEL_SHIFT = 3;
+
+    static final int SPARSE_CHILD_COUNT = 6;
+    // Offset to the first child pointer of a spare node (laid out from the start of the block)
+    static final int SPARSE_CHILDREN_OFFSET = 0 - SPARSE_OFFSET;
+    // Offset to the first transition byte of a sparse node (laid out after the child pointers)
+    static final int SPARSE_BYTES_OFFSET = SPARSE_CHILD_COUNT * 4 - SPARSE_OFFSET;
+    // Offset to the order word of a sparse node (laid out after the children (pointer + transition byte))
+    static final int SPARSE_ORDER_OFFSET = SPARSE_CHILD_COUNT * 5 - SPARSE_OFFSET;  // 0
+
+    // Offset of the flag byte in a prefix node. In shared blocks, this contains the offset of the next node.
+    static final int PREFIX_FLAGS_OFFSET = 4 - PREFIX_OFFSET;
+    // Offset of the content id
+    static final int PREFIX_CONTENT_OFFSET = 0 - PREFIX_OFFSET;
+    // Offset of the next pointer in a non-shared prefix node
+    static final int PREFIX_POINTER_OFFSET = LAST_POINTER_OFFSET - PREFIX_OFFSET;
+
+    // Initial capacity for the node data buffer.
+    static final int INITIAL_BUFFER_CAPACITY = 256;
+
+    /**
+     * Value used as null for node pointers.
+     * No node can use this address (we enforce this by not allowing chain nodes to grow to position 0).
+     * Do not change this as the code relies there being a NONE placed in all bytes of the block that are not set.
+     */
+    static final int NONE = 0;
+
+    volatile int root;
+
+    final UnsafeBuffer buffer;
+
+    volatile AtomicReferenceArray<T> contentArray;
+
+    MemtableReadTrie(UnsafeBuffer buffer, AtomicReferenceArray<T> contentArray, int root)
+    {
+        this.buffer = buffer;
+        this.contentArray = contentArray;
+        this.root = root;
+    }
+
+    /*
+     Buffer, content list and block management
+     */
+
+    /**
+     * Pointer offset for a node pointer.
+     */
+    int offset(int pos)
+    {
+        return pos & (BLOCK_SIZE - 1);
+    }
+
+    final int getUnsignedByte(int pos)
+    {
+        return buffer.getByte(pos) & 0xFF;
+    }
+
+    final int getUnsignedShort(int pos)
+    {
+        return buffer.getShort(pos) & 0xFFFF;
+    }
+
+    final int getInt(int pos) { return buffer.getInt(pos); }
+
+    T getContent(int index)
+    {
+        return contentArray.get(index);
+    }
+
+    /*
+     Reading node content
+     */
+
+    boolean isNull(int node)
+    {
+        return node == NONE;
+    }
+
+    boolean isLeaf(int node)
+    {
+        return node < NONE;
+    }
+
+    boolean isNullOrLeaf(int node)
+    {
+        return node <= NONE;
+    }
+
+    /**
+     * Returns the number of transitions in a chain block entered with the given pointer.
+     */
+    private int chainBlockLength(int node)
+    {
+        return LAST_POINTER_OFFSET - offset(node);
+    }
+
+    /**
+     * Get a node's child for the given transition character
+     */
+    int getChild(int node, int trans)
+    {
+        if (isNullOrLeaf(node))
+            return NONE;
+
+        node = followContentTransition(node);
+
+        switch (offset(node))
+        {
+            case SPARSE_OFFSET:
+                return getSparseChild(node, trans);
+            case SPLIT_OFFSET:
+                return getSplitChild(node, trans);
+            case CHAIN_MAX_OFFSET:
+                if (trans != getUnsignedByte(node))
+                    return NONE;
+                return getInt(node + 1);
+            default:
+                if (trans != getUnsignedByte(node))
+                    return NONE;
+                return node + 1;
+        }
+    }
+
+    protected int followContentTransition(int node)
+    {
+        if (isNullOrLeaf(node))
+            return NONE;
+
+        if (offset(node) == PREFIX_OFFSET)
+        {
+            int b = getUnsignedByte(node + PREFIX_FLAGS_OFFSET);
+            if (b < BLOCK_SIZE)
+                node = node - PREFIX_OFFSET + b;
+            else
+                node = getInt(node + PREFIX_POINTER_OFFSET);
+
+            assert node >= 0 && offset(node) != PREFIX_OFFSET;
+        }
+        return node;
+    }
+
+    /**
+     * Advance as long as the cell pointed to by the given pointer will let you.
+     * <p>
+     * This is the same as getChild(node, first), except for chain nodes where it would walk the fill chain as long as
+     * the input source matches.
+     */
+    int advance(int node, int first, ByteSource rest)
+    {
+        if (isNullOrLeaf(node))
+            return NONE;
+
+        node = followContentTransition(node);
+
+        switch (offset(node))
+        {
+            case SPARSE_OFFSET:
+                return getSparseChild(node, first);
+            case SPLIT_OFFSET:
+                return getSplitChild(node, first);
+            default:
+                // Check the first byte matches the expected
+                if (getUnsignedByte(node++) != first)
+                    return NONE;
+                // Check the rest of the bytes provided by the chain node
+                for (int length = chainBlockLength(node); length > 0; --length)
+                {
+                    first = rest.next();
+                    if (getUnsignedByte(node++) != first)
+                        return NONE;
+                }
+                // All bytes matched, node is now positioned on the child pointer. Follow it.
+                return getInt(node);
+        }
+    }
+
+    /**
+     * Get the child for the given transition character, knowing that the node is sparse
+     */
+    int getSparseChild(int node, int trans)
+    {
+        for (int i = 0; i < SPARSE_CHILD_COUNT; ++i)
+        {
+            if (getUnsignedByte(node + SPARSE_BYTES_OFFSET + i) == trans)
+            {
+                int child = getInt(node + SPARSE_CHILDREN_OFFSET + i * 4);
+
+                // we can't trust the transition character read above, because it may have been fetched before a
+                // concurrent update happened, and the update may have managed to modify the pointer by now.
+                // However, if we read it now that we have accessed the volatile pointer, it must have the correct
+                // value as it is set before the pointer.
+                if (child != NONE && getUnsignedByte(node + SPARSE_BYTES_OFFSET + i) == trans)
+                    return child;
+            }
+        }
+        return NONE;
+    }
+
+    /**
+     * Given a transition, returns the corresponding index (within the node block) of the pointer to the mid block of
+     * a split node.
+     */
+    int splitNodeMidIndex(int trans)
+    {
+        // first 2 bits of the 2-3-3 split
+        return (trans >> 6) & 0x3;
+    }
+
+    /**
+     * Given a transition, returns the corresponding index (within the mid block) of the pointer to the tail block of
+     * a split node.
+     */
+    int splitNodeTailIndex(int trans)
+    {
+        // second 3 bits of the 2-3-3 split
+        return (trans >> 3) & 0x7;
+    }
+
+    /**
+     * Given a transition, returns the corresponding index (within the tail block) of the pointer to the child of
+     * a split node.
+     */
+    int splitNodeChildIndex(int trans)
+    {
+        // third 3 bits of the 2-3-3 split
+        return trans & 0x7;
+    }
+
+    /**
+     * Get the child for the given transition character, knowing that the node is split
+     */
+    int getSplitChild(int node, int trans)
+    {
+        int mid = getSplitBlockPointer(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT);
+        if (isNull(mid))
+            return NONE;
+
+        int tail = getSplitBlockPointer(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT);
+        if (isNull(tail))
+            return NONE;
+        return getSplitBlockPointer(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT);
+    }
+
+    /**
+     * Get the content for a given node
+     */
+    T getNodeContent(int node)
+    {
+        if (isLeaf(node))
+            return getContent(~node);
+
+        if (offset(node) != PREFIX_OFFSET)
+            return null;
+
+        int index = getInt(node + PREFIX_CONTENT_OFFSET);
+        return (index >= 0)
+               ? getContent(index)
+               : null;
+    }
+
+    int splitBlockPointerAddress(int node, int childIndex, int subLevelLimit)
+    {
+        return node - SPLIT_OFFSET + (8 - subLevelLimit + childIndex) * 4;
+    }
+
+    int getSplitBlockPointer(int node, int childIndex, int subLevelLimit)
+    {
+        return getInt(splitBlockPointerAddress(node, childIndex, subLevelLimit));
+    }
+
+    /**
+     * Backtracking state for a cursor.
+     *
+     * To avoid allocations and pointer-chasing, the backtracking data is stored in a simple int array with
+     * BACKTRACK_INTS_PER_ENTRY ints for each level.
+     */
+    private static class CursorBacktrackingState
+    {
+        static final int BACKTRACK_INTS_PER_ENTRY = 3;
+        static final int BACKTRACK_INITIAL_SIZE = 16;
+        private int[] backtrack = new int[BACKTRACK_INITIAL_SIZE * BACKTRACK_INTS_PER_ENTRY];
+        int backtrackDepth = 0;
+
+        void addBacktrack(int node, int data, int depth)
+        {
+            if (backtrackDepth * BACKTRACK_INTS_PER_ENTRY >= backtrack.length)
+                backtrack = Arrays.copyOf(backtrack, backtrack.length * 2);
+            backtrack[backtrackDepth * BACKTRACK_INTS_PER_ENTRY + 0] = node;
+            backtrack[backtrackDepth * BACKTRACK_INTS_PER_ENTRY + 1] = data;
+            backtrack[backtrackDepth * BACKTRACK_INTS_PER_ENTRY + 2] = depth;
+            ++backtrackDepth;
+        }
+
+        int node(int backtrackDepth)
+        {
+            return backtrack[backtrackDepth * BACKTRACK_INTS_PER_ENTRY + 0];
+        }
+
+        int data(int backtrackDepth)
+        {
+            return backtrack[backtrackDepth * BACKTRACK_INTS_PER_ENTRY + 1];
+        }
+
+        int depth(int backtrackDepth)
+        {
+            return backtrack[backtrackDepth * BACKTRACK_INTS_PER_ENTRY + 2];
+        }
+    }
+
+    /*
+     * Cursor implementation.
+     *
+     * MemtableTrie cursors maintain their backtracking state in CursorBacktrackingState where they store
+     * information about the node to backtrack to and the transitions still left to take or attempt.
+     *
+     * This information is different for the different types of node:
+     * - for leaf and chain no backtracking is saved (because we know there are no further transitions)
+     * - for sparse we store the remainder of the order word
+     * - for split we store one entry per sub-level of the 2-3-3 split
+     *
+     * When the cursor is asked to advance it first checks the current node for children, and if there aren't any
+     * (i.e. it is positioned on a leaf node), it goes one level up the backtracking chain, where we are guaranteed to
+     * have a remaining child to advance to. When there's nothing to backtrack to, the trie is exhausted.
+     */
+    class MemtableCursor extends CursorBacktrackingState implements Cursor<T>
+    {
+        private int currentNode;
+        private int incomingTransition;
+        private T content;
+        private int depth = -1;
+
+        MemtableCursor()
+        {
+            descendInto(root, -1);
+        }
+
+        @Override
+        public int advance()
+        {
+            if (isNullOrLeaf(currentNode))
+                return backtrack();
+            else
+                return advanceToFirstChild(currentNode);
+        }
+
+        @Override
+        public int advanceMultiple(TransitionsReceiver receiver)
+        {
+            int node = currentNode;
+            if (!isChainNode(node))
+                return advance();
+
+            // Jump directly to the chain's child.
+            int bytesJumped = chainBlockLength(node) - 1;   // leave the last byte for incomingTransition
+            if (receiver != null && bytesJumped > 0)
+                receiver.addPathBytes(buffer, node, bytesJumped);
+            depth += bytesJumped;    // descendInto will add one
+            node += bytesJumped;
+
+            // inChunkNode is now positioned on the last byte of the chain.
+            // Consume it to be the new state's incomingTransition.
+            int transition = getUnsignedByte(node++);
+            // inChunkNode is now positioned on the child pointer.
+            int child = getInt(node);
+            return descendInto(child, transition);
+        }
+
+        @Override
+        public int skipChildren()
+        {
+            return backtrack();
+        }
+
+        @Override
+        public int depth()
+        {
+            return depth;
+        }
+
+        @Override
+        public T content()
+        {
+            return content;
+        }
+
+        @Override
+        public int incomingTransition()
+        {
+            return incomingTransition;
+        }
+
+        private int backtrack()
+        {
+            if (--backtrackDepth < 0)
+                return depth = -1;
+
+            depth = depth(backtrackDepth);
+            return advanceToNextChild(node(backtrackDepth), data(backtrackDepth));
+        }
+
+        private int advanceToFirstChild(int node)
+        {
+            assert (!isNullOrLeaf(node));
+
+            switch (offset(node))
+            {
+                case SPLIT_OFFSET:
+                    return descendInSplitSublevel(node, SPLIT_START_LEVEL_LIMIT, 0, SPLIT_LEVEL_SHIFT * 2);
+                case SPARSE_OFFSET:
+                    return nextValidSparseTransition(node, getUnsignedShort(node + SPARSE_ORDER_OFFSET));
+                default:
+                    return getChainTransition(node);
+            }
+        }
+
+        private int advanceToNextChild(int node, int data)
+        {
+            assert (!isNullOrLeaf(node));
+
+            switch (offset(node))
+            {
+                case SPLIT_OFFSET:
+                    return nextValidSplitTransition(node, data);
+                case SPARSE_OFFSET:
+                    return nextValidSparseTransition(node, data);
+                default:
+                    throw new AssertionError("Unexpected node type in backtrack state.");
+            }
+        }
+
+        /**
+         * Descend into the sub-levels of a split node. Advances to the first child and creates backtracking entries
+         * for the following ones. We use the bits of trans (lowest non-zero ones) to identify which sub-level an
+         * entry refers to.
+         *
+         * @param node The node or block id, must have offset SPLIT_OFFSET.
+         * @param limit The transition limit for the current sub-level (4 for the start, 8 for the others).
+         * @param collected The transition bits collected from the parent chain (e.g. 0x40 after following 1 on the top
+         *                  sub-level).
+         * @param shift This level's bit shift (6 for start, 3 for mid and 0 for tail).
+         * @return the depth reached after descending.
+         */
+        private int descendInSplitSublevel(int node, int limit, int collected, int shift)
+        {
+            while (true)
+            {
+                assert offset(node) == SPLIT_OFFSET;
+                int childIndex;
+                int child = NONE;
+                // find the first non-null child
+                for (childIndex = 0; childIndex < limit; ++childIndex)
+                {
+                    child = getSplitBlockPointer(node, childIndex, limit);
+                    if (!isNull(child))
+                        break;
+                }
+                // there must be at least one child
+                assert childIndex < limit && child != NONE;
+
+                // look for any more valid transitions and add backtracking if found
+                maybeAddSplitBacktrack(node, childIndex, limit, collected, shift);
+
+                // add the bits just found
+                collected |= childIndex << shift;
+                // descend to next sub-level or child
+                if (shift == 0)
+                    return descendInto(child, collected);
+                // continue with next sublevel; same as
+                // return descendInSplitSublevel(child + SPLIT_OFFSET, 8, collected, shift - 3)
+                node = child;
+                limit = SPLIT_OTHER_LEVEL_LIMIT;
+                shift -= SPLIT_LEVEL_SHIFT;
+            }
+        }
+
+        /**
+         * Backtrack to a split sub-level. The level is identified by the lowest non-0 bits in trans.
+         */
+        private int nextValidSplitTransition(int node, int trans)
+        {
+            assert trans >= 0 && trans <= 0xFF;
+            int childIndex = splitNodeChildIndex(trans);
+            if (childIndex > 0)
+            {
+                maybeAddSplitBacktrack(node,
+                                       childIndex,
+                                       SPLIT_OTHER_LEVEL_LIMIT,
+                                       trans & -(1 << (SPLIT_LEVEL_SHIFT * 1)),
+                                       SPLIT_LEVEL_SHIFT * 0);
+                int child = getSplitBlockPointer(node, childIndex, SPLIT_OTHER_LEVEL_LIMIT);
+                return descendInto(child, trans);
+            }
+            int tailIndex = splitNodeTailIndex(trans);
+            if (tailIndex > 0)
+            {
+                maybeAddSplitBacktrack(node,
+                                       tailIndex,
+                                       SPLIT_OTHER_LEVEL_LIMIT,
+                                       trans & -(1 << (SPLIT_LEVEL_SHIFT * 2)),
+                                       SPLIT_LEVEL_SHIFT * 1);
+                int tail = getSplitBlockPointer(node, tailIndex, SPLIT_OTHER_LEVEL_LIMIT);
+                return descendInSplitSublevel(tail,
+                                              SPLIT_OTHER_LEVEL_LIMIT,
+                                              trans,
+                                              SPLIT_LEVEL_SHIFT * 0);
+            }
+            int midIndex = splitNodeMidIndex(trans);
+            assert midIndex > 0;
+            maybeAddSplitBacktrack(node,
+                                   midIndex,
+                                   SPLIT_START_LEVEL_LIMIT,
+                                   0,
+                                   SPLIT_LEVEL_SHIFT * 2);
+            int mid = getSplitBlockPointer(node, midIndex, SPLIT_START_LEVEL_LIMIT);
+            return descendInSplitSublevel(mid,
+                                          SPLIT_OTHER_LEVEL_LIMIT,
+                                          trans,
+                                          SPLIT_LEVEL_SHIFT * 1);
+        }
+
+        /**
+         * Look for any further non-null transitions on this sub-level and, if found, add a backtracking entry.
+         */
+        private void maybeAddSplitBacktrack(int node, int startAfter, int limit, int collected, int shift)
+        {
+            int nextChildIndex;
+            for (nextChildIndex = startAfter + 1; nextChildIndex < limit; ++nextChildIndex)
+            {
+                if (!isNull(getSplitBlockPointer(node, nextChildIndex, limit)))
+                    break;
+            }
+            if (nextChildIndex < limit)
+                addBacktrack(node, collected | (nextChildIndex << shift), depth);
+        }
+
+        private int nextValidSparseTransition(int node, int data)
+        {
+            // Peel off the next index.
+            int index = data % SPARSE_CHILD_COUNT;
+            data = data / SPARSE_CHILD_COUNT;
+
+            // If there are remaining transitions, add backtracking entry.
+            if (data > 0)
+                addBacktrack(node, data, depth);
+
+            // Follow the transition.
+            int child = getInt(node + SPARSE_CHILDREN_OFFSET + index * 4);
+            int transition = getUnsignedByte(node + SPARSE_BYTES_OFFSET + index);
+            return descendInto(child, transition);
+        }
+
+        private int getChainTransition(int node)
+        {
+            // No backtracking needed.
+            int transition = getUnsignedByte(node);
+            int next = node + 1;
+            if (offset(next) <= CHAIN_MAX_OFFSET)
+                return descendIntoChain(next, transition);
+            else
+                return descendInto(getInt(node + 1), transition);
+        }
+
+        private int descendInto(int child, int transition)
+        {
+            ++depth;
+            incomingTransition = transition;
+            content = getNodeContent(child);
+            currentNode = followContentTransition(child);
+            return depth;
+        }
+
+        private int descendIntoChain(int child, int transition)
+        {
+            ++depth;
+            incomingTransition = transition;
+            content = null;
+            currentNode = child;
+            return depth;
+        }
+    }
+
+    private boolean isChainNode(int node)
+    {
+        return !isNullOrLeaf(node) && offset(node) <= CHAIN_MAX_OFFSET;
+    }
+
+    public MemtableCursor cursor()
+    {
+        return new MemtableCursor();
+    }
+
+    /*
+     Direct read methods
+     */
+
+    /**
+     * Get the content mapped by the specified key.
+     * Fast implementation using integer node addresses.
+     */
+    public T get(ByteComparable path)
+    {
+        int n = root;
+        ByteSource source = path.asComparableBytes(BYTE_COMPARABLE_VERSION);
+        while (!isNull(n))
+        {
+            int c = source.next();
+            if (c == ByteSource.END_OF_STREAM)
+                return getNodeContent(n);
+
+            n = advance(n, c, source);
+        }
+
+        return null;
+    }
+
+    public boolean isEmpty()
+    {
+        return isNull(root);
+    }
+
+    /**
+     * Override of dump to provide more detailed printout that includes the type of each node in the trie.
+     * We do this via a wrapping cursor that returns a content string for the type of node for every node we return.
+     */
+    @Override
+    public String dump(Function<T, String> contentToString)
+    {
+        MemtableCursor source = cursor();
+        class TypedNodesCursor implements Cursor<String>
+        {
+            @Override
+            public int advance()
+            {
+                return source.advance();
+            }
+
+
+            @Override
+            public int advanceMultiple(TransitionsReceiver receiver)
+            {
+                return source.advanceMultiple(receiver);
+            }
+
+            @Override
+            public int skipChildren()
+            {
+                return source.skipChildren();
+            }
+
+            @Override
+            public int depth()
+            {
+                return source.depth();
+            }
+
+            @Override
+            public int incomingTransition()
+            {
+                return source.incomingTransition();
+            }
+
+            @Override
+            public String content()
+            {
+                String type = null;
+                int node = source.currentNode;
+                if (!isNullOrLeaf(node))
+                {
+                    switch (offset(node))
+                    {
+                        case SPARSE_OFFSET:
+                            type = "[SPARSE]";
+                            break;
+                        case SPLIT_OFFSET:
+                            type = "[SPLIT]";
+                            break;
+                        case PREFIX_OFFSET:
+                            throw new AssertionError("Unexpected prefix as cursor currentNode.");
+                        default:
+                            type = "[CHAIN]";
+                            break;
+                    }
+                }
+                T content = source.content();
+                if (content != null)
+                {
+                    if (type != null)
+                        return contentToString.apply(content) + " -> " + type;
+                    else
+                        return contentToString.apply(content);
+                }
+                else
+                    return type;
+            }
+        }
+        return process(new TrieDumper<>(Function.identity()), new TypedNodesCursor());
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.java b/src/java/org/apache/cassandra/db/tries/MemtableTrie.java
new file mode 100644
index 0000000000..00d68fbf5a
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.java
@@ -0,0 +1,1003 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.LoggerFactory;
+
+import org.agrona.concurrent.UnsafeBuffer;
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.github.jamm.MemoryLayoutSpecification;
+
+/**
+ * Memtable trie, i.e. an in-memory trie built for fast modification and reads executing concurrently with writes from
+ * a single mutator thread.
+ *
+ * This class can currently only provide atomicity (i.e. reads seeing either the content before a write, or the
+ * content after it; any read seeing the write enforcing any subsequent (i.e. started after it completed) reads to
+ * also see it) for singleton writes (i.e. calls to {@link #putRecursive}, {@link #putSingleton} or {@link #apply}
+ * with a singleton trie as argument).
+ *
+ * Because it uses 32-bit pointers in byte buffers, this trie has a fixed size limit of 2GB.
+ */
+public class MemtableTrie<T> extends MemtableReadTrie<T>
+{
+    // See the trie format description in MemtableReadTrie.
+
+    /**
+     * Trie size limit. This is not enforced, but users must check from time to time that it is not exceeded (using
+     * {@link #reachedAllocatedSizeThreshold()}) and start switching to a new trie if it is.
+     * This must be done to avoid tries growing beyond their hard 2GB size limit (due to the 32-bit pointers).
+     */
+    @VisibleForTesting
+    static final int ALLOCATED_SIZE_THRESHOLD;
+    static
+    {
+        // Default threshold + 10% == 2 GB. This should give the owner enough time to react to the
+        // {@link #reachedAllocatedSizeThreshold()} signal and switch this trie out before it fills up.
+        int limitInMB = CassandraRelevantProperties.MEMTABLE_OVERHEAD_SIZE.getInt(2048 * 10 / 11);
+        if (limitInMB < 1 || limitInMB > 2047)
+            throw new AssertionError(CassandraRelevantProperties.MEMTABLE_OVERHEAD_SIZE.getKey() +
+                                     " must be within 1 and 2047");
+        ALLOCATED_SIZE_THRESHOLD = 1024 * 1024 * limitInMB;
+    }
+
+    private int allocatedPos = 0;
+    private int contentCount = 0;
+
+    private final BufferType bufferType;    // on or off heap
+
+    private static final long EMPTY_SIZE_ON_HEAP; // for space calculations
+    private static final long EMPTY_SIZE_OFF_HEAP; // for space calculations
+
+    static
+    {
+        MemtableTrie<Object> empty = new MemtableTrie<>(BufferType.ON_HEAP);
+        EMPTY_SIZE_ON_HEAP = ObjectSizes.measureDeep(empty)
+                             - empty.contentArray.length() * MemoryLayoutSpecification.SPEC.getReferenceSize()
+                             - empty.buffer.capacity();
+        empty = new MemtableTrie<>(BufferType.OFF_HEAP);
+        EMPTY_SIZE_OFF_HEAP = ObjectSizes.measureDeep(empty)
+                              - empty.contentArray.length() * MemoryLayoutSpecification.SPEC.getReferenceSize()
+                              - empty.buffer.capacity();
+    }
+
+    public MemtableTrie(BufferType bufferType)
+    {
+        super(new UnsafeBuffer(bufferType.allocate(INITIAL_BUFFER_CAPACITY)), new AtomicReferenceArray<>(16), NONE);
+        this.bufferType = bufferType;
+        assert INITIAL_BUFFER_CAPACITY % BLOCK_SIZE == 0;
+    }
+
+    // Buffer, content list and block management
+
+    /**
+     * Because we use buffers and 32-bit pointers, the trie cannot grow over 2GB of size. This exception is thrown if
+     * a trie operation needs it to grow over that limit.
+     *
+     * To avoid this problem, users should query {@link #reachedAllocatedSizeThreshold} from time to time. If the call
+     * returns true, they should switch to a new trie (e.g. by flushing a memtable) as soon as possible. The threshold
+     * is configurable, and is set by default to 10% under the 2GB limit to give ample time for the switch to happen.
+     */
+    public static class SpaceExhaustedException extends Exception
+    {
+        public SpaceExhaustedException()
+        {
+            super("The hard 2GB limit on trie size has been exceeded");
+        }
+    }
+
+    private int allocateBlock() throws SpaceExhaustedException
+    {
+        // Note: If this method is modified, please run MemtableTrieTest.testOver1GSize to verify it acts correctly
+        // close to the 2G limit.
+        int v = allocatedPos;
+        if (buffer.capacity() == v)
+        {
+            int newSize;
+            if (v >= ALLOCATED_SIZE_THRESHOLD)
+            {
+                // we don't expect to write much after the threshold has been reached
+                // to avoid allocating too much space which will be left unused,
+                // grow by 10% of the limit, rounding up to BLOCK_SIZE
+                newSize = (v + ALLOCATED_SIZE_THRESHOLD / 10 + BLOCK_SIZE - 1) & -BLOCK_SIZE;
+                // If we do this repeatedly and the calculated size grows over 2G, it will overflow and result in a
+                // negative integer. In that case, cap it to a size that can be allocated.
+                if (newSize < 0)
+                {
+                    newSize = 0x7FFFFF00;   // 2G - 256 bytes
+                    if (newSize == allocatedPos)    // already at limit
+                        throw new SpaceExhaustedException();
+                    LoggerFactory.getLogger(getClass()).debug("Growing memtable trie to maximum size {}",
+                                                              FBUtilities.prettyPrintMemory(newSize));
+                }
+                else
+                    LoggerFactory.getLogger(getClass()).debug("Growing memtable trie by 10% over the {} limit to {}",
+                                                              FBUtilities.prettyPrintMemory(ALLOCATED_SIZE_THRESHOLD),
+                                                              FBUtilities.prettyPrintMemory(newSize));
+            } else
+                newSize = v * 2;
+
+            ByteBuffer newBuffer = bufferType.allocate(newSize);
+            buffer.getBytes(0, newBuffer, v);
+            buffer.wrap(newBuffer);
+            // The above does not contain any happens-before enforcing writes, thus at this point the new buffer may be
+            // invisible to any concurrent readers. Touching the volatile root pointer (which any new read must go
+            // through) enforces a happens-before that makes it visible to all new reads (note: when the write completes
+            // it must do some volatile write, but that will be in the new buffer and without the line below could
+            // remain unreachable by other cores).
+            root = root;
+        }
+
+        allocatedPos += BLOCK_SIZE;
+        return v;
+    }
+
+    private int addContent(T value)
+    {
+        int index = contentCount++;
+        if (index == contentArray.length())
+        {
+            AtomicReferenceArray<T> newContent = new AtomicReferenceArray<>(index * 2);
+            for (int i = 0; i < contentArray.length(); ++i)
+                newContent.lazySet(i, contentArray.get(i));
+            contentArray = newContent;  // This is a volatile set, hence all previous stores must become visible
+        }
+        contentArray.lazySet(index, value); // no need for a volatile set here; at this point the item is not referenced
+                                            // by any node in the trie, and a volatile set will be made to reference it.
+        return index;
+    }
+
+    private void setContent(int index, T value)
+    {
+        contentArray.set(index, value);
+    }
+
+    // Write methods
+
+    // Write visibility model: writes are not volatile, with the exception of the final write before a call returns
+    // the same value that was present before (e.g. content was updated in-place / existing node got a new child or had
+    // a child pointer updated); if the whole path including the root node changed, the root itself gets a volatile
+    // write.
+    // This final write is the point where any new cells created during the write become visible for readers for the
+    // first time, and such readers must pass through reading that pointer, which forces a happens-before relationship
+    // that extends to all values written by this thread before it.
+
+    /**
+     * Attach a child to the given non-content node. This may be an update for an existing branch, or a new child for
+     * the node. An update _is_ required (i.e. this is only called when the newChild pointer is not the same as the
+     * existing value).
+     */
+    private int attachChild(int node, int trans, int newChild) throws SpaceExhaustedException
+    {
+        assert !isLeaf(node) : "attachChild cannot be used on content nodes.";
+
+        switch (offset(node))
+        {
+            case PREFIX_OFFSET:
+                assert false : "attachChild cannot be used on content nodes.";
+            case SPARSE_OFFSET:
+                return attachChildToSparse(node, trans, newChild);
+            case SPLIT_OFFSET:
+                attachChildToSplit(node, trans, newChild);
+                return node;
+            case LAST_POINTER_OFFSET - 1:
+                // If this is the last character in a Chain block, we can modify the child in-place
+                if (trans == getUnsignedByte(node))
+                {
+                    buffer.putIntVolatile(node + 1, newChild);
+                    return node;
+                }
+                // else pass through
+            default:
+                return attachChildToChain(node, trans, newChild);
+        }
+    }
+
+    /**
+     * Attach a child to the given split node. This may be an update for an existing branch, or a new child for the node.
+     */
+    private void attachChildToSplit(int node, int trans, int newChild) throws SpaceExhaustedException
+    {
+        int midPos = splitBlockPointerAddress(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT);
+        int mid = getInt(midPos);
+        if (isNull(mid))
+        {
+            mid = createEmptySplitNode();
+            int tailPos = splitBlockPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT);
+            int tail = createEmptySplitNode();
+            int childPos = splitBlockPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT);
+            buffer.putInt(childPos, newChild);
+            buffer.putInt(tailPos, tail);
+            buffer.putIntVolatile(midPos, mid);
+            return;
+        }
+
+        int tailPos = splitBlockPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT);
+        int tail = getInt(tailPos);
+        if (isNull(tail))
+        {
+            tail = createEmptySplitNode();
+            int childPos = splitBlockPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT);
+            buffer.putInt(childPos, newChild);
+            buffer.putIntVolatile(tailPos, tail);
+            return;
+        }
+
+        int childPos = splitBlockPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT);
+        buffer.putIntVolatile(childPos, newChild);
+    }
+
+    /**
+     * Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node.
+     */
+    private int attachChildToSparse(int node, int trans, int newChild) throws SpaceExhaustedException
+    {
+        int index;
+        int smallerCount = 0;
+        // first check if this is an update and modify in-place if so
+        for (index = 0; index < SPARSE_CHILD_COUNT; ++index)
+        {
+            if (isNull(getInt(node + SPARSE_CHILDREN_OFFSET + index * 4)))
+                break;
+            final int existing = getUnsignedByte(node + SPARSE_BYTES_OFFSET + index);
+            if (existing == trans)
+            {
+                buffer.putIntVolatile(node + SPARSE_CHILDREN_OFFSET + index * 4, newChild);
+                return node;
+            }
+            else if (existing < trans)
+                ++smallerCount;
+        }
+        int childCount = index;
+
+        if (childCount == SPARSE_CHILD_COUNT)
+        {
+            // Node is full. Switch to split
+            int split = createEmptySplitNode();
+            for (int i = 0; i < SPARSE_CHILD_COUNT; ++i)
+            {
+                int t = getUnsignedByte(node + SPARSE_BYTES_OFFSET + i);
+                int p = getInt(node + SPARSE_CHILDREN_OFFSET + i * 4);
+                attachChildToSplitNonVolatile(split, t, p);
+            }
+            attachChildToSplitNonVolatile(split, trans, newChild);
+            return split;
+        }
+
+        // Add a new transition. They are not kept in order, so append it at the first free position.
+        buffer.putByte(node + SPARSE_BYTES_OFFSET + childCount, (byte) trans);
+
+        // Update order word.
+        int order = getUnsignedShort(node + SPARSE_ORDER_OFFSET);
+        int newOrder = insertInOrderWord(order, childCount, smallerCount);
+
+        // Sparse nodes have two access modes: via the order word, when listing transitions, or directly to characters
+        // and addresses.
+        // To support the former, we volatile write to the order word last, and everything is correctly set up.
+        // The latter does not touch the order word. To support that too, we volatile write the address, as the reader
+        // can't determine if the position is in use based on the character byte alone (00 is also a valid transition).
+        // Note that this means that reader must check the transition byte AFTER the address, to ensure they get the
+        // correct value (see getSparseChild).
+
+        // setting child enables reads to start seeing the new branch
+        buffer.putIntVolatile(node + SPARSE_CHILDREN_OFFSET + childCount * 4, newChild);
+
+        // some readers will decide whether to check the pointer based on the order word
+        // write that volatile to make sure they see the new change too
+        buffer.putShortVolatile(node + SPARSE_ORDER_OFFSET,  (short) newOrder);
+        return node;
+    }
+
+    /**
+     * Insert the given newIndex in the base-6 encoded order word in the correct position with respect to the ordering.
+     *
+     * E.g.
+     *   - insertOrderWord(120, 3, 0) must return 1203 (decimal 48*6 + 3)
+     *   - insertOrderWord(120, 3, 1, ptr) must return 1230 (decimal 8*36 + 3*6 + 0)
+     *   - insertOrderWord(120, 3, 2, ptr) must return 1320 (decimal 1*216 + 3*36 + 12)
+     *   - insertOrderWord(120, 3, 3, ptr) must return 3120 (decimal 3*216 + 48)
+     */
+    private static int insertInOrderWord(int order, int newIndex, int smallerCount)
+    {
+        int r = 1;
+        for (int i = 0; i < smallerCount; ++i)
+            r *= 6;
+        int head = order / r;
+        int tail = order % r;
+        // insert newIndex after the ones we have passed (order % r) and before the remaining (order / r)
+        return tail + (head * 6 + newIndex) * r;
+    }
+
+    /**
+     * Non-volatile version of attachChildToSplit. Used when the split node is not reachable yet (during the conversion
+     * from sparse).
+     */
+    private void attachChildToSplitNonVolatile(int node, int trans, int newChild) throws SpaceExhaustedException
+    {
+        assert offset(node) == SPLIT_OFFSET : "Invalid split node in trie";
+        int midPos = splitBlockPointerAddress(node, splitNodeMidIndex(trans), SPLIT_START_LEVEL_LIMIT);
+        int mid = getInt(midPos);
+        if (isNull(mid))
+        {
+            mid = createEmptySplitNode();
+            buffer.putInt(midPos, mid);
+        }
+
+        assert offset(mid) == SPLIT_OFFSET : "Invalid split node in trie";
+        int tailPos = splitBlockPointerAddress(mid, splitNodeTailIndex(trans), SPLIT_OTHER_LEVEL_LIMIT);
+        int tail = getInt(tailPos);
+        if (isNull(tail))
+        {
+            tail = createEmptySplitNode();
+            buffer.putInt(tailPos, tail);
+        }
+
+        assert offset(tail) == SPLIT_OFFSET : "Invalid split node in trie";
+        int childPos = splitBlockPointerAddress(tail, splitNodeChildIndex(trans), SPLIT_OTHER_LEVEL_LIMIT);
+        buffer.putInt(childPos, newChild);
+    }
+
+    /**
+     * Attach a child to the given chain node. This may be an update for an existing branch with different target
+     * address, or a second child for the node.
+     * This method always copies the node -- with the exception of updates that change the child of the last node in a
+     * chain block with matching transition byte (which this method is not used for, see attachChild), modifications to
+     * chain nodes cannot be done in place, either because we introduce a new transition byte and have to convert from
+     * the single-transition chain type to sparse, or because we have to remap the child from the implicit node + 1 to
+     * something else.
+     */
+    private int attachChildToChain(int node, int transitionByte, int newChild) throws SpaceExhaustedException
+    {
+        int existingByte = getUnsignedByte(node);
+        if (transitionByte == existingByte)
+        {
+            // This will only be called if new child is different from old, and the update is not on the final child
+            // where we can change it in place (see attachChild). We must always create something new.
+            // If the child is a chain, we can expand it (since it's a different value, its branch must be new and
+            // nothing can already reside in the rest of the block).
+            return expandOrCreateChainNode(transitionByte, newChild);
+        }
+
+        // The new transition is different, so we no longer have only one transition. Change type.
+        int existingChild = node + 1;
+        if (offset(existingChild) == LAST_POINTER_OFFSET)
+        {
+            existingChild = getInt(existingChild);
+        }
+        return createSparseNode(existingByte, existingChild, transitionByte, newChild);
+    }
+
+    private boolean isExpandableChain(int newChild)
+    {
+        int newOffset = offset(newChild);
+        return newChild > 0 && newChild - 1 > NONE && newOffset > CHAIN_MIN_OFFSET && newOffset <= CHAIN_MAX_OFFSET;
+    }
+
+    /**
+     * Create a sparse node with two children.
+     */
+    private int createSparseNode(int byte1, int child1, int byte2, int child2) throws SpaceExhaustedException
+    {
+        assert byte1 != byte2 : "Attempted to create a sparse node with two of the same transition";
+        if (byte1 > byte2)
+        {
+            // swap them so the smaller is byte1, i.e. there's always something bigger than child 0 so 0 never is
+            // at the end of the order
+            int t = byte1; byte1 = byte2; byte2 = t;
+            t = child1; child1 = child2; child2 = t;
+        }
+
+        int node = allocateBlock() + SPARSE_OFFSET;
+        buffer.putByte(node + SPARSE_BYTES_OFFSET + 0,  (byte) byte1);
+        buffer.putByte(node + SPARSE_BYTES_OFFSET + 1,  (byte) byte2);
+        buffer.putInt(node + SPARSE_CHILDREN_OFFSET + 0 * 4, child1);
+        buffer.putInt(node + SPARSE_CHILDREN_OFFSET + 1 * 4, child2);
+        buffer.putShort(node + SPARSE_ORDER_OFFSET,  (short) (1 * 6 + 0));
+        // Note: this does not need a volatile write as it is a new node, returning a new pointer, which needs to be
+        // put in an existing node or the root. That action ends in a happens-before enforcing write.
+        return node;
+    }
+
+    /**
+     * Creates a chain node with the single provided transition (pointing to the provided child).
+     * Note that to avoid creating inefficient tries with under-utilized chain nodes, this should only be called from
+     * {@link #expandOrCreateChainNode} and other call-sites should call {@link #expandOrCreateChainNode}.
+     */
+    private int createNewChainNode(int transitionByte, int newChild) throws SpaceExhaustedException
+    {
+        int newNode = allocateBlock() + LAST_POINTER_OFFSET - 1;
+        buffer.putByte(newNode, (byte) transitionByte);
+        buffer.putInt(newNode + 1, newChild);
+        // Note: this does not need a volatile write as it is a new node, returning a new pointer, which needs to be
+        // put in an existing node or the root. That action ends in a happens-before enforcing write.
+        return newNode;
+    }
+
+    /** Like {@link #createNewChainNode}, but if the new child is already a chain node and has room, expand
+     * it instead of creating a brand new node. */
+    private int expandOrCreateChainNode(int transitionByte, int newChild) throws SpaceExhaustedException
+    {
+        if (isExpandableChain(newChild))
+        {
+            // attach as a new character in child node
+            int newNode = newChild - 1;
+            buffer.putByte(newNode, (byte) transitionByte);
+            return newNode;
+        }
+
+        return createNewChainNode(transitionByte, newChild);
+    }
+
+    private int createEmptySplitNode() throws SpaceExhaustedException
+    {
+        return allocateBlock() + SPLIT_OFFSET;
+    }
+
+    private int createPrefixNode(int contentIndex, int child, boolean isSafeChain) throws SpaceExhaustedException
+    {
+        assert !isNullOrLeaf(child) : "Prefix node cannot reference a childless node.";
+
+        int offset = offset(child);
+        int node;
+        if (offset == SPLIT_OFFSET || isSafeChain && offset > (PREFIX_FLAGS_OFFSET + PREFIX_OFFSET) && offset <= CHAIN_MAX_OFFSET)
+        {
+            // We can do an embedded prefix node
+            // Note: for chain nodes we have a risk that the node continues beyond the current point, in which case
+            // creating the embedded node may overwrite information that is still needed by concurrent readers or the
+            // mutation process itself.
+            node = (child & -BLOCK_SIZE) | PREFIX_OFFSET;
+            buffer.putByte(node + PREFIX_FLAGS_OFFSET, (byte) offset);
+        }
+        else
+        {
+            // Full prefix node
+            node = allocateBlock() + PREFIX_OFFSET;
+            buffer.putByte(node + PREFIX_FLAGS_OFFSET, (byte) 0xFF);
+            buffer.putInt(node + PREFIX_POINTER_OFFSET, child);
+        }
+
+        buffer.putInt(node + PREFIX_CONTENT_OFFSET, contentIndex);
+        return node;
+    }
+
+    private int updatePrefixNodeChild(int node, int child) throws SpaceExhaustedException
+    {
+        assert offset(node) == PREFIX_OFFSET : "updatePrefix called on non-prefix node";
+        assert !isNullOrLeaf(child) : "Prefix node cannot reference a childless node.";
+
+        // We can only update in-place if we have a full prefix node
+        if (!isEmbeddedPrefixNode(node))
+        {
+            // This attaches the child branch and makes it reachable -- the write must be volatile.
+            buffer.putIntVolatile(node + PREFIX_POINTER_OFFSET, child);
+            return node;
+        }
+        else
+        {
+            int contentIndex = getInt(node + PREFIX_CONTENT_OFFSET);
+            return createPrefixNode(contentIndex, child, true);
+        }
+    }
+
+    private boolean isEmbeddedPrefixNode(int node)
+    {
+        return getUnsignedByte(node + PREFIX_FLAGS_OFFSET) < BLOCK_SIZE;
+    }
+
+    /**
+     * Copy the content from an existing node, if it has any, to a newly-prepared update for its child.
+     *
+     * @param existingPreContentNode pointer to the existing node before skipping over content nodes, i.e. this is
+     *                               either the same as existingPostContentNode or a pointer to a prefix or leaf node
+     *                               whose child is existingPostContentNode
+     * @param existingPostContentNode pointer to the existing node being updated, after any content nodes have been
+     *                                skipped and before any modification have been applied; always a non-content node
+     * @param updatedPostContentNode is the updated node, i.e. the node to which all relevant modifications have been
+     *                               applied; if the modifications were applied in-place, this will be the same as
+     *                               existingPostContentNode, otherwise a completely different pointer; always a non-
+     *                               content node
+     * @return a node which has the children of updatedPostContentNode combined with the content of
+     *         existingPreContentNode
+     */
+    private int preserveContent(int existingPreContentNode,
+                                int existingPostContentNode,
+                                int updatedPostContentNode) throws SpaceExhaustedException
+    {
+        if (existingPreContentNode == existingPostContentNode)
+            return updatedPostContentNode;     // no content to preserve
+
+        if (existingPostContentNode == updatedPostContentNode)
+            return existingPreContentNode;     // child didn't change, no update necessary
+
+        // else we have existing prefix node, and we need to reference a new child
+        if (isLeaf(existingPreContentNode))
+        {
+            return createPrefixNode(~existingPreContentNode, updatedPostContentNode, true);
+        }
+
+        assert offset(existingPreContentNode) == PREFIX_OFFSET : "Unexpected content in non-prefix and non-leaf node.";
+        return updatePrefixNodeChild(existingPreContentNode, updatedPostContentNode);
+    }
+
+    final ApplyState applyState = new ApplyState();
+
+    /**
+     * Represents the state for an {@link #apply} operation. Contains a stack of all nodes we descended through
+     * and used to update the nodes with any new data during ascent.
+     *
+     * To make this as efficient and GC-friendly as possible, we use an integer array (instead of is an object stack)
+     * and we reuse the same object. The latter is safe because memtable tries cannot be mutated in parallel by multiple
+     * writers.
+     */
+    class ApplyState
+    {
+        int[] data = new int[16 * 5];
+        int currentDepth = -1;
+
+        void reset()
+        {
+            currentDepth = -1;
+        }
+
+        /**
+         * Pointer to the existing node before skipping over content nodes, i.e. this is either the same as
+         * existingPostContentNode or a pointer to a prefix or leaf node whose child is existingPostContentNode.
+         */
+        int existingPreContentNode()
+        {
+            return data[currentDepth * 5 + 0];
+        }
+        void setExistingPreContentNode(int value)
+        {
+            data[currentDepth * 5 + 0] = value;
+        }
+
+        /**
+         * Pointer to the existing node being updated, after any content nodes have been skipped and before any
+         * modification have been applied. Always a non-content node.
+         */
+        int existingPostContentNode()
+        {
+            return data[currentDepth * 5 + 1];
+        }
+        void setExistingPostContentNode(int value)
+        {
+            data[currentDepth * 5 + 1] = value;
+        }
+
+        /**
+         * The updated node, i.e. the node to which the relevant modifications are being applied. This will change as
+         * children are processed and attached to the node. After all children have been processed, this will contain
+         * the fully updated node (i.e. the union of existingPostContentNode and mutationNode) without any content,
+         * which will be processed separately and, if necessary, attached ahead of this. If the modifications were
+         * applied in-place, this will be the same as existingPostContentNode, otherwise a completely different
+         * pointer. Always a non-content node.
+         */
+        int updatedPostContentNode()
+        {
+            return data[currentDepth * 5 + 2];
+        }
+        void setUpdatedPostContentNode(int value)
+        {
+            data[currentDepth * 5 + 2] = value;
+        }
+
+        /**
+         * The transition we took on the way down.
+         */
+        int transition()
+        {
+            return data[currentDepth * 5 + 3];
+        }
+        void setTransition(int transition)
+        {
+            data[currentDepth * 5 + 3] = transition;
+        }
+
+        /**
+         * The compiled content index. Needed because we can only access a cursor's content on the way down but we can't
+         * attach it until we ascend from the node.
+         */
+        int contentIndex()
+        {
+            return data[currentDepth * 5 + 4];
+        }
+        void setContentIndex(int value)
+        {
+            data[currentDepth * 5 + 4] = value;
+        }
+
+        /**
+         * Descend to a child node. Prepares a new entry in the stack for the node.
+         */
+        <U> void descend(int transition, U mutationContent, final UpsertTransformer<T, U> transformer)
+        {
+            int existingPreContentNode;
+            if (currentDepth < 0)
+                existingPreContentNode = root;
+            else
+            {
+                setTransition(transition);
+                existingPreContentNode = isNull(existingPostContentNode())
+                                         ? NONE
+                                         : getChild(existingPostContentNode(), transition);
+            }
+
+            ++currentDepth;
+            if (currentDepth * 5 >= data.length)
+                data = Arrays.copyOf(data, currentDepth * 5 * 2);
+            setExistingPreContentNode(existingPreContentNode);
+
+            int existingContentIndex = -1;
+            int existingPostContentNode;
+            if (isLeaf(existingPreContentNode))
+            {
+                existingContentIndex = ~existingPreContentNode;
+                existingPostContentNode = NONE;
+            }
+            else if (offset(existingPreContentNode) == PREFIX_OFFSET)
+            {
+                existingContentIndex = getInt(existingPreContentNode + PREFIX_CONTENT_OFFSET);
+                existingPostContentNode = followContentTransition(existingPreContentNode);
+            }
+            else
+                existingPostContentNode = existingPreContentNode;
+            setExistingPostContentNode(existingPostContentNode);
+            setUpdatedPostContentNode(existingPostContentNode);
+
+            int contentIndex = updateContentIndex(mutationContent, existingContentIndex, transformer);
+            setContentIndex(contentIndex);
+        }
+
+        /**
+         * Combine existing and new content.
+         */
+        private <U> int updateContentIndex(U mutationContent, int existingContentIndex, final UpsertTransformer<T, U> transformer)
+        {
+            if (mutationContent != null)
+            {
+                if (existingContentIndex != -1)
+                {
+                    final T existingContent = contentArray.get(existingContentIndex);
+                    T combinedContent = transformer.apply(existingContent, mutationContent);
+                    assert (combinedContent != null) : "Transformer cannot be used to remove content.";
+                    setContent(existingContentIndex, combinedContent);
+                    return existingContentIndex;
+                }
+                else
+                {
+                    T combinedContent = transformer.apply(null, mutationContent);
+                    assert (combinedContent != null) : "Transformer cannot be used to remove content.";
+                    return addContent(combinedContent);
+                }
+            }
+            else
+                return existingContentIndex;
+        }
+
+        /**
+         * Attach a child to the current node.
+         */
+        private void attachChild(int transition, int child) throws SpaceExhaustedException
+        {
+            int updatedPostContentNode = updatedPostContentNode();
+            if (isNull(updatedPostContentNode))
+                setUpdatedPostContentNode(expandOrCreateChainNode(transition, child));
+            else
+                setUpdatedPostContentNode(MemtableTrie.this.attachChild(updatedPostContentNode,
+                                                                        transition,
+                                                                        child));
+        }
+
+        /**
+         * Apply the collected content to a node. Converts NONE to a leaf node, and adds or updates a prefix for all
+         * others.
+         */
+        private int applyContent() throws SpaceExhaustedException
+        {
+            int contentIndex = contentIndex();
+            int updatedPostContentNode = updatedPostContentNode();
+            if (contentIndex == -1)
+                return updatedPostContentNode;
+
+            if (isNull(updatedPostContentNode))
+                return ~contentIndex;
+
+            int existingPreContentNode = existingPreContentNode();
+            int existingPostContentNode = existingPostContentNode();
+
+            // We can't update in-place if there was no preexisting prefix, or if the prefix was embedded and the target
+            // node must change.
+            if (existingPreContentNode == existingPostContentNode ||
+                isNull(existingPostContentNode) ||
+                isEmbeddedPrefixNode(existingPreContentNode) && updatedPostContentNode != existingPostContentNode)
+                return createPrefixNode(contentIndex, updatedPostContentNode, isNull(existingPostContentNode));
+
+            // Otherwise modify in place
+            if (updatedPostContentNode != existingPostContentNode) // to use volatile write but also ensure we don't corrupt embedded nodes
+                buffer.putIntVolatile(existingPreContentNode + PREFIX_POINTER_OFFSET, updatedPostContentNode);
+            assert contentIndex == getInt(existingPreContentNode + PREFIX_CONTENT_OFFSET) : "Unexpected change of content index.";
+            return existingPreContentNode;
+        }
+
+        /**
+         * After a node's children are processed, this is called to ascend from it. This means applying the collected
+         * content to the compiled updatedPostContentNode and creating a mapping in the parent to it (or updating if
+         * one already exists).
+         * Returns true if still have work to do, false if the operation is completed.
+         */
+        private boolean attachAndMoveToParentState() throws SpaceExhaustedException
+        {
+            int updatedPreContentNode = applyContent();
+            int existingPreContentNode = existingPreContentNode();
+            --currentDepth;
+            if (currentDepth == -1)
+            {
+                assert root == existingPreContentNode : "Unexpected change to root. Concurrent trie modification?";
+                if (updatedPreContentNode != existingPreContentNode)
+                {
+                    // Only write to root if they are different (value doesn't change, but
+                    // we don't want to invalidate the value in other cores' caches unnecessarily).
+                    root = updatedPreContentNode;
+                }
+                return false;
+            }
+            if (updatedPreContentNode != existingPreContentNode)
+                attachChild(transition(), updatedPreContentNode);
+            return true;
+        }
+    }
+
+    /**
+     * Somewhat similar to {@link MergeResolver}, this encapsulates logic to be applied whenever new content is being
+     * upserted into a {@link MemtableTrie}. Unlike {@link MergeResolver}, {@link UpsertTransformer} will be applied no
+     * matter if there's pre-existing content for that trie key/path or not.
+     *
+     * @param <T> The content type for this {@link MemtableTrie}.
+     * @param <U> The type of the new content being applied to this {@link MemtableTrie}.
+     */
+    public interface UpsertTransformer<T, U>
+    {
+        /**
+         * Called when there's content in the updating trie.
+         *
+         * @param existing Existing content for this key, or null if there isn't any.
+         * @param update   The update, always non-null.
+         * @return The combined value to use. Cannot be null.
+         */
+        T apply(T existing, U update);
+    }
+
+    /**
+     * Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved
+     * with the given function before being placed in this trie (even if there's no pre-existing content in this trie).
+     * @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type
+     * different than the element type for this memtable trie.
+     * @param transformer a function applied to the potentially pre-existing value for the given key, and the new
+     * value. Applied even if there's no pre-existing value in the memtable trie.
+     */
+    public <U> void apply(Trie<U> mutation, final UpsertTransformer<T, U> transformer) throws SpaceExhaustedException
+    {
+        Cursor<U> mutationCursor = mutation.cursor();
+        assert mutationCursor.depth() == 0 : "Unexpected non-fresh cursor.";
+        ApplyState state = applyState;
+        state.reset();
+        state.descend(-1, mutationCursor.content(), transformer);
+        assert state.currentDepth == 0 : "Unexpected change to applyState. Concurrent trie modification?";
+
+        while (true)
+        {
+            int depth = mutationCursor.advance();
+            while (state.currentDepth >= depth)
+            {
+                // There are no more children. Ascend to the parent state to continue walk.
+                if (!state.attachAndMoveToParentState())
+                {
+                    assert depth == -1 : "Unexpected change to applyState. Concurrent trie modification?";
+                    return;
+                }
+            }
+
+            // We have a transition, get child to descend into
+            state.descend(mutationCursor.incomingTransition(), mutationCursor.content(), transformer);
+            assert state.currentDepth == depth : "Unexpected change to applyState. Concurrent trie modification?";
+        }
+    }
+
+    /**
+     * Map-like put method, using the apply machinery above which cannot run into stack overflow. When the correct
+     * position in the trie has been reached, the value will be resolved with the given function before being placed in
+     * the trie (even if there's no pre-existing content in this trie).
+     * @param key the trie path/key for the given value.
+     * @param value the value being put in the memtable trie. Note that it can be of type different than the element
+     * type for this memtable trie. It's up to the {@code transformer} to return the final value that will stay in
+     * the memtable trie.
+     * @param transformer a function applied to the potentially pre-existing value for the given key, and the new
+     * value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied
+     * even if there's no pre-existing value in the memtable trie.
+     */
+    public <R> void putSingleton(ByteComparable key,
+                                 R value,
+                                 UpsertTransformer<T, ? super R> transformer) throws SpaceExhaustedException
+    {
+        apply(Trie.singleton(key, value), transformer);
+    }
+
+    /**
+     * A version of putSingleton which uses recursive put if the last argument is true.
+     */
+    public <R> void putSingleton(ByteComparable key,
+                                 R value,
+                                 UpsertTransformer<T, ? super R> transformer,
+                                 boolean useRecursive) throws SpaceExhaustedException
+    {
+        if (useRecursive)
+            putRecursive(key, value, transformer);
+        else
+            putSingleton(key, value, transformer);
+    }
+
+    /**
+     * Map-like put method, using a fast recursive implementation through the key bytes. May run into stack overflow if
+     * the trie becomes too deep. When the correct position in the trie has been reached, the value will be resolved
+     * with the given function before being placed in the trie (even if there's no pre-existing content in this trie).
+     * @param key the trie path/key for the given value.
+     * @param value the value being put in the memtable trie. Note that it can be of type different than the element
+     * type for this memtable trie. It's up to the {@code transformer} to return the final value that will stay in
+     * the memtable trie.
+     * @param transformer a function applied to the potentially pre-existing value for the given key, and the new
+     * value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied
+     * even if there's no pre-existing value in the memtable trie.
+     */
+    public <R> void putRecursive(ByteComparable key, R value, final UpsertTransformer<T, R> transformer) throws SpaceExhaustedException
+    {
+        int newRoot = putRecursive(root, key.asComparableBytes(BYTE_COMPARABLE_VERSION), value, transformer);
+        if (newRoot != root)
+            root = newRoot;
+    }
+
+    private <R> int putRecursive(int node, ByteSource key, R value, final UpsertTransformer<T, R> transformer) throws SpaceExhaustedException
+    {
+        int transition = key.next();
+        if (transition == ByteSource.END_OF_STREAM)
+            return applyContent(node, value, transformer);
+
+        int child = getChild(node, transition);
+
+        int newChild = putRecursive(child, key, value, transformer);
+        if (newChild == child)
+            return node;
+
+        int skippedContent = followContentTransition(node);
+        int attachedChild = !isNull(skippedContent)
+                            ? attachChild(skippedContent, transition, newChild)  // Single path, no copying required
+                            : expandOrCreateChainNode(transition, newChild);
+
+        return preserveContent(node, skippedContent, attachedChild);
+    }
+
+    private <R> int applyContent(int node, R value, UpsertTransformer<T, R> transformer) throws SpaceExhaustedException
+    {
+        if (isNull(node))
+            return ~addContent(transformer.apply(null, value));
+
+        if (isLeaf(node))
+        {
+            int contentIndex = ~node;
+            setContent(contentIndex, transformer.apply(getContent(contentIndex), value));
+            return node;
+        }
+
+        if (offset(node) == PREFIX_OFFSET)
+        {
+            int contentIndex = getInt(node + PREFIX_CONTENT_OFFSET);
+            setContent(contentIndex, transformer.apply(getContent(contentIndex), value));
+            return node;
+        }
+        else
+            return createPrefixNode(addContent(transformer.apply(null, value)), node, false);
+    }
+
+    /**
+     * Returns true if the allocation threshold has been reached. To be called by the the writing thread (ideally, just
+     * after the write completes). When this returns true, the user should switch to a new trie as soon as feasible.
+     *
+     * The trie expects up to 10% growth above this threshold. Any growth beyond that may be done inefficiently, and
+     * the trie will fail altogether when the size grows beyond 2G - 256 bytes.
+     */
+    public boolean reachedAllocatedSizeThreshold()
+    {
+        return allocatedPos >= ALLOCATED_SIZE_THRESHOLD;
+    }
+
+    /**
+     * For tests only! Advance the allocation pointer (and allocate space) by this much to test behaviour close to
+     * full.
+     */
+    @VisibleForTesting
+    int advanceAllocatedPos(int wantedPos) throws SpaceExhaustedException
+    {
+        while (allocatedPos < wantedPos)
+            allocateBlock();
+        return allocatedPos;
+    }
+
+    /** Returns the off heap size of the memtable trie itself, not counting any space taken by referenced content. */
+    public long sizeOffHeap()
+    {
+        return bufferType == BufferType.ON_HEAP ? 0 : allocatedPos;
+    }
+
+    /** Returns the on heap size of the memtable trie itself, not counting any space taken by referenced content. */
+    public long sizeOnHeap()
+    {
+        return contentCount * MemoryLayoutSpecification.SPEC.getReferenceSize() +
+               (bufferType == BufferType.ON_HEAP ? allocatedPos + EMPTY_SIZE_ON_HEAP : EMPTY_SIZE_OFF_HEAP);
+    }
+
+    @Override
+    public Iterable<T> valuesUnordered()
+    {
+        return () -> new Iterator<T>()
+        {
+            int idx = 0;
+
+            public boolean hasNext()
+            {
+                return idx < contentCount;
+            }
+
+            public T next()
+            {
+                if (!hasNext())
+                    throw new NoSuchElementException();
+
+                return getContent(idx++);
+            }
+        };
+    }
+
+    public int valuesCount()
+    {
+        return contentCount;
+    }
+
+    public long unusedReservedMemory()
+    {
+        int bufferOverhead = 0;
+        if (bufferType == BufferType.ON_HEAP)
+            bufferOverhead = buffer.capacity() - this.allocatedPos;
+
+        int contentOverhead = (contentArray.length() - contentCount) * MemoryLayoutSpecification.SPEC.getReferenceSize();
+
+        return bufferOverhead + contentOverhead;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md
new file mode 100644
index 0000000000..901e29ce16
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md
@@ -0,0 +1,753 @@
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+ 
+     http://www.apache.org/licenses/LICENSE-2.0
+ 
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+# MemtableTrie Design
+
+The `MemtableTrie` is one of the main components of the trie infrastructure, a mutable in-memory trie built for fast
+modification and reads executing concurrently with writes from a single mutator thread.
+
+The main features of its implementation are:
+- full support of the `Trie` interface
+- using nodes of several different types for efficiency
+- support for content on any node, including intermediate (prefix)
+- support for writes from a single mutator thread concurrent with multiple readers
+- maximum trie size of 2GB
+
+
+## Memory layout
+
+One of the main design drivers of the memtable trie is the desire to avoid on-heap storage and Java object management.
+The trie thus implements its own memory management for the structure of the trie (content is, at this time, still given
+as Java objects in a content array). The structure resides in one `UnsafeBuffer` (which can be on or off heap as
+desired) and is broken up in 32-byte "cells" (also called "blocks" in the code), which are the unit of allocation,
+update and reuse.
+
+Like all tries, `MemtableTrie` is built from nodes and has a root pointer. The nodes reside in cells, but there is no
+1:1 correspondence between nodes and cells - some node types pack multiple in one cell, while other types require
+multiple cells.
+
+### Pointers and node types
+
+A "pointer" is an integer that points to a node in the trie buffer. A pointer specifies the location of the node
+(its starting cell), but also defines the type of node in its 5 lowest-order bits (i.e. the offset within the cell).
+If a pointer has a negative value, it refers to a value in the content array, and implies a leaf node with the specified
+content. Additionally, the special pointer value `NONE` (0) is used to specify "no child". We use 32-bit integers as
+pointers, therefore the size of the trie structure is limited to a little less than 2GB.
+
+For example, the pointer `0x0109E` specifies a node residing in the cell at bytes `0x01080`-`0x0109F` in the buffer
+(specified by the pointers' 27 leading bits), where the node type is `Sparse` (specified by `0x1E` in the last 5 bits).
+
+The pointer `0xFFFFFFF0` specifies a leaf node (being negative), where the content's index is `0xF` (obtained by
+negating all bits of the pointer).
+
+To save space and reduce pointer chasing, we use several different types of nodes that address different common patterns
+in a trie. It is common for a trie to have one or a couple of top levels which have many children, and where it is
+important to make decisions with as few if-then-else branches as possible (served by the `Split` type), another one or
+two levels of nodes with a small number of children, where it is most important to save space as the number of these
+nodes is high (served by the `Sparse` type), and a lot of sequences of single-child nodes containing the trailing bytes
+of the key or of some common key prefix (served by the `Chain` type). Most of the payload/content of the trie resides
+at the leaves, where it makes sense to avoid taking any space for a node (the `Leaf` type), but we must also allow the
+possibility for values to be present in intermediate nodes &mdash; because this is rare, we support it with a special
+`Prefix` type instead of reserving a space for payload in all other node types.
+
+The Split-Sparse-Chain-Leaf/Prefix pattern may repeat several times. For example, we could have these four layers for
+the partition key with some metadata associated with the partition, then for the first component of the clustering key,
+then for the second component etc.
+
+The sections below specify the layout of each supported node type.
+
+#### Leaf nodes
+
+Leaf nodes do not have a corresponding cell in the buffer. Instead, they reference a value (i.e. a POJO in the
+`MemtableTrie`'s content type) in the content array. The index of the value is specified by `~pointer` (unlike `-x`,
+`~x` allows one to also encode 0 in a negative number).
+
+Leaf nodes have no children, and return the specified value for `content()`.
+
+Example: -1 is a leaf cell with content `contentArray[0]`.
+
+#### `Chain` nodes - single path, multiple transitions in one cell
+
+Chain nodes are one-child nodes. Multiple chain nodes, forming a chain of transitions to one target, can reside in a
+single cell. Chain nodes are identified by the lowest 5 bits of a pointer being between `0x00` and `0x1B`. In addition
+to the the type of node, in this case the bits also define the length of the chain &mdash; the difference between
+`0x1C` and the pointer offset specifies the number of characters in the chain.
+
+The simplest chain node has one transition leading to one child and is laid out like this:
+
+offset|content|example
+---|---|---
+00 - 1A|unused|
+1B     |character|41 A
+1C - 1F|child pointer|FFFFFFFF
+
+where the pointer points to the `1B` line in the cell.
+
+Example: The cell `xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxx41 FFFFFFFF` at bytes `0x120`-`0x13F` and
+pointer `0x13B` point to a node with one child with transition `0x41` `A` to a leaf node with content `contentArray[0]`.
+
+Another chain cell, which points to this one, can be added in the same cell by placing a character at offset `1A`. This
+new node is effectively laid out as
+
+offset|content|example
+---|---|---
+00 - 19|unused|
+1A     |character|48 H
+1B - 1F|unused|
+
+where the pointer points to line `1A`. This node has one transition, and the child pointer is implicit as the node's
+pointer plus one.
+
+This can continue until all the bytes in the "unused" area are filled.
+
+Example: The cell `xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xx434841 FFFFFFFF` at bytes `0x120`-`0x13F` and
+pointer `0x139` point to a node with one child with transition `0x43` `C` to a node with one child with transition
+`0x48` `H` to a node with one child with transition `0x41` `A` to a leaf node with content `contentArray[0]`.
+
+offset|content|example
+---|---|---
+00 - 18|unused|
+19     |character|43 C
+1A     |character|48 H
+1B     |character|41 A
+1C - 1F|child pointer|FFFFFFFF
+
+
+In this example `0x13A` and `0x13B` are also valid pointers to the respective chains and could be referenced from other
+nodes (an example will be given below). In any case, the byte pointed directly by the node pointer contains the
+transition byte. The child pointer is either `pointer + 1` (if the lowest 5 pointer bits are less than `0x1B`), or the
+integer stored at `pointer + 1` (if the pointer's last 5 bits are `0x1B`).
+
+![graph](MemtableTrie.md.g1.svg)
+
+Note: offset `0x00` also specifies a chain node, but the pointer 0 is a special case and care must be taken to ensure no
+28-byte chain node is placed in the cell at bytes `0x00`-`0x1F`.
+
+#### `Sparse` nodes - between 2 and 6 children in one cell
+
+Sparse nodes are used when a node has at least two children, and all pointers and transition characters can fit in one
+cell, which limits the maximum number of children to 6. Their layout is:
+
+offset|content|
+---|---|
+00 - 03|child pointer 0|
+04 - 07|child pointer 1|
+08 - 0B|child pointer 2|
+0C - 0F|child pointer 3|
+10 - 13|child pointer 4|
+14 - 17|child pointer 5|
+18     |character 0|
+19     |character 1|
+1A     |character 2|
+1B     |character 3|
+1C     |character 4|
+1D     |character 5|
+1E - 1F|order word|
+
+where the pointer points to the line `1E` (i.e. the type identifier for a sparse node is `0x1E`).
+
+It is important to note that the pointers and characters are not in order. This is done so that an update to a sparse
+node where a new child is inserted can be done while the previous state of the node is still valid and readable for
+any concurrent readers. Instead, new children are appended, and the order is maintained in the "order word". This word
+is a number whose digits specify the order of the children's transition characters (where higher-order digits specify
+bigger characters) encoded, to be able to fit into a 16-bit word, in base 6. Its number of digits also specifies the
+number of children of the node.
+
+To explain this better, we will give an example of the evolution of a sparse node. Suppose we had the `0x139` node from
+the previous section, and some update needs to attach a second child to that, e.g. with the character `A` and child
+`0x238`.
+
+![graph](MemtableTrie.md.g2.svg)
+
+To do this, the mutating thread will have to convert the chain node into a sparse by allocating a new cell
+(e.g. `0x240`-`0x25F`) and filling in the sparse node `00000238 0000013A 00000000 00000000 00000000 00000000 41430000
+00000006` with pointer `0x25E`:
+
+offset|content|example
+---|---|---
+00 - 03|child pointer 0| 00000238
+04 - 07|child pointer 1| 0000013A
+08 - 17|unused|
+18     |character 0| 41 A
+19     |character 1| 43 C
+1A - 1D|unused|
+1E - 1F|order word, always 10| 0006 = 10 (base 6)
+
+This is the smallest kind of sparse node, with just two children. Two-children sparse nodes always
+put their two children in order (we can do this as this does not happen in response to an addition of a new child to
+an existing sparse node, but this is constructed directly) and thus their order word is always 10 (if they were
+not in order, the order word would have to be 01, which would be misinterpreted as the single-digit 1).
+
+This node has two (the number of digits in the order word) children. The first child is at the position specified by the
+least significant digit of the order word, 0. The second child is specified by the second least significant digit, 1.
+
+Suppose we then need to add a new child, using character `0x35` `5` and child `0x33B`. The node will change to `00000238
+0000013A 0000033B 00000000 00000000 00000000 41433500 00000026` and the pointer to it stays the same.
+
+offset|content|example
+---|---|---
+00 - 03|child pointer 0| 00000238
+04 - 07|child pointer 1| 0000013A
+08 - 0B|child pointer 2| 0000033B
+0C - 17|unused|
+18     |character 0| 41 A
+19     |character 1| 43 C
+1A     |character 2| 35 5
+1B - 1D|unused|
+1E - 1F|order word| 0026 = 102 (base 6)
+
+This node has three (the number of digits in the order word) children. The first child is at the position specified by
+the least significant digit of the order word, 2. The second child is specified by the second least significant digit,
+0, and the last child is specified by the leading digit, 1.
+
+Note that because of the ordering of the two children in the smallest sparse node, the digit 0 is always preceded by a
+more-significant 1 in the order word in base 6. Therefore the leading digit of the order word can never be 0 and thus we
+cannot miscount the number of children.
+
+The addition of children can continue until we have 6, for example `00000238 0000013A 0000033B 0000035C 0000037A
+0000041B 41433542 50338129` (pointer `0x25E`) for
+
+offset|content|example
+---|---|---
+00 - 03|child pointer 0| 00000238
+04 - 07|child pointer 1| 0000013A
+08 - 0B|child pointer 2| 0000033B
+0C - 0F|child pointer 3| 0000035C
+10 - 13|child pointer 4| 0000037A
+14 - 17|child pointer 5| 0000041B
+18     |character 0| 41 A
+19     |character 1| 43 C
+1A     |character 2| 35 5
+1B     |character 3| 42 B
+1C     |character 4| 50 P
+1D     |character 5| 33 3
+1E - 1F|order word| 8129 = 413025 (base 6)
+
+Beyond 6 children, a node needs to be converted to split.
+
+#### `Split` nodes - up to 256 children in multiple cells
+
+Split nodes are used to handle the nodes with a large number of children. We can only allocate cells of 32 bytes, thus
+we have to distribute the child transitions among cells in some way that is efficient for reading and updating. The
+method we chose is to construct a "mini-trie" with 2-3-3 bit transitions.
+
+A split node is identified by the `0x1C` offset. The starting cell of a split node has this layout:
+
+offset|content|
+---|---|
+00 - 0F|unused|
+10 - 13|mid-cell for leading 00|
+14 - 17|mid-cell for leading 01|
+18 - 1B|mid-cell for leading 10|
+1C - 1F|mid-cell for leading 11|
+
+(pointers to this node point to the `1C` line) and where each mid-cell contains:
+
+offset|content|
+---|---|
+00 - 03|end-cell for middle 000|
+04 - 07|end-cell for middle 001|
+08 - 0B|end-cell for middle 010|
+0C - 0F|end-cell for middle 011|
+10 - 13|end-cell for middle 100|
+14 - 17|end-cell for middle 101|
+18 - 1B|end-cell for middle 110|
+1C - 1F|end-cell for middle 111|
+
+and end-cell:
+
+offset|content|
+---|---|
+00 - 03|pointer to child for ending 000|
+04 - 07|pointer to child for ending 001|
+08 - 0B|pointer to child for ending 010|
+0C - 0F|pointer to child for ending 011|
+10 - 13|pointer to child for ending 100|
+14 - 17|pointer to child for ending 101|
+18 - 1B|pointer to child for ending 110|
+1C - 1F|pointer to child for ending 111|
+
+In any of the cell or pointer positions we can have `NONE`, meaning that such a child (or block of children) does not
+exist. At minimum, a split node occupies 3 cells (one leading, one mid and one end), and at maximum &mdash;
+`1 + 4 + 4*8 = 37` cells i.e. `1184` bytes. If we could allocate contiguous arrays, a full split node would use `1024`
+bytes, thus this splitting can add ~15% overhead. However, real data often has additional structure that this can make
+use of to avoid creating some of the blocks, e.g. if the trie encodes US-ASCII or UTF-encoded strings where some
+character ranges are not allowed at all, and others are prevalent. Another benefit is that to change a transition while
+preserving the previous state of the node for concurrent readers we have to only copy three blocks and not the entire
+range of children (applications of this will be given later).
+
+As an example, suppose we need to add a `0x51` `Q` transition to `0x455` to the 6-children sparse node from the previous
+section. This will generate the following structure:
+
+Leading cell (e.g. `0x500`-`0x51F` with pointer `0x51C`)
+
+offset|content|example
+---|---|---
+00 - 0F|unused|
+10 - 13|mid-cell for leading 00|0000053C
+14 - 17|mid-cell for leading 01|0000057C
+18 - 1B|mid-cell for leading 10|00000000 NONE
+1C - 1F|mid-cell for leading 11|00000000 NONE
+
+Mid cell `00` at `0x520`-`0x53F`:
+
+offset|content|example
+---|---|---
+00 - 03|end-cell for middle 000|00000000 NONE
+04 - 07|end-cell for middle 001|00000000 NONE
+08 - 0B|end-cell for middle 010|00000000 NONE
+0C - 0F|end-cell for middle 011|00000000 NONE
+10 - 13|end-cell for middle 100|00000000 NONE
+14 - 17|end-cell for middle 101|00000000 NONE
+18 - 1B|end-cell for middle 110|0000055C
+1C - 1F|end-cell for middle 111|00000000 NONE
+
+End cell `00 110` at `0x540`-`0x55F`:
+
+offset|content|example
+---|---|---
+00 - 03|pointer to child for ending 000|00000000 NONE
+04 - 07|pointer to child for ending 001|00000000 NONE
+08 - 0B|pointer to child for ending 010|00000000 NONE
+0C - 0F|pointer to child for ending 011|0000041B
+10 - 13|pointer to child for ending 100|00000000 NONE
+14 - 17|pointer to child for ending 101|0000033B
+18 - 1B|pointer to child for ending 110|00000000 NONE
+1C - 1F|pointer to child for ending 111|00000000 NONE
+
+Mid cell `01` at `0x560`-`0x57F`:
+
+offset|content|example
+---|---|---
+00 - 03|end-cell for middle 000|0000059C
+04 - 07|end-cell for middle 001|00000000 NONE
+08 - 0B|end-cell for middle 010|000005BC
+0C - 0F|end-cell for middle 011|00000000 NONE
+10 - 13|end-cell for middle 100|00000000 NONE
+14 - 17|end-cell for middle 101|00000000 NONE
+18 - 1B|end-cell for middle 110|00000000 NONE
+1C - 1F|end-cell for middle 111|00000000 NONE
+
+End cell `01 000` at `0x580`-`0x59F`:
+
+offset|content|example
+---|---|---
+00 - 03|pointer to child for ending 000|00000000 NONE
+04 - 07|pointer to child for ending 001|00000238
+08 - 0B|pointer to child for ending 010|0000035C
+0C - 0F|pointer to child for ending 011|0000013A
+10 - 13|pointer to child for ending 100|00000000 NONE
+14 - 17|pointer to child for ending 101|00000000 NONE
+18 - 1B|pointer to child for ending 110|00000000 NONE
+1C - 1F|pointer to child for ending 111|00000000 NONE
+
+End cell `01 010` at `0x5A0`-`0x5BF`:
+
+offset|content|example
+---|---|---
+00 - 03|pointer to child for ending 000|0000037A
+04 - 07|pointer to child for ending 001|00000455
+08 - 0B|pointer to child for ending 010|00000000 NONE
+0C - 0F|pointer to child for ending 011|00000000 NONE
+10 - 13|pointer to child for ending 100|00000000 NONE
+14 - 17|pointer to child for ending 101|00000000 NONE
+18 - 1B|pointer to child for ending 110|00000000 NONE
+1C - 1F|pointer to child for ending 111|00000000 NONE
+
+To find a child in this structure, we follow the transitions along the bits of the mini-trie. For example, for `0x42`
+`B` = `0b01000010` we start at `0x51C`, take the `01` pointer to `0x57C`, then the `000` pointer to `0x59C` and finally
+the `010` index to retrieve the node pointer `0x35C`. Note that the intermediate cells (dashed in the diagram) are not
+reachable with pointers, they only make sense as substructure of the split node.
+
+![graph](MemtableTrie.md.g3.svg)
+
+#### Content `Prefix`
+
+Prefix nodes are not nodes in themselves, but they add information to the node they lead to. Specifically, they
+encode an index in the content array, and a pointer to the node to which this content is attached. In anything other
+than the content, they are equivalent to the linked node &mdash; i.e. a prefix node pointer has the same children as
+the node it links to (another way to see this is as a content-carrying node is one that has an _ε_ transition to the
+linked node and no other features except added content). We do not allow more than one prefix to a node (i.e. prefix
+can't point to another prefix), and the child of a prefix node cannot be a leaf.
+
+There are two types of prefixes:
+- standalone, which has a full 32-bit pointer to the linked node
+- embedded, which occupies unused space in `Chain` or `Split` nodes and specifies the 5-bit offset within the same cell
+of the linked node
+
+Standalone prefixes have this layout:
+
+offset|content|example
+---|---|---
+00 - 03|content index|00000001
+04|standalone flag, 0xFF|FF
+05 - 1B|unused|
+1C - 1F|linked node pointer|0000025E
+
+and pointer offset `0x1F`. The sample values above will be the ones used to link a prefix node to our `Sparse`
+example, where a prefix cannot be embedded as all the bytes of the cell are in use.
+
+If we want to attach the same prefix to the `Split` example, we will place this
+
+offset|content|example
+---|---|---
+00 - 03|content index|00000001
+04|embedded offset within cell|1C
+05 - 1F|unused|
+
+_inside_ the leading split cell, with pointer `0x1F`. Since this is an embedded node, the augmented one resides within
+the same cell, and thus we need only 5 bits to encode the pointer (the other 27 are the same as the prefix's).
+The combined content of the cell at `0x500-0x51F` will then be `00000001 1C000000 00000000 00000000 00000520 00000560
+00000000 00000000`:
+
+offset|content|example
+---|---|---
+00 - 03|content index|00000001
+04|embedded offset within cell|1C
+05 - 0F|unused|
+10 - 13|mid-cell for leading 00|00000520
+14 - 17|mid-cell for leading 01|00000560
+18 - 1B|mid-cell for leading 10|00000000 NONE
+1C - 1F|mid-cell for leading 11|00000000 NONE
+
+Both `0x51C` and `0x51F` are valid pointers in this cell. The former refers to the plain split node, the latter to its
+content-augmented version. The only difference between the two is the result of a call to `content()`.
+
+![graph](MemtableTrie.md.g4.svg)
+
+
+## Reading a trie
+
+`MemtableTrie` is mainly meant to be used as an implementation of `Trie`. As such, the main method of retrieval of
+information is via some selection (i.e. intersection) of a subtrie followed by a walk over the content in this
+subtrie. Straightforward methods for direct retrieval of data by key are also provided, but they are mainly for testing.
+
+The methods for iterating over and transforming tries are provided by the `Trie` interface and are built on the cursor
+interface implemented by `MemtableTrie` (see `Trie.md` for a description of cursors).
+
+![graph](MemtableTrie.md.wc1.svg)
+
+(Edges in black show the trie's structure, and the ones in <span style="color:lightblue">light blue</span> the path the cursor walk takes.)
+
+### Cursors over `MemtableTrie`
+
+`MemtableTrie` implements cursors using arrays of integers to store the backtracking state (as the simplest
+possible structure that can be easily walked and garbage collected). No backtracking state is added for `Chain` or 
+`Leaf` nodes and any prefix. For `Sparse` we store the node address, depth and the remainder of the sparse order word.
+That is, we read the sparse order word on entry, peel off the next index to descend and store the remainder. When we 
+backtrack to the node we peel off another index -- if the remainder becomes 0, there are no further children and the 
+backtracking entry can be removed.
+
+For `Split` nodes we store one entry per split node cell. This means:
+- one entry for the head cell with address, depth and next child bits `0bHH000000` where HH is between 1 and 3
+- one entry for the mid cell with address, depth and next child bits `0bHHMMM000` where MMM is between 1 and 7
+- one entry for the tail cell with address, depth and next child bits `0bHHMMMTTT` where TTT is between 1 and 7
+
+On backtracking we recognize the sublevel by the position of the lowest non-zero bit triple. For example, if the last
+three are not `0b000`, this is a tail cell, we can advance in it and use the HHMMM bits to form the transition byte.
+
+This substructure is a little more efficient than storing only one entry for the split node (the head-to-mid and 
+mid-to-tail links do not need to be followed for every new child) and also allows us to easily get the precise next 
+child and remove the backtracking entry when a cell has no further children.
+
+`MemtableTrie` cursors also implement `advanceMultiple`, which jumps over intermediate nodes in `Chain` blocks:
+
+![graph](MemtableTrie.md.wc2.svg)
+
+## Mutation
+
+Mutation of `MemtableTrie` must be done by one thread only (for performance reasons we don't enforce it, the user must
+make sure that's the case), but writes may be concurrent with multiple reads over the data that is being mutated. The
+trie is built to support this by making sure that any modification of a node is safe for any reader that is operating
+concurrently.
+
+The main method for mutating a `MemtableTrie` is `apply`, which merges the structure of another `Trie` in. 
+`MemtableTrie` also provides simpler recursive method of modification, `putRecursive`, which creates a single 
+`key -> value` mapping in the trie. We will describe the mutation process starting with a `putRecursive` example.
+
+### Adding a new key -> value mapping using `putRecursive`
+
+Suppose we want to insert the value `traverse` into the trie described in the previous paragraph. The recursive
+insertion process walks the trie to find corresponding existing nodes for the ones in the path to be inserted.
+When it has to leave the existing trie, because it has no entries for the path, the process continues using `NONE` as
+the trie node.
+
+![graph](MemtableTrie.md.m1.svg)
+
+When it reaches the end of the path, it needs to attach the value. Unless this is a prefix of an existing entry, the 
+matching trie node will either be `NONE` or a leaf node. Here it's `NONE`, so we create a item in the
+content array, `contentArray[3]`, put the value in it, and thus form the leaf node `~3` (`0xFFFFFFFC`). The recursive
+process returns this to the previous step.
+
+The previous step must attach a child with the transition `e` to the node `NONE`. Since this is a new node, we do this
+by creating a new `Chain` node at address `0x0BB` mapping `e` to `~3` and return that. For the node above, we again
+need to attach a child to `NONE`, but this time the child is a `Chain` node, so we can do this by expanding it, i.e.
+writing the new character at the address just before the child pointer, and returning that address (note that the
+child chain node is newly created, so we can't be overwriting any existing data there). We can do this several more
+times.
+
+![graph](MemtableTrie.md.m2.svg)
+
+(<span style="color:lightblue">Light blue</span> specifies the descent path, <span style="color:pink">pink</span>
+the values returned, <span style="color:blue">blue</span> stands for newly-created nodes and links, and
+<span style="color:lightgray">light gray</span> for obsoleted nodes and links.)
+
+In the next step we must attach the child `0x0B8` with transition `v` to the existing `Chain` node `0x018`. This is a
+different transition from the one that node already has, so the change cannot be accommodated by a node of type `Chain`,
+thus we need to copy this into a new `Sparse` node `0x0DE` with two children, the existing `c -> 0x019` and the new
+`v -> 0x0B8` and return `0x0DE` to the parent step.
+
+The parent step must then change its existing pointer for the character `a` from `0x018` to `0x0DE` which it can do in
+place by writing the new value in its pointer cell for `a`. This is the attachment point for the newly created
+substructure, i.e. before this, the new nodes were not reachable, and now become reachable; before this, the node
+`0x018 ` was reachable, and now becomes unreachable. The attachment is done by a volatile write, to enforce a 
+happens-before relationship that makes sure that all the new substructure (all written by this thread) is fully readable
+by all readers who pass through the new pointer (which is the only way they can reach it). The same happens-before also 
+ensures that any new readers cannot reach the obsoleted nodes (there may be existing reader threads that are already in 
+them).
+
+It can then return its address `0x07E` unchanged up, and no changes need to be done in any of the remaining steps. The
+process finishes in a new value for `root`, which in this case remains unchanged.
+
+![graph](MemtableTrie.md.m3.svg)
+
+The process created a few new nodes (in blue), and made one obsolete (in grey). What concurrent readers can see depends
+on where they are at the time the attachment point write is done. Forward traversals, if they are in the path below
+`0x07E`, will continue working with the obsoleted data and will not see any of the new changes. If they are above
+`0x07E`, they will see the updated content. If they are _at_ the `0x07E` node, they may see either, depending on the
+time they read the pointer for `a`. Reverse traversals that happen to be in the region to the right of the new nodes
+_will_ see the updated content, as they will read the pointer after it has been updated.
+
+In any case, the obsolete paths remain correct and usable for any thread that has already reached them, and the new
+paths are correct and usable from the moment they become reachable.
+
+Note that if we perform multiple mutations in sequence, and a reader happens to be stalled between them (in iteration
+order), such reader may see only the mutation that is ahead of it _in iteration order_, which is not necessarily the
+mutation that happened first. For the example above, if we also inserted `trespass`, a reader thread that was paused
+at `0x018` in a forward traversal and wakes up after both insertions have completed will see `trespass`, but _will not_
+see `traverse` even though it was inserted earlier.
+
+### In-place modifications
+
+When the backtracking process returns with a new mapping, there are several cases when we can apply a change in place
+(creating an attachment point for the new path). We will explain these in detail, as it is important to understand what
+exactly happens from concurrent readers' point of view in all of them.
+
+Note that if a modification cannot be done in place, we copy the content to a new node. The copied node is always
+unreachable and there will always be an attachment point that makes it reachable somewhere in the parent chain.
+
+#### Changing the child pointer of the last `Chain` node in a chain
+
+This happens when the existing transition matches the transition of the new character, but the pointer is different,
+and only applies to `Chain` nodes whose offset is `0x1B`. In this case the child pointer is written at offset `0x1C`,
+and we can put in the new value by performing a volatile write.
+
+For example, updating `N -> 0x39C` is accomplished by making the volatile write:
+
+offset|content|before|after
+---|---|---|---
+00-1A|irrelevant||
+1B|character|N|N
+1C-1F|pointer|0000031E|_**0000039C**_
+
+(Here and below normal writes are in bold and volatile writes in bold italic.)
+
+Readers have to read the pointer to reach the child (old or new), so this achieves the happens-before guarantees we
+seek. Readers either see the old value (where none of the branch's data has been modified in any way), or the new value
+(where the happens-before guarantees all writes creating the attached substructure are fully visible).
+
+Note that if the node is not the last in the chain, the pointer is implicit and we cannot change it. Thus we have
+to copy, i.e. create a new node, which in this case will also be a `Chain` node, because there is nothing else in the
+original node that needs to be preserved (the only existing transition is replaced by the update).
+
+#### Changing the child pointer of a `Sparse` or `Split` node
+
+Similarly to above, in this case the transition matches an existing one, and thus we already have a 4-byte location
+where the pointer to the old child is written, and we can update it by doing a volatile write.
+
+For example, updating `C -> 0x51E` in a sparse node can be:
+
+offset|content|before|after
+---|---|---|---
+00 - 03|child pointer 0| 00000238|00000238
+04 - 07|child pointer 1| 0000013A|_**0000051E**_
+08 - 0B|child pointer 2| 0000033B|0000033B
+0C - 17|unused|
+18     |character 0| 41 A|41 A
+19     |character 1| 43 C|43 C
+1A     |character 2| 35 5|35 5
+1B - 1D|unused|
+1E - 1F|order word| 0026 = 102 (base 6)
+
+
+#### Adding a new child to `Split`
+
+If we already have the substructure that leads to the pointer for the new transition (i.e. a mid- and end-cell for the
+transition's first 2-3 bits already exists), the situation is as above, where the existing pointer is `NONE`, and we can
+simply perform a volatile write.
+
+If an end-cell mapping does not exist, we allocate a new cleared cell (so that all pointers are `NONE`), write the new
+pointer at its position using a non-volatile write, and then create a mapping to this end-cell in the mid cell by
+volatile writing its pointer over the `NONE` in the correct offset. Similarly, if there's no mid-cell either, we create
+empty end-cell and mid-cell, write pointer in end-cell and mapping in mid-cell non-volatile, and write the mapping in
+the leading cell volatile.
+
+In any of these cases, readers have to pass through the volatile update to reach any of the new content.
+
+For example, to add `x -> 0x71A` (`x` is `0x78` or `0b01111000`) to the split node example needs a new end cell for
+`01 111` (for example at `0x720-0x73F`) (these writes can be non-volatile):
+
+offset|content|before|after
+---|---|---|---
+00 - 03|pointer to child for ending 000|n/a|**0000071A**
+04 - 07|pointer to child for ending 001|n/a|**00000000** NONE
+08 - 0B|pointer to child for ending 010|n/a|**00000000** NONE
+0C - 0F|pointer to child for ending 011|n/a|**00000000** NONE
+10 - 13|pointer to child for ending 100|n/a|**00000000** NONE
+14 - 17|pointer to child for ending 101|n/a|**00000000** NONE
+18 - 1B|pointer to child for ending 110|n/a|**00000000** NONE
+1C - 1F|pointer to child for ending 111|n/a|**00000000** NONE
+
+and this volatile write to the mid cell `0x520`:
+
+offset|content|before|after
+---|---|---|---
+00 - 03|end-cell for middle 000|00000000 NONE|00000000 NONE
+04 - 07|end-cell for middle 001|00000000 NONE|00000000 NONE
+08 - 0B|end-cell for middle 010|00000000 NONE|00000000 NONE
+0C - 0F|end-cell for middle 011|00000000 NONE|00000000 NONE
+10 - 13|end-cell for middle 100|00000000 NONE|00000000 NONE
+14 - 17|end-cell for middle 101|00000000 NONE|00000000 NONE
+18 - 1B|end-cell for middle 110|0000055C|0000055C
+1C - 1F|end-cell for middle 111|00000000 NONE|_**0000073C**_
+
+The start cell, and the other mid and end cells remain unchanged.
+
+#### Adding a new child to `Sparse` with 5 or fewer existing children
+
+The need to maintain a correct view for concurrent readers without blocking is the reason why we cannot keep the
+children in a `Sparse` cell ordered (if we insert ordered, we open ourselves to readers possibly seeing the same pointer
+or child twice, or even going back in the iteration order). Instead we always add new characters and pointers at the
+next free position and then update the order word to include it. More precisely:
+- we find the smallest index `i < 6` for which the pointer is `NONE`
+- we write the transition character at position `i`
+- we write the pointer at position `i` over `NONE` volatile
+- we compile a new order word by inserting `i` after all indexes with greater transition and before all indexes with
+  smaller in the base-6 representation (e.g. to insert `j` in sparse node that has `a@4 f@0 g@2 k@1 q@3` we change the
+  order word `31204` to `315204`) and write it volatile
+
+This ensures that any reader that iterates over children (i.e. one that needs the order word) will have to pass through
+the volatile order word update and will see the correct character and pointer values. Readers who have read the order
+word at some earlier time will not include the new pointer or character in the iteration.
+
+Readers that directly select the child for a given transition must read the pointer for each index _before_ reading the
+character to ensure they can see the properly updated value (otherwise they could match e.g. a `00` transition to the
+new branch because the real character was not written when they read the byte, but the pointer was when they got to it)
+and stop searching when they find a `NONE` pointer.
+
+For example, adding `x -> 0x71A` to the sparse example above is done by:
+
+offset|content|before|after
+---|---|---|---
+00 - 03|child pointer 0| 00000238|00000238
+04 - 07|child pointer 1| 0000051E|0000051E
+08 - 0B|child pointer 2| 0000033B|0000033B
+0C - 0F|child pointer 3|any|_**0000071A**_
+10 - 17|unused|NONE|NONE
+18     |character 0| 41 A|41 A
+19     |character 1| 43 C|43 C
+1A     |character 2| 35 5|35 5
+1B     |character 3| any |**78** x
+1C - 1D|unused|00 00|00 00
+1E - 1F|order word|0026 = 102 (base 6)|_**02AE**_ = 3102 (base 6)
+
+where we first write the character, then volatile write the pointer, and finally the order word.
+
+#### Changing the root pointer
+
+If an update propagates with copying modifications all the way to the root, we must update the root pointer. The latter
+is a volatile variable, so this also enforces the happens-before relationship we need.
+
+### Merging a branch using `apply`
+
+This is a generalization of the mutation procedure above, which applies to more complex branches, where each node may
+potentially need multiple updates to attach more than one child. The process proceeds following the nodes that a cursor
+of the mutation trie produces and we maintain full (i.e. for all nodes in the parent chain) backtracking information in
+an array of integers that contains
+- `existingNode`, the corresponding pointer in the memtable trie,
+- `updatedNode`, the current corresponding pointer in the memtable trie, which may be different from the above if the
+  mutation node is branching and one or more of its children have been already added,
+- `transition`, the incoming edge we took to reach the node.
+
+When we descend, we follow the transitions in the memtable trie corresponding to the ones from an iteration over the
+structure of the mutation trie to obtain the `existingNode` pointers, and initialize `updatedNode` to the same. When the
+iteration processes a child, we apply the update to the node, which may happen in place, or may require copying
+&mdash; in the latter case `updatedNode` will change to the new value. Note that if `updatedNode` was different from
+the original `existingNode`, it was pointing to an unreachable copied node which will remain unreachable as we will only
+attach the newer version.
+
+After all modifications coming as the result of application of child branches have been applied, we have an
+`updatedNode` that reflects all. As we ascend we apply that new value to the parent's `updatedNode`.
+
+For example (adding a trie containing "traverse, truck" to the "tractor, tree, trie" one):
+
+![graph](MemtableTrie.md.a1.svg)
+
+In this diagram `existingNode`s are the ones reached through the <span style="color:lightblue">light blue</span> arrows during the descent phase (e.g.
+`0x018` for the `ApplyState` at `tra`, or `NONE` for `tru`), and `updatedNode`s are the ones ascent (<span style="color:pink">pink</span> arrows)
+returns with (e.g . `0x0DE` and `0x0FA` for the respective states).
+
+During this process, readers can see any modifications made in place (each in-place modification is an attachment point
+which makes part of the new nodes reachable). The update mechanism above makes sure both that the state before the
+update is preserved, and that the state after the update is fully visible for readers that can reach it, but it does not
+guarantee that the mutation is seen atomically by the readers if it contains multiple separate branches.
+It is possible for a reader to see only a part of the update, for example:
+- a reading thread racing with the mutator can iterate over `traverse` but finish iterating before the mutator
+manages to attach `truck`;
+- a reading thread that iterated to `tree` (while `traverse` was not yet attached) and paused, will see `truck` if the
+mutating thread applies the update during the pause.
+
+### Handling prefix nodes
+
+The descriptions above were given without prefix nodes. Handling prefixes is just a little complication over the update
+process where we must augment `updatedNode` with any applicable content before applying the change to the parent.
+To do this we expand the state tracked to:
+- `existingPreContentNode` which points to the existing node including any prefix,
+- `existingPostContentNode` which is obtained by skipping over the prefix (for simplicity we also treat leaf
+  nodes like a prefix with no child) and is the base for all child updates (i.e. it takes the role of
+  `existingNode` in the descriptions above),
+- `updatedPostContentNode` which is the node as changed/copied after children modifications are applied,
+- `contentIndex` which is the index in the content array for the result of merging existing and newly introduced 
+  content, (Note: The mutation content is only readable when the cursor enters the node, and we can only attach it when
+  we ascend from it.)
+- `transition` remains as before.
+
+and we then attach `contentIndex` to compile an `updatedPreContentNode` which the parent is made to link to. This will
+be equal to `updatedPostContentNode` if no content applies, i.e. `contentIndex == -1`.
+
+("Pre-" and "post-" refer to descent/iteration order, not to construction order; e.g. `updatedPreContentNode` is
+constructed after `updatedPostContentNode` but links above it in the trie.)
+
+As an example, consider the process of adding `trees` to our sample trie:
+
+![graph](MemtableTrie.md.p1.svg)
+
+When descending at `tree` we set `existingPreContentNode = ~1`, `existingPostContentNode = NONE` and `contentIndex = 1`.
+Ascending back to add the child `~3`, we add a child to `NONE` and get `updatedPostContentNode = 0x0BB`. To then apply
+the existing content, we create the embedded prefix node `updatedPreContentNode = 0x0BF` with `contentIndex = 1` and
+pass that on to the recursion.
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.a1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.a1.svg
new file mode 100644
index 0000000000..4237fb599a
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.a1.svg
@@ -0,0 +1,599 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+# embedded DOT (plantuml) works in IDEA preview
+# but not on GitHub
+```plantuml
+digraph G {
+    { rank=same root -> start [style=invis] }
+    start [label="start/end"]
+
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"; color = "lightgrey"; fontcolor = lightgray]
+
+    tree [label = "contentArray[1]"]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    root -> t [label = " t"]
+    t -> tr [label = " r"]
+    trac -> tract [label = " t"]
+    tract -> tracto [label = " o"]
+    tracto -> tractor [label = " r"]
+
+    tr -> tra [label = " a"; color = "lightgrey"; fontcolor = lightgray]
+    tra -> trac [label = " c"; color = "lightgrey"; fontcolor = lightgray]
+
+    tr -> tre [label = " e"]
+    tre -> tree [label = " e"]
+
+    tr -> tri [label = " i"]
+    tri -> trie [label = " e"]
+
+    subgraph path {
+        edge [color = "lightblue"; fontcolor="blue"; arrowhead="vee"]
+        node [color = "blue"; fontcolor="blue"]
+
+        start -> root
+
+        root -> t [label = " t"]
+        t -> tr [label = " r"]
+        tr -> tra [label = " a"]
+        tra -> trav [label = " v"]
+        trav -> trave [label = " e"]
+        trave -> traver [label = " r"]
+        traver -> travers [label = " s"]
+        travers -> traverse [label = " e"]
+
+        tra2 [label = "Sparse\n0x0DE"]
+        trav [label = "Chain\n0x0B8"]
+        trave [label = "0x0B9"]
+        traver [label = "0x0BA"]
+        travers [label = "0x0BB"]
+        traverse [label = "contentArray[3]"]
+
+        tr -> tru [label = " u"]
+        tru -> truc [label = " c"]
+        truc -> truck [label = " k"]
+
+        tru [label = "Chain\n0x0FA"]
+        truc [label = "0x0FB"]
+        truck [label = "contentArray[4]"]
+    }
+
+    {rank=same tra -> tra2 -> tre -> tri -> tru [style=invis]}
+    {rank=same trac -> trav -> tree -> trie -> truc [style=invis]}
+
+    {
+        edge [color = "blue"]
+        tr -> tra2 [label = " a"]
+        tra2 -> trac [label = " c"]
+        tra2 -> trav [label = " v"]
+        trav -> trave [label = " e"]
+        trave -> traver [label = " r"]
+        traver -> travers [label = " s"]
+        travers -> traverse [label = " e"]
+        tr -> tru [label = " u"]
+        tru -> truc [label = " c"]
+        truc -> truck [label = " k"]
+    }
+
+    subgraph back {
+        edge [color = "pink"; fontcolor="red"; arrowhead="vee"; constrain="false"]
+
+        traverse -> travers [label = " ~3"]
+        travers -> traver [label = "0x0BB"]
+        traver -> trave [label = "0x0BA"]
+        trave -> trav [label = "0x0B9"]
+        trav -> tra2 [label = "0x0B8"]
+        tra2 -> tr [label = "0x0DE"]
+        tr -> t [label = "0x07E"]
+        t -> root [label = "0x09B"]
+        root -> start [label = "0x09A"]
+
+        truck -> truc [label = "~4"]
+        truc -> tru [label = "0x0FB"]
+        tru -> tr [label = "0x0FA"]
+    }
+}
+```
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="757pt" height="846pt"
+     viewBox="0.00 0.00 757.47 845.73" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 841.7251)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-841.7251 753.4738,-841.7251 753.4738,4 -4,4"/>
+        <!-- root -->
+        <g id="node1" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="357.2369" cy="-808.3095" rx="33.1337" ry="29.3315"/>
+            <text text-anchor="middle" x="357.2369" y="-812.5095" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="357.2369" y="-795.7095" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- start -->
+        <g id="node2" class="node">
+            <title>start</title>
+            <ellipse fill="none" stroke="#000000" cx="490.2369" cy="-808.3095" rx="44.0775" ry="18"/>
+            <text text-anchor="middle" x="490.2369" y="-804.1095" font-family="Times,serif" font-size="14.00" fill="#000000">start/end</text>
+        </g>
+        <!-- root&#45;&gt;start -->
+        <!-- root&#45;&gt;start -->
+        <g id="edge51" class="edge">
+            <title>root&#45;&gt;start</title>
+            <path fill="none" stroke="#ffc0cb" d="M390.7984,-808.3095C404.3037,-808.3095 420.2339,-808.3095 435.3398,-808.3095"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="445.6261,-808.3095 435.6262,-812.8096 440.6261,-808.3095 435.6261,-808.3096 435.6261,-808.3096 435.6261,-808.3096 440.6261,-808.3095 435.6261,-803.8096 445.6261,-808.3095 445.6261,-808.3095"/>
+            <text text-anchor="middle" x="418.2509" y="-815.5095" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x09A</text>
+        </g>
+        <!-- t -->
+        <g id="node13" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="357.2369" cy="-708.0939" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="357.2369" y="-703.8939" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge2" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#000000" d="M337.9564,-784.2612C330.7393,-772.3485 325.2768,-757.6462 329.8463,-744.0939 331.1636,-740.1869 333.0926,-736.3684 335.3269,-732.7663"/>
+            <polygon fill="#000000" stroke="#000000" points="338.3844,-734.5068 341.3357,-724.3313 332.683,-730.4454 338.3844,-734.5068"/>
+            <text text-anchor="middle" x="333.9322" y="-748.2939" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge14" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#add8e6" d="M357.2369,-778.7835C357.2369,-765.2984 357.2369,-749.4062 357.2369,-736.1092"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="357.2369,-726.0972 361.737,-736.0971 357.2369,-731.0972 357.237,-736.0972 357.237,-736.0972 357.237,-736.0972 357.2369,-731.0972 352.737,-736.0972 357.2369,-726.0972 357.2369,-726.0972"/>
+            <text text-anchor="middle" x="360.9322" y="-748.2939" font-family="Times,serif" font-size="14.00" fill="#0000ff"> t</text>
+        </g>
+        <!-- start&#45;&gt;root -->
+        <g id="edge13" class="edge">
+            <title>start&#45;&gt;root</title>
+            <path fill="none" stroke="#add8e6" d="M446.4042,-805.6333C434.2938,-805.1818 421.1479,-804.9584 409.0184,-805.3095 406.3264,-805.3874 403.5568,-805.4881 400.7643,-805.6051"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="390.6504,-806.089 400.4239,-801.1161 395.6447,-805.85 400.639,-805.611 400.639,-805.611 400.639,-805.611 395.6447,-805.85 400.8541,-810.1058 390.6504,-806.089 390.6504,-806.089"/>
+        </g>
+        <!-- tractor -->
+        <g id="node3" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tracto -->
+        <g id="node4" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="74.2369" cy="-195.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="74.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge6" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#000000" d="M74.0297,-177.2006C73.8934,-165.0949 73.7122,-149.0076 73.5575,-135.2674"/>
+            <polygon fill="#000000" stroke="#000000" points="77.0529,-134.8319 73.4404,-124.872 70.0533,-134.9108 77.0529,-134.8319"/>
+            <text text-anchor="middle" x="78.3172" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- tract -->
+        <g id="node5" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="78.2369" cy="-284.4" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="78.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge5" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#000000" d="M77.4081,-266.0006C76.8628,-253.8949 76.1381,-237.8076 75.5192,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="78.9975,-223.5044 75.0509,-213.672 72.0046,-223.8194 78.9975,-223.5044"/>
+            <text text-anchor="middle" x="82.4869" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> o</text>
+        </g>
+        <!-- trac -->
+        <g id="node6" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="94.2369" cy="-384.6156" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="94.2369" y="-380.4156" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge4" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#000000" d="M91.3022,-366.2345C88.9115,-351.2603 85.4955,-329.8643 82.756,-312.7055"/>
+            <polygon fill="#000000" stroke="#000000" points="86.1858,-311.9877 81.1529,-302.6646 79.2734,-313.0913 86.1858,-311.9877"/>
+            <text text-anchor="middle" x="89.9322" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- trav -->
+        <g id="node14" class="node">
+            <title>trav</title>
+            <ellipse fill="none" stroke="#0000ff" cx="202.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="202.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#0000ff">Chain</text>
+            <text text-anchor="middle" x="202.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0B8</text>
+        </g>
+        <!-- trac&#45;&gt;trav -->
+        <!-- tra -->
+        <g id="node7" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#d3d3d3" cx="133.2369" cy="-496.2469" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="133.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#d3d3d3">Chain</text>
+            <text text-anchor="middle" x="133.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#d3d3d3">0x018</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge8" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#d3d3d3" d="M114.5188,-470.6739C110.3016,-463.8622 106.2666,-456.3098 103.5223,-448.8313 99.3758,-437.5318 97.0731,-424.4776 95.7973,-413.1021"/>
+            <polygon fill="#d3d3d3" stroke="#d3d3d3" points="99.2538,-412.4638 94.8622,-402.822 92.2826,-413.098 99.2538,-412.4638"/>
+            <text text-anchor="middle" x="109.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#d3d3d3"> c</text>
+        </g>
+        <!-- tra&#45;&gt;trav -->
+        <g id="edge17" class="edge">
+            <title>tra&#45;&gt;trav</title>
+            <path fill="none" stroke="#add8e6" d="M130.7527,-466.6197C131.0399,-455.1889 132.9734,-442.3975 138.7369,-432.0313 144.2651,-422.0881 152.8469,-413.6214 161.933,-406.7148"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="170.346,-400.8129 164.7439,-410.2398 166.2527,-403.6844 162.1595,-406.5559 162.1595,-406.5559 162.1595,-406.5559 166.2527,-403.6844 159.5751,-402.872 170.346,-400.8129 170.346,-400.8129"/>
+            <text text-anchor="middle" x="144.4869" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> v</text>
+        </g>
+        <!-- tra2 -->
+        <g id="node19" class="node">
+            <title>tra2</title>
+            <ellipse fill="none" stroke="#0000ff" cx="244.2369" cy="-496.2469" rx="39.2145" ry="29.3315"/>
+            <text text-anchor="middle" x="244.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#0000ff">Sparse</text>
+            <text text-anchor="middle" x="244.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0DE</text>
+        </g>
+        <!-- tra&#45;&gt;tra2 -->
+        <!-- tree -->
+        <g id="node8" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#000000" cx="348.2369" cy="-384.6156" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="348.2369" y="-380.4156" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- trie -->
+        <g id="node10" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="530.2369" cy="-384.6156" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="530.2369" y="-380.4156" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tree&#45;&gt;trie -->
+        <!-- tre -->
+        <g id="node9" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="357.2369" cy="-496.2469" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="357.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="357.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge10" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#000000" d="M354.8487,-466.6249C353.5073,-449.9873 351.8422,-429.334 350.5186,-412.9163"/>
+            <polygon fill="#000000" stroke="#000000" points="353.9973,-412.5103 349.7049,-402.8239 347.0199,-413.0729 353.9973,-412.5103"/>
+            <text text-anchor="middle" x="359.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tri -->
+        <g id="node11" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="468.2369" cy="-496.2469" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="468.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="468.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- tre&#45;&gt;tri -->
+        <!-- truc -->
+        <g id="node21" class="node">
+            <title>truc</title>
+            <ellipse fill="none" stroke="#0000ff" cx="676.2369" cy="-384.6156" rx="37.1616" ry="18"/>
+            <text text-anchor="middle" x="676.2369" y="-380.4156" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0FB</text>
+        </g>
+        <!-- trie&#45;&gt;truc -->
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge12" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#000000" d="M483.2454,-469.2241C492.9216,-451.8019 505.4604,-429.2258 515.1517,-411.7765"/>
+            <polygon fill="#000000" stroke="#000000" points="518.4109,-413.1168 520.2066,-402.6752 512.2914,-409.718 518.4109,-413.1168"/>
+            <text text-anchor="middle" x="507.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tru -->
+        <g id="node20" class="node">
+            <title>tru</title>
+            <ellipse fill="none" stroke="#0000ff" cx="580.2369" cy="-496.2469" rx="38.626" ry="29.3315"/>
+            <text text-anchor="middle" x="580.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#0000ff">Chain</text>
+            <text text-anchor="middle" x="580.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0FA</text>
+        </g>
+        <!-- tri&#45;&gt;tru -->
+        <!-- tr -->
+        <g id="node12" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="357.2369" cy="-607.8782" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="357.2369" y="-612.0782" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="357.2369" y="-595.2782" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge7" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#d3d3d3" d="M321.8872,-597.2153C295.7144,-588.7379 259.543,-575.7815 229.5223,-560.4626 208.2971,-549.6318 186.0738,-535.0548 168.3323,-522.5268"/>
+            <polygon fill="#d3d3d3" stroke="#d3d3d3" points="170.1754,-519.5418 160.0068,-516.5667 166.1007,-525.2336 170.1754,-519.5418"/>
+            <text text-anchor="middle" x="235.0942" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#d3d3d3"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge16" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#add8e6" d="M319.8208,-603.7035C283.8354,-598.2919 229.0725,-586.2816 188.5223,-560.4626 176.3594,-552.7182 165.4058,-541.4906 156.5184,-530.5822"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="150.1392,-522.3658 159.8263,-527.5049 153.2055,-526.3152 156.2719,-530.2646 156.2719,-530.2646 156.2719,-530.2646 153.2055,-526.3152 152.7174,-533.0243 150.1392,-522.3658 150.1392,-522.3658"/>
+            <text text-anchor="middle" x="194.0942" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge9" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#000000" d="M357.2369,-578.2562C357.2369,-565.3881 357.2369,-550.1179 357.2369,-536.2631"/>
+            <polygon fill="#000000" stroke="#000000" points="360.737,-536.0074 357.2369,-526.0074 353.737,-536.0074 360.737,-536.0074"/>
+            <text text-anchor="middle" x="362.0942" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge11" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#000000" d="M380.5061,-584.4766C397.0825,-567.806 419.5291,-545.2318 437.5674,-527.0908"/>
+            <polygon fill="#000000" stroke="#000000" points="440.4292,-529.1767 444.9983,-519.6177 435.4654,-524.241 440.4292,-529.1767"/>
+            <text text-anchor="middle" x="423.9322" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> i</text>
+        </g>
+        <!-- tr&#45;&gt;t -->
+        <g id="edge49" class="edge">
+            <title>tr&#45;&gt;t</title>
+            <path fill="none" stroke="#ffc0cb" d="M355.1399,-637.287C354.605,-648.2159 354.2816,-660.7205 354.6843,-672.0939 354.7703,-674.5242 354.889,-677.0451 355.0283,-679.5733"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="355.6875,-689.7931 350.5531,-680.1035 355.3656,-684.8034 355.0438,-679.8138 355.0438,-679.8138 355.0438,-679.8138 355.3656,-684.8034 359.5344,-679.5241 355.6875,-689.7931 355.6875,-689.7931"/>
+            <text text-anchor="middle" x="373.5132" y="-659.4939" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x07E</text>
+        </g>
+        <!-- tr&#45;&gt;tra2 -->
+        <g id="edge33" class="edge">
+            <title>tr&#45;&gt;tra2</title>
+            <path fill="none" stroke="#0000ff" d="M333.9323,-584.2913C321.8427,-572.1071 306.8029,-557.03 293.2369,-543.6626 287.6045,-538.1127 281.5754,-532.2309 275.74,-526.5682"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="278.1153,-523.9962 268.4966,-519.5541 273.2458,-529.0249 278.1153,-523.9962"/>
+            <text text-anchor="middle" x="314.0942" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tru -->
+        <g id="edge22" class="edge">
+            <title>tr&#45;&gt;tru</title>
+            <path fill="none" stroke="#add8e6" d="M389.1874,-591.8842C428.4573,-572.2261 495.1969,-538.817 538.4895,-517.1452"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="547.6205,-512.5743 540.6927,-521.0747 543.1494,-514.8125 538.6783,-517.0507 538.6783,-517.0507 538.6783,-517.0507 543.1494,-514.8125 536.664,-513.0267 547.6205,-512.5743 547.6205,-512.5743"/>
+            <text text-anchor="middle" x="486.4869" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> u</text>
+        </g>
+        <!-- tr&#45;&gt;tru -->
+        <g id="edge40" class="edge">
+            <title>tr&#45;&gt;tru</title>
+            <path fill="none" stroke="#0000ff" d="M393.3967,-599.0169C422.2538,-591.1383 463.1788,-578.126 496.2369,-560.4626 514.2689,-550.8278 532.5102,-537.4219 547.3655,-525.3499"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="549.7845,-527.8908 555.2473,-518.8128 545.3158,-522.5028 549.7845,-527.8908"/>
+            <text text-anchor="middle" x="525.4869" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> u</text>
+        </g>
+        <!-- t&#45;&gt;root -->
+        <g id="edge50" class="edge">
+            <title>t&#45;&gt;root</title>
+            <path fill="none" stroke="#ffc0cb" d="M362.0886,-726.2786C363.3802,-731.9426 364.5942,-738.2388 365.2369,-744.0939 366.1247,-752.1817 365.87,-760.8327 365.048,-769.081"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="363.7469,-779.2407 360.5537,-768.7501 364.3821,-774.2812 365.0173,-769.3217 365.0173,-769.3217 365.0173,-769.3217 364.3821,-774.2812 369.4808,-769.8934 363.7469,-779.2407 363.7469,-779.2407"/>
+            <text text-anchor="middle" x="384.9052" y="-748.2939" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x09B</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge3" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#000000" d="M378.7729,-693.2308C385.3244,-687.4682 391.7179,-680.2751 395.2369,-672.0939 398.1872,-665.2348 397.7115,-662.3385 395.2369,-655.2939 393.2659,-649.6828 390.3202,-644.2436 386.9247,-639.1727"/>
+            <polygon fill="#000000" stroke="#000000" points="389.6469,-636.9677 380.8782,-631.0211 384.0247,-641.138 389.6469,-636.9677"/>
+            <text text-anchor="middle" x="402.3172" y="-659.4939" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge15" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#add8e6" d="M340.324,-692.0138C335.3995,-686.2543 330.6543,-679.3722 328.0763,-672.0939 324.4827,-661.9481 326.7954,-651.1983 331.382,-641.4556"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="336.3576,-632.4758 335.4471,-643.4038 333.9342,-636.8493 331.5109,-641.2228 331.5109,-641.2228 331.5109,-641.2228 333.9342,-636.8493 327.5747,-639.0418 336.3576,-632.4758 336.3576,-632.4758"/>
+            <text text-anchor="middle" x="332.3172" y="-659.4939" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- trav&#45;&gt;tree -->
+        <!-- trave -->
+        <g id="node15" class="node">
+            <title>trave</title>
+            <ellipse fill="none" stroke="#0000ff" cx="202.2369" cy="-284.4" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="202.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0B9</text>
+        </g>
+        <!-- trav&#45;&gt;trave -->
+        <g id="edge18" class="edge">
+            <title>trav&#45;&gt;trave</title>
+            <path fill="none" stroke="#add8e6" d="M181.4692,-359.4476C176.9833,-352.6281 172.8815,-344.9707 170.5223,-337.2 168.3531,-330.0554 167.8843,-327.3851 170.5223,-320.4 172.1555,-316.0753 174.5712,-311.9338 177.3533,-308.0992"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="183.8676,-300.2431 180.9485,-310.8134 180.6761,-304.0921 177.4845,-307.941 177.4845,-307.941 177.4845,-307.941 180.6761,-304.0921 174.0205,-305.0686 183.8676,-300.2431 183.8676,-300.2431"/>
+            <text text-anchor="middle" x="176.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- trav&#45;&gt;trave -->
+        <g id="edge36" class="edge">
+            <title>trav&#45;&gt;trave</title>
+            <path fill="none" stroke="#0000ff" d="M202.2369,-355.0897C202.2369,-341.6046 202.2369,-325.7123 202.2369,-312.4153"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="205.737,-312.4033 202.2369,-302.4033 198.737,-312.4034 205.737,-312.4033"/>
+            <text text-anchor="middle" x="207.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- trav&#45;&gt;tra2 -->
+        <g id="edge47" class="edge">
+            <title>trav&#45;&gt;tra2</title>
+            <path fill="none" stroke="#ffc0cb" d="M213.0163,-412.9388C215.4002,-419.2228 217.909,-425.8514 220.2369,-432.0313 223.4375,-440.5282 226.8619,-449.6636 230.0924,-458.3025"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="233.6352,-467.7845 225.9197,-459.992 231.8851,-463.1007 230.1351,-458.417 230.1351,-458.417 230.1351,-458.417 231.8851,-463.1007 234.3505,-456.8419 233.6352,-467.7845 233.6352,-467.7845"/>
+            <text text-anchor="middle" x="244.9052" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0B8</text>
+        </g>
+        <!-- trave&#45;&gt;trav -->
+        <g id="edge46" class="edge">
+            <title>trave&#45;&gt;trav</title>
+            <path fill="none" stroke="#ffc0cb" d="M208.9024,-302.4722C210.6782,-308.1317 212.3487,-314.4535 213.2369,-320.4 214.4724,-328.6713 214.0766,-337.5213 212.8888,-345.926"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="211.1036,-355.888 208.4382,-345.2509 211.9856,-350.9664 212.8676,-346.0448 212.8676,-346.0448 212.8676,-346.0448 211.9856,-350.9664 217.297,-346.8386 211.1036,-355.888 211.1036,-355.888"/>
+            <text text-anchor="middle" x="232.9052" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0B9</text>
+        </g>
+        <!-- traver -->
+        <g id="node16" class="node">
+            <title>traver</title>
+            <ellipse fill="none" stroke="#0000ff" cx="202.2369" cy="-195.6" rx="38.8671" ry="18"/>
+            <text text-anchor="middle" x="202.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0BA</text>
+        </g>
+        <!-- trave&#45;&gt;traver -->
+        <g id="edge19" class="edge">
+            <title>trave&#45;&gt;traver</title>
+            <path fill="none" stroke="#add8e6" d="M185.324,-268.32C180.3995,-262.5605 175.6543,-255.6784 173.0763,-248.4 170.5834,-241.3618 170.5834,-238.6382 173.0763,-231.6 174.4962,-227.5912 176.5735,-223.7026 178.9762,-220.0556"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="185.0164,-212.0415 182.5911,-222.7358 182.007,-216.0344 178.9975,-220.0273 178.9975,-220.0273 178.9975,-220.0273 182.007,-216.0344 175.4039,-217.3188 185.0164,-212.0415 185.0164,-212.0415"/>
+            <text text-anchor="middle" x="177.3172" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- trave&#45;&gt;traver -->
+        <g id="edge37" class="edge">
+            <title>trave&#45;&gt;traver</title>
+            <path fill="none" stroke="#0000ff" d="M202.2369,-266.0006C202.2369,-253.8949 202.2369,-237.8076 202.2369,-224.0674"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="205.737,-223.672 202.2369,-213.672 198.737,-223.6721 205.737,-223.672"/>
+            <text text-anchor="middle" x="206.3172" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- traver&#45;&gt;trave -->
+        <g id="edge45" class="edge">
+            <title>traver&#45;&gt;trave</title>
+            <path fill="none" stroke="#ffc0cb" d="M207.6937,-213.7509C209.1467,-219.4136 210.5127,-225.7174 211.2369,-231.6 212.1492,-239.0107 212.1492,-240.9893 211.2369,-248.4 210.9201,-250.9736 210.4804,-253.6279 209.9643,-256.2738"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="207.6937,-266.2491 205.5255,-255.4997 208.8035,-261.3738 209.9132,-256.4985 209.9132,-256.4985 209.9132,-256.4985 208.8035,-261.3738 214.301,-257.4973 207.6937,-266.2491 207.6937,-266.2491"/>
+            <text text-anchor="middle" x="232.4585" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0BA</text>
+        </g>
+        <!-- travers -->
+        <g id="node17" class="node">
+            <title>travers</title>
+            <ellipse fill="none" stroke="#0000ff" cx="203.2369" cy="-106.8" rx="38.305" ry="18"/>
+            <text text-anchor="middle" x="203.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0BB</text>
+        </g>
+        <!-- traver&#45;&gt;travers -->
+        <g id="edge20" class="edge">
+            <title>traver&#45;&gt;travers</title>
+            <path fill="none" stroke="#add8e6" d="M183.9854,-179.6658C174.6965,-169.5397 166.3137,-155.9116 171.2923,-142.8 172.9385,-138.4645 175.3723,-134.3169 178.1749,-130.4793"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="184.7367,-122.6213 181.7812,-133.1814 181.5319,-126.4592 178.3271,-130.2971 178.3271,-130.2971 178.3271,-130.2971 181.5319,-126.4592 174.873,-127.4128 184.7367,-122.6213 184.7367,-122.6213"/>
+            <text text-anchor="middle" x="176.7092" y="-147" font-family="Times,serif" font-size="14.00" fill="#0000ff"> s</text>
+        </g>
+        <!-- traver&#45;&gt;travers -->
+        <g id="edge38" class="edge">
+            <title>traver&#45;&gt;travers</title>
+            <path fill="none" stroke="#0000ff" d="M202.4441,-177.2006C202.5804,-165.0949 202.7616,-149.0076 202.9163,-135.2674"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="206.4205,-134.9108 203.0334,-124.872 199.4209,-134.8319 206.4205,-134.9108"/>
+            <text text-anchor="middle" x="207.7092" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> s</text>
+        </g>
+        <!-- travers&#45;&gt;traver -->
+        <g id="edge44" class="edge">
+            <title>travers&#45;&gt;traver</title>
+            <path fill="none" stroke="#ffc0cb" d="M209.0666,-124.8376C211.6638,-135.0186 213.8193,-147.9908 212.2369,-159.6 211.8717,-162.2793 211.3602,-165.0407 210.7599,-167.7863"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="208.2983,-177.4866 206.3963,-166.6869 209.5282,-172.6402 210.758,-167.7938 210.758,-167.7938 210.758,-167.7938 209.5282,-172.6402 215.1198,-168.9007 208.2983,-177.4866 208.2983,-177.4866"/>
+            <text text-anchor="middle" x="233.0735" y="-147" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0BB</text>
+        </g>
+        <!-- traverse -->
+        <g id="node18" class="node">
+            <title>traverse</title>
+            <ellipse fill="none" stroke="#0000ff" cx="203.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="203.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#0000ff">contentArray[3]</text>
+        </g>
+        <!-- travers&#45;&gt;traverse -->
+        <g id="edge21" class="edge">
+            <title>travers&#45;&gt;traverse</title>
+            <path fill="none" stroke="#add8e6" d="M184.8676,-90.9569C179.5135,-85.2132 174.3474,-78.2805 171.5223,-70.8 168.8843,-63.8149 168.8843,-60.9851 171.5223,-54 172.8668,-50.4399 174.7415,-47.004 176.9144,-43.7612"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="183.203,-35.676 180.6155,-46.3323 180.1332,-39.6227 177.0634,-43.5695 177.0634,-43.5695 177.0634,-43.5695 180.1332,-39.6227 173.5114,-40.8067 183.203,-35.676 183.203,-35.676"/>
+            <text text-anchor="middle" x="177.0942" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- travers&#45;&gt;traverse -->
+        <g id="edge39" class="edge">
+            <title>travers&#45;&gt;traverse</title>
+            <path fill="none" stroke="#0000ff" d="M203.2369,-88.4006C203.2369,-76.2949 203.2369,-60.2076 203.2369,-46.4674"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="206.737,-46.072 203.2369,-36.072 199.737,-46.0721 206.737,-46.072"/>
+            <text text-anchor="middle" x="208.0942" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- traverse&#45;&gt;travers -->
+        <g id="edge43" class="edge">
+            <title>traverse&#45;&gt;travers</title>
+            <path fill="none" stroke="#ffc0cb" d="M209.9024,-36.0722C211.6782,-41.7317 213.3487,-48.0535 214.2369,-54 215.3399,-61.3847 215.3399,-63.4153 214.2369,-70.8 213.8344,-73.4945 213.2713,-76.2661 212.6106,-79.0181"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="209.9024,-88.7278 208.2546,-77.8864 211.2458,-83.9116 212.5891,-79.0954 212.5891,-79.0954 212.5891,-79.0954 211.2458,-83.9116 216.9237,-80.3044 209.9024,-88.7278 209.9024,-88.7278"/>
+            <text text-anchor="middle" x="224.2732" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#ff0000"> ~3</text>
+        </g>
+        <!-- tra2&#45;&gt;trac -->
+        <g id="edge34" class="edge">
+            <title>tra2&#45;&gt;trac</title>
+            <path fill="none" stroke="#0000ff" d="M216.023,-475.2499C189.2207,-455.3034 149.1456,-425.4792 122.3699,-405.5525"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="124.2894,-402.6181 114.1775,-399.4556 120.1102,-408.2337 124.2894,-402.6181"/>
+            <text text-anchor="middle" x="183.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> c</text>
+        </g>
+        <!-- tra2&#45;&gt;tre -->
+        <!-- tra2&#45;&gt;tr -->
+        <g id="edge48" class="edge">
+            <title>tra2&#45;&gt;tr</title>
+            <path fill="none" stroke="#ffc0cb" d="M240.4944,-525.6305C240.4789,-537.419 242.498,-550.5344 249.5777,-560.4626 263.7679,-580.3622 288.2176,-592.0186 310.2338,-598.7899"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="320.1326,-601.5694 309.2883,-603.1984 315.3187,-600.2177 310.5049,-598.866 310.5049,-598.866 310.5049,-598.866 315.3187,-600.2177 311.7214,-594.5335 320.1326,-601.5694 320.1326,-601.5694"/>
+            <text text-anchor="middle" x="270.0665" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0DE</text>
+        </g>
+        <!-- tra2&#45;&gt;trav -->
+        <g id="edge35" class="edge">
+            <title>tra2&#45;&gt;trav</title>
+            <path fill="none" stroke="#0000ff" d="M222.659,-471.5823C217.635,-464.6387 212.8503,-456.7959 209.7369,-448.8313 206.7154,-441.1019 204.8136,-432.4806 203.6368,-424.1275"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="207.1115,-423.7045 202.5344,-414.1494 200.1538,-424.4733 207.1115,-423.7045"/>
+            <text text-anchor="middle" x="215.4869" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> v</text>
+        </g>
+        <!-- tru&#45;&gt;tr -->
+        <g id="edge54" class="edge">
+            <title>tru&#45;&gt;tr</title>
+            <path fill="none" stroke="#ffc0cb" d="M566.5842,-524.088C558.9936,-536.8985 548.4041,-551.2608 535.2369,-560.4626 496.6598,-587.4217 443.613,-599.0546 405.481,-604.073"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="395.1457,-605.3203 404.5344,-599.6545 400.1097,-604.7212 405.0736,-604.1221 405.0736,-604.1221 405.0736,-604.1221 400.1097,-604.7212 405.6128,-608.5897 395.1457,-605.3203 395.1457,-605.3203"/>
+            <text text-anchor="middle" x="571.6815" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0FA</text>
+        </g>
+        <!-- tru&#45;&gt;truc -->
+        <g id="edge23" class="edge">
+            <title>tru&#45;&gt;truc</title>
+            <path fill="none" stroke="#add8e6" d="M584.906,-467.0117C587.8872,-455.247 592.7225,-442.1094 600.5223,-432.0313 610.5674,-419.0519 625.2451,-408.7011 639.0092,-401.0235"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="647.8644,-396.3635 641.1106,-405.0028 643.4397,-398.692 639.0149,-401.0205 639.0149,-401.0205 639.0149,-401.0205 643.4397,-398.692 636.9193,-397.0383 647.8644,-396.3635 647.8644,-396.3635"/>
+            <text text-anchor="middle" x="605.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> c</text>
+        </g>
+        <!-- tru&#45;&gt;truc -->
+        <g id="edge41" class="edge">
+            <title>tru&#45;&gt;truc</title>
+            <path fill="none" stroke="#0000ff" d="M601.5424,-471.4723C617.5177,-452.8958 639.2774,-427.5931 655.1898,-409.0897"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="657.9656,-411.2298 661.8323,-401.3657 652.6583,-406.6656 657.9656,-411.2298"/>
+            <text text-anchor="middle" x="638.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> c</text>
+        </g>
+        <!-- truc&#45;&gt;tru -->
+        <g id="edge53" class="edge">
+            <title>truc&#45;&gt;tru</title>
+            <path fill="none" stroke="#ffc0cb" d="M670.9731,-402.6462C666.3263,-416.3625 658.5213,-435.1464 647.2369,-448.8313 639.6641,-458.015 629.8999,-466.2538 620.1806,-473.1865"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="611.7071,-478.9336 617.4572,-469.5962 615.8452,-476.127 619.9832,-473.3204 619.9832,-473.3204 619.9832,-473.3204 615.8452,-476.127 622.5091,-477.0446 611.7071,-478.9336 611.7071,-478.9336"/>
+            <text text-anchor="middle" x="676.2965" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0FB</text>
+        </g>
+        <!-- truck -->
+        <g id="node22" class="node">
+            <title>truck</title>
+            <ellipse fill="none" stroke="#0000ff" cx="676.2369" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="676.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#0000ff">contentArray[4]</text>
+        </g>
+        <!-- truc&#45;&gt;truck -->
+        <g id="edge24" class="edge">
+            <title>truc&#45;&gt;truck</title>
+            <path fill="none" stroke="#add8e6" d="M661.1247,-367.9416C654.3993,-359.4018 647.222,-348.4489 643.7369,-337.2 641.5272,-330.0678 641.0565,-327.369 643.7369,-320.4 645.1585,-316.7039 647.1547,-313.1507 649.4663,-309.8129"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="655.7148,-302.0002 652.9831,-312.6204 652.5918,-305.9049 649.4689,-309.8097 649.4689,-309.8097 649.4689,-309.8097 652.5918,-305.9049 645.9546,-306.999 655.7148,-302.0002 655.7148,-302.0002"/>
+            <text text-anchor="middle" x="648.4869" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> k</text>
+        </g>
+        <!-- truc&#45;&gt;truck -->
+        <g id="edge42" class="edge">
+            <title>truc&#45;&gt;truck</title>
+            <path fill="none" stroke="#0000ff" d="M676.2369,-366.2345C676.2369,-351.2603 676.2369,-329.8643 676.2369,-312.7055"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="679.737,-312.6645 676.2369,-302.6646 672.737,-312.6646 679.737,-312.6645"/>
+            <text text-anchor="middle" x="681.4869" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> k</text>
+        </g>
+        <!-- truck&#45;&gt;truc -->
+        <g id="edge52" class="edge">
+            <title>truck&#45;&gt;truc</title>
+            <path fill="none" stroke="#ffc0cb" d="M683.5061,-302.4275C685.4432,-308.0852 687.266,-314.4172 688.2369,-320.4 690.1842,-332.3997 688.3971,-345.6086 685.6614,-356.9103"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="682.9623,-366.6639 681.2924,-355.8259 684.2958,-361.845 685.6294,-357.0261 685.6294,-357.0261 685.6294,-357.0261 684.2958,-361.845 689.9664,-358.2263 682.9623,-366.6639 682.9623,-366.6639"/>
+            <text text-anchor="middle" x="695.5232" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#ff0000">~4</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g1.svg
new file mode 100644
index 0000000000..e43b324e2b
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g1.svg
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="154pt" height="310pt"
+     viewBox="0.00 0.00 154.47 310.40" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 306.4)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-306.4 150.4738,-306.4 150.4738,4 -4,4"/>
+        <!-- 0x13B -->
+        <g id="node1" class="node">
+            <title>0x13B</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x13B</text>
+        </g>
+        <!-- contentArray[0] -->
+        <g id="node2" class="node">
+            <title>contentArray[0]</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- 0x13B&#45;&gt;contentArray[0] -->
+        <g id="edge1" class="edge">
+            <title>0x13B&#45;&gt;contentArray[0]</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-88.4006C73.2369,-76.2949 73.2369,-60.2076 73.2369,-46.4674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-46.072 73.2369,-36.072 69.737,-46.0721 76.737,-46.072"/>
+            <text text-anchor="middle" x="81.7902" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;A</text>
+        </g>
+        <!-- 0x13A -->
+        <g id="node3" class="node">
+            <title>0x13A</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-195.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x13A</text>
+        </g>
+        <!-- 0x13A&#45;&gt;0x13B -->
+        <g id="edge2" class="edge">
+            <title>0x13A&#45;&gt;0x13B</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-177.2006C73.2369,-165.0949 73.2369,-149.0076 73.2369,-135.2674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-134.872 73.2369,-124.872 69.737,-134.8721 76.737,-134.872"/>
+            <text text-anchor="middle" x="81.4052" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;H</text>
+        </g>
+        <!-- 0x139 -->
+        <g id="node4" class="node">
+            <title>0x139</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-284.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x139</text>
+        </g>
+        <!-- 0x139&#45;&gt;0x13A -->
+        <g id="edge3" class="edge">
+            <title>0x139&#45;&gt;0x13A</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-266.0006C73.2369,-253.8949 73.2369,-237.8076 73.2369,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-223.672 73.2369,-213.672 69.737,-223.6721 76.737,-223.672"/>
+            <text text-anchor="middle" x="81.4052" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;C</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g2.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g2.svg
new file mode 100644
index 0000000000..a5c7eed609
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g2.svg
@@ -0,0 +1,116 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+strict digraph G {
+  "0x139" [color=grey,fontcolor=grey];
+
+  "0x13B" -> "contentArray[0]" [label="  A"];
+  "0x13A" -> "0x13B" [label="  H"];
+  "0x139" -> "0x13A" [label="  C",color="grey",fontcolor="grey"];
+
+  "0x25E" -> "0x238" [label="  A"];
+  "0x25E" -> "0x13A" [label="  C"];
+//   "0x25E" -> "0x33B" [label="  5"];
+//   "0x25E" -> "0x35C" [label="  B"];
+//   "0x25E" -> "0x37A" [label="  P"];
+//   "0x25E" -> "0x41B" [label="  3"];
+}
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="207pt" height="310pt"
+     viewBox="0.00 0.00 207.49 310.40" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 306.4)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-306.4 203.4867,-306.4 203.4867,4 -4,4"/>
+        <!-- 0x139 -->
+        <g id="node1" class="node">
+            <title>0x139</title>
+            <ellipse fill="none" stroke="#c0c0c0" cx="73.2369" cy="-284.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#c0c0c0">0x139</text>
+        </g>
+        <!-- 0x13A -->
+        <g id="node4" class="node">
+            <title>0x13A</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-195.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x13A</text>
+        </g>
+        <!-- 0x139&#45;&gt;0x13A -->
+        <g id="edge3" class="edge">
+            <title>0x139&#45;&gt;0x13A</title>
+            <path fill="none" stroke="#c0c0c0" d="M73.2369,-266.0006C73.2369,-253.8949 73.2369,-237.8076 73.2369,-224.0674"/>
+            <polygon fill="#c0c0c0" stroke="#c0c0c0" points="76.737,-223.672 73.2369,-213.672 69.737,-223.6721 76.737,-223.672"/>
+            <text text-anchor="middle" x="81.4052" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#c0c0c0"> &#160;C</text>
+        </g>
+        <!-- 0x13B -->
+        <g id="node2" class="node">
+            <title>0x13B</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x13B</text>
+        </g>
+        <!-- contentArray[0] -->
+        <g id="node3" class="node">
+            <title>contentArray[0]</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- 0x13B&#45;&gt;contentArray[0] -->
+        <g id="edge1" class="edge">
+            <title>0x13B&#45;&gt;contentArray[0]</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-88.4006C73.2369,-76.2949 73.2369,-60.2076 73.2369,-46.4674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-46.072 73.2369,-36.072 69.737,-46.0721 76.737,-46.072"/>
+            <text text-anchor="middle" x="81.7902" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;A</text>
+        </g>
+        <!-- 0x13A&#45;&gt;0x13B -->
+        <g id="edge2" class="edge">
+            <title>0x13A&#45;&gt;0x13B</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-177.2006C73.2369,-165.0949 73.2369,-149.0076 73.2369,-135.2674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-134.872 73.2369,-124.872 69.737,-134.8721 76.737,-134.872"/>
+            <text text-anchor="middle" x="81.4052" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;H</text>
+        </g>
+        <!-- 0x25E -->
+        <g id="node5" class="node">
+            <title>0x25E</title>
+            <ellipse fill="none" stroke="#000000" cx="163.2369" cy="-284.4" rx="36.5014" ry="18"/>
+            <text text-anchor="middle" x="163.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x25E</text>
+        </g>
+        <!-- 0x25E&#45;&gt;0x13A -->
+        <g id="edge5" class="edge">
+            <title>0x25E&#45;&gt;0x13A</title>
+            <path fill="none" stroke="#000000" d="M146.7295,-268.1127C132.7742,-254.3435 112.6083,-234.4465 96.9293,-218.9765"/>
+            <polygon fill="#000000" stroke="#000000" points="99.2162,-216.3161 89.6396,-211.784 94.2998,-221.299 99.2162,-216.3161"/>
+            <text text-anchor="middle" x="135.4052" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;C</text>
+        </g>
+        <!-- 0x238 -->
+        <g id="node6" class="node">
+            <title>0x238</title>
+            <ellipse fill="none" stroke="#000000" cx="164.2369" cy="-195.6" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="164.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x238</text>
+        </g>
+        <!-- 0x25E&#45;&gt;0x238 -->
+        <g id="edge4" class="edge">
+            <title>0x25E&#45;&gt;0x238</title>
+            <path fill="none" stroke="#000000" d="M163.4441,-266.0006C163.5804,-253.8949 163.7616,-237.8076 163.9163,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="167.4205,-223.7108 164.0334,-213.672 160.4209,-223.6319 167.4205,-223.7108"/>
+            <text text-anchor="middle" x="171.7902" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;A</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g3.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g3.svg
new file mode 100644
index 0000000000..b0326c96dd
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g3.svg
@@ -0,0 +1,253 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+# http://www.graphviz.org/content/cluster
+
+strict digraph G {
+    "0x51C";
+
+    subgraph cluster_51C {
+        label= "Split node 0x51C";
+        color=grey;
+        node [style=dashed] "0x53C", "0x55C", "0x57C", "0x59C", "0x5BC";
+
+        "0x51C" -> "0x53C" [label="  00"];
+        "0x51C" -> "0x57C" [label="  01"];
+        "0x53C" -> "0x55C" [label="  110"];
+        "0x57C" -> "0x59C" [label="  000"];
+        "0x57C" -> "0x5BC" [label="  010"];
+    }
+
+    "0x55C" -> "0x41B" [label="  011"];
+    "0x55C" -> "0x33B" [label="  101"];
+    "0x59C" -> "0x238" [label="  001"];
+    "0x59C" -> "0x35C" [label="  010"];
+    "0x59C" -> "0x13A" [label="  011"];
+    "0x5BC" -> "0x37A" [label="  000"];
+    "0x5BC" -> "0x455" [label="  001"];
+
+    "0x13B" -> "contentArray[0]" [label="  A"];
+    "0x13A" -> "0x13B" [label="  H"];
+}
+
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="628pt" height="605pt"
+     viewBox="0.00 0.00 627.97 604.80" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 600.8)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-600.8 623.9695,-600.8 623.9695,4 -4,4"/>
+        <g id="clust1" class="cluster">
+            <title>cluster_51C</title>
+            <polygon fill="none" stroke="#c0c0c0" points="177.7906,-258.4 177.7906,-588.8 444.7906,-588.8 444.7906,-258.4 177.7906,-258.4"/>
+            <text text-anchor="middle" x="311.2906" y="-572.2" font-family="Times,serif" font-size="14.00" fill="#000000">Split node 0x51C</text>
+        </g>
+        <!-- 0x51C -->
+        <g id="node1" class="node">
+            <title>0x51C</title>
+            <ellipse fill="none" stroke="#000000" cx="264.7906" cy="-538" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="264.7906" y="-533.8" font-family="Times,serif" font-size="14.00" fill="#000000">0x51C</text>
+        </g>
+        <!-- 0x53C -->
+        <g id="node2" class="node">
+            <title>0x53C</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="220.7906" cy="-411.2" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="220.7906" y="-407" font-family="Times,serif" font-size="14.00" fill="#000000">0x53C</text>
+        </g>
+        <!-- 0x51C&#45;&gt;0x53C -->
+        <g id="edge1" class="edge">
+            <title>0x51C&#45;&gt;0x53C</title>
+            <path fill="none" stroke="#000000" d="M258.5798,-520.1016C251.2095,-498.8616 238.8421,-463.2211 230.2726,-438.5254"/>
+            <polygon fill="#000000" stroke="#000000" points="233.5547,-437.3073 226.9698,-429.0074 226.9416,-439.6022 233.5547,-437.3073"/>
+            <text text-anchor="middle" x="256.2906" y="-470.4" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;00</text>
+        </g>
+        <!-- 0x57C -->
+        <g id="node4" class="node">
+            <title>0x57C</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="308.7906" cy="-411.2" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="308.7906" y="-407" font-family="Times,serif" font-size="14.00" fill="#000000">0x57C</text>
+        </g>
+        <!-- 0x51C&#45;&gt;0x57C -->
+        <g id="edge2" class="edge">
+            <title>0x51C&#45;&gt;0x57C</title>
+            <path fill="none" stroke="#000000" d="M271.0014,-520.1016C278.3718,-498.8616 290.7391,-463.2211 299.3086,-438.5254"/>
+            <polygon fill="#000000" stroke="#000000" points="302.6397,-439.6022 302.6114,-429.0074 296.0265,-437.3073 302.6397,-439.6022"/>
+            <text text-anchor="middle" x="300.2906" y="-470.4" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;01</text>
+        </g>
+        <!-- 0x55C -->
+        <g id="node3" class="node">
+            <title>0x55C</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="220.7906" cy="-284.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="220.7906" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x55C</text>
+        </g>
+        <!-- 0x53C&#45;&gt;0x55C -->
+        <g id="edge3" class="edge">
+            <title>0x53C&#45;&gt;0x55C</title>
+            <path fill="none" stroke="#000000" d="M220.7906,-393.0327C220.7906,-372.0352 220.7906,-337.2261 220.7906,-312.679"/>
+            <polygon fill="#000000" stroke="#000000" points="224.2907,-312.5336 220.7906,-302.5336 217.2907,-312.5337 224.2907,-312.5336"/>
+            <text text-anchor="middle" x="234.7906" y="-343.6" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;110</text>
+        </g>
+        <!-- 0x41B -->
+        <g id="node7" class="node">
+            <title>0x41B</title>
+            <ellipse fill="none" stroke="#000000" cx="36.7906" cy="-195.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="36.7906" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x41B</text>
+        </g>
+        <!-- 0x55C&#45;&gt;0x41B -->
+        <g id="edge6" class="edge">
+            <title>0x55C&#45;&gt;0x41B</title>
+            <path fill="none" stroke="#000000" d="M192.599,-273.4087C175.6606,-266.6053 153.7802,-257.4635 134.7906,-248.4 112.7572,-237.8837 88.6053,-224.889 69.791,-214.4226"/>
+            <polygon fill="#000000" stroke="#000000" points="71.4518,-211.3412 61.0164,-209.5097 68.0319,-217.449 71.4518,-211.3412"/>
+            <text text-anchor="middle" x="148.7906" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;011</text>
+        </g>
+        <!-- 0x33B -->
+        <g id="node8" class="node">
+            <title>0x33B</title>
+            <ellipse fill="none" stroke="#000000" cx="128.7906" cy="-195.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="128.7906" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x33B</text>
+        </g>
+        <!-- 0x55C&#45;&gt;0x33B -->
+        <g id="edge7" class="edge">
+            <title>0x55C&#45;&gt;0x33B</title>
+            <path fill="none" stroke="#000000" d="M203.9164,-268.1127C189.651,-254.3435 169.037,-234.4465 153.0095,-218.9765"/>
+            <polygon fill="#000000" stroke="#000000" points="155.1837,-216.2106 145.5579,-211.784 150.3223,-221.2472 155.1837,-216.2106"/>
+            <text text-anchor="middle" x="197.7906" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;101</text>
+        </g>
+        <!-- 0x59C -->
+        <g id="node5" class="node">
+            <title>0x59C</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="308.7906" cy="-284.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="308.7906" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x59C</text>
+        </g>
+        <!-- 0x57C&#45;&gt;0x59C -->
+        <g id="edge4" class="edge">
+            <title>0x57C&#45;&gt;0x59C</title>
+            <path fill="none" stroke="#000000" d="M308.7906,-393.0327C308.7906,-372.0352 308.7906,-337.2261 308.7906,-312.679"/>
+            <polygon fill="#000000" stroke="#000000" points="312.2907,-312.5336 308.7906,-302.5336 305.2907,-312.5337 312.2907,-312.5336"/>
+            <text text-anchor="middle" x="322.7906" y="-343.6" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;000</text>
+        </g>
+        <!-- 0x5BC -->
+        <g id="node6" class="node">
+            <title>0x5BC</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="399.7906" cy="-284.4" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="399.7906" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x5BC</text>
+        </g>
+        <!-- 0x57C&#45;&gt;0x5BC -->
+        <g id="edge5" class="edge">
+            <title>0x57C&#45;&gt;0x5BC</title>
+            <path fill="none" stroke="#000000" d="M321.0633,-394.0992C336.5995,-372.451 363.5002,-334.9674 381.471,-309.9267"/>
+            <polygon fill="#000000" stroke="#000000" points="384.4347,-311.7999 387.4218,-301.6348 378.7476,-307.7185 384.4347,-311.7999"/>
+            <text text-anchor="middle" x="373.7906" y="-343.6" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;010</text>
+        </g>
+        <!-- 0x238 -->
+        <g id="node9" class="node">
+            <title>0x238</title>
+            <ellipse fill="none" stroke="#000000" cx="218.7906" cy="-195.6" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="218.7906" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x238</text>
+        </g>
+        <!-- 0x59C&#45;&gt;0x238 -->
+        <g id="edge8" class="edge">
+            <title>0x59C&#45;&gt;0x238</title>
+            <path fill="none" stroke="#000000" d="M292.2832,-268.1127C278.3279,-254.3435 258.162,-234.4465 242.483,-218.9765"/>
+            <polygon fill="#000000" stroke="#000000" points="244.77,-216.3161 235.1934,-211.784 239.8535,-221.299 244.77,-216.3161"/>
+            <text text-anchor="middle" x="286.7906" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;001</text>
+        </g>
+        <!-- 0x35C -->
+        <g id="node10" class="node">
+            <title>0x35C</title>
+            <ellipse fill="none" stroke="#000000" cx="308.7906" cy="-195.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="308.7906" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x35C</text>
+        </g>
+        <!-- 0x59C&#45;&gt;0x35C -->
+        <g id="edge9" class="edge">
+            <title>0x59C&#45;&gt;0x35C</title>
+            <path fill="none" stroke="#000000" d="M308.7906,-266.0006C308.7906,-253.8949 308.7906,-237.8076 308.7906,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="312.2907,-223.672 308.7906,-213.672 305.2907,-223.6721 312.2907,-223.672"/>
+            <text text-anchor="middle" x="322.7906" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;010</text>
+        </g>
+        <!-- 0x13A -->
+        <g id="node11" class="node">
+            <title>0x13A</title>
+            <ellipse fill="none" stroke="#000000" cx="400.7906" cy="-195.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="400.7906" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x13A</text>
+        </g>
+        <!-- 0x59C&#45;&gt;0x13A -->
+        <g id="edge10" class="edge">
+            <title>0x59C&#45;&gt;0x13A</title>
+            <path fill="none" stroke="#000000" d="M325.6648,-268.1127C339.9303,-254.3435 360.5443,-234.4465 376.5717,-218.9765"/>
+            <polygon fill="#000000" stroke="#000000" points="379.259,-221.2472 384.0234,-211.784 374.3976,-216.2106 379.259,-221.2472"/>
+            <text text-anchor="middle" x="377.7906" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;011</text>
+        </g>
+        <!-- 0x37A -->
+        <g id="node12" class="node">
+            <title>0x37A</title>
+            <ellipse fill="none" stroke="#000000" cx="493.7906" cy="-195.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="493.7906" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x37A</text>
+        </g>
+        <!-- 0x5BC&#45;&gt;0x37A -->
+        <g id="edge11" class="edge">
+            <title>0x5BC&#45;&gt;0x37A</title>
+            <path fill="none" stroke="#000000" d="M417.0317,-268.1127C431.6072,-254.3435 452.6693,-234.4465 469.0452,-218.9765"/>
+            <polygon fill="#000000" stroke="#000000" points="471.7931,-221.1955 476.6589,-211.784 466.9861,-216.107 471.7931,-221.1955"/>
+            <text text-anchor="middle" x="469.7906" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;000</text>
+        </g>
+        <!-- 0x455 -->
+        <g id="node13" class="node">
+            <title>0x455</title>
+            <ellipse fill="none" stroke="#000000" cx="584.7906" cy="-195.6" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="584.7906" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x455</text>
+        </g>
+        <!-- 0x5BC&#45;&gt;0x455 -->
+        <g id="edge12" class="edge">
+            <title>0x5BC&#45;&gt;0x455</title>
+            <path fill="none" stroke="#000000" d="M429.4765,-273.1983C446.7051,-266.4645 468.7051,-257.4645 487.7906,-248.4 509.7186,-237.9855 533.6828,-224.9577 552.3055,-214.4487"/>
+            <polygon fill="#000000" stroke="#000000" points="554.288,-217.3478 561.2515,-209.3627 550.8284,-211.2625 554.288,-217.3478"/>
+            <text text-anchor="middle" x="533.7906" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;001</text>
+        </g>
+        <!-- 0x13B -->
+        <g id="node14" class="node">
+            <title>0x13B</title>
+            <ellipse fill="none" stroke="#000000" cx="400.7906" cy="-106.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="400.7906" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x13B</text>
+        </g>
+        <!-- 0x13A&#45;&gt;0x13B -->
+        <g id="edge14" class="edge">
+            <title>0x13A&#45;&gt;0x13B</title>
+            <path fill="none" stroke="#000000" d="M400.7906,-177.2006C400.7906,-165.0949 400.7906,-149.0076 400.7906,-135.2674"/>
+            <polygon fill="#000000" stroke="#000000" points="404.2907,-134.872 400.7906,-124.872 397.2907,-134.8721 404.2907,-134.872"/>
+            <text text-anchor="middle" x="409.3439" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;H</text>
+        </g>
+        <!-- contentArray[0] -->
+        <g id="node15" class="node">
+            <title>contentArray[0]</title>
+            <ellipse fill="none" stroke="#000000" cx="400.7906" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="400.7906" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- 0x13B&#45;&gt;contentArray[0] -->
+        <g id="edge13" class="edge">
+            <title>0x13B&#45;&gt;contentArray[0]</title>
+            <path fill="none" stroke="#000000" d="M400.7906,-88.4006C400.7906,-76.2949 400.7906,-60.2076 400.7906,-46.4674"/>
+            <polygon fill="#000000" stroke="#000000" points="404.2907,-46.072 400.7906,-36.072 397.2907,-46.0721 404.2907,-46.072"/>
+            <text text-anchor="middle" x="409.3439" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;A</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g4.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g4.svg
new file mode 100644
index 0000000000..152377272b
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g4.svg
@@ -0,0 +1,290 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+# http://www.graphviz.org/content/cluster
+
+strict digraph G {
+//  "0x139" [color=grey,fontcolor=grey];
+
+  "0x13B" -> "contentArray[0]" [label="  A"];
+  "0x13A" -> "0x13B" [label="  H"];
+//   "0x139" -> "0x13A" [label="  C"];
+//  "0x139" -> "0x13A" [label="  C",color="grey",fontcolor="grey"];
+
+//   "0x25E" -> "0x238" [label="  A"];
+//   "0x25E" -> "0x13A" [label="  C"];
+//   "0x25E" -> "0x33B" [label="  5"];
+//   "0x25E" -> "0x35C" [label="  B"];
+//   "0x25E" -> "0x37A" [label="  P"];
+//   "0x25E" -> "0x41B" [label="  3"];
+
+  subgraph cluster_51F {
+    label = "Node 0x51F"
+    "0x51F" [label="Prefix 0x51F\ncontentArray[1]"]
+    "0x51F" -> "0x51C" [label="ε"];
+
+    subgraph cluster_51C {
+      label= "Split node 0x51C";
+      ranksep=1
+      color=grey;
+      node [style=dashed] "0x53C", "0x55C", "0x57C", "0x59C", "0x5BC";
+
+      "0x51C" -> "0x53C" [label="  00"];
+      "0x51C" -> "0x57C" [label="  01"];
+      "0x53C" -> "0x55C" [label="  110"];
+      "0x57C" -> "0x59C" [label="  000"];
+      "0x57C" -> "0x5BC" [label="  010"];
+    }
+  }
+  "0x55C" -> "0x41B" [label="  011",minlen=2];
+  "0x55C" -> "0x33B" [label="  101",minlen=2];
+  "0x59C" -> "0x238" [label="  001",minlen=2];
+  "0x59C" -> "0x35C" [label="  010",minlen=2];
+  "0x59C" -> "0x13A" [label="  011",minlen=2];
+  "0x5BC" -> "0x37A" [label="  000",minlen=2];
+  "0x5BC" -> "0x455" [label="  001",minlen=2];
+
+  { rank=same "0x238" -> "0x35C" -> "0x13A" [style=invis,constrain=false]}
+}
+-->
+<!-- Title: G Pages: 1 -->
+<svg width="663pt" height="739pt"
+     viewBox="0.00 0.00 662.97 739.23" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 735.2313)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-735.2313 658.9695,-735.2313 658.9695,4 -4,4"/>
+        <g id="clust1" class="cluster">
+            <title>cluster_51F</title>
+            <polygon fill="none" stroke="#000000" points="186.7906,-231.4 186.7906,-723.2313 469.7906,-723.2313 469.7906,-231.4 186.7906,-231.4"/>
+            <text text-anchor="middle" x="328.2906" y="-706.6313" font-family="Times,serif" font-size="14.00" fill="#000000">Node 0x51F</text>
+        </g>
+        <g id="clust2" class="cluster">
+            <title>cluster_51C</title>
+            <polygon fill="none" stroke="#c0c0c0" points="194.7906,-239.4 194.7906,-577.8 461.7906,-577.8 461.7906,-239.4 194.7906,-239.4"/>
+            <text text-anchor="middle" x="328.2906" y="-561.2" font-family="Times,serif" font-size="14.00" fill="#000000">Split node 0x51C</text>
+        </g>
+        <!-- 0x13B -->
+        <g id="node1" class="node">
+            <title>0x13B</title>
+            <ellipse fill="none" stroke="#000000" cx="435.7906" cy="-88.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="435.7906" y="-84.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x13B</text>
+        </g>
+        <!-- contentArray[0] -->
+        <g id="node2" class="node">
+            <title>contentArray[0]</title>
+            <ellipse fill="none" stroke="#000000" cx="435.7906" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="435.7906" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- 0x13B&#45;&gt;contentArray[0] -->
+        <g id="edge1" class="edge">
+            <title>0x13B&#45;&gt;contentArray[0]</title>
+            <path fill="none" stroke="#000000" d="M435.7906,-70.5672C435.7906,-63.2743 435.7906,-54.6987 435.7906,-46.6137"/>
+            <polygon fill="#000000" stroke="#000000" points="439.2907,-46.417 435.7906,-36.417 432.2907,-46.4171 439.2907,-46.417"/>
+            <text text-anchor="middle" x="444.3439" y="-49.2" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;A</text>
+        </g>
+        <!-- 0x13A -->
+        <g id="node3" class="node">
+            <title>0x13A</title>
+            <ellipse fill="none" stroke="#000000" cx="435.7906" cy="-159.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="435.7906" y="-155.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x13A</text>
+        </g>
+        <!-- 0x13A&#45;&gt;0x13B -->
+        <g id="edge2" class="edge">
+            <title>0x13A&#45;&gt;0x13B</title>
+            <path fill="none" stroke="#000000" d="M435.7906,-141.3672C435.7906,-134.0743 435.7906,-125.4987 435.7906,-117.4137"/>
+            <polygon fill="#000000" stroke="#000000" points="439.2907,-117.217 435.7906,-107.217 432.2907,-117.2171 439.2907,-117.217"/>
+            <text text-anchor="middle" x="444.3439" y="-120" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;H</text>
+        </g>
+        <!-- 0x51F -->
+        <g id="node4" class="node">
+            <title>0x51F</title>
+            <ellipse fill="none" stroke="#000000" cx="318.7906" cy="-661.0156" rx="117.2629" ry="29.3315"/>
+            <text text-anchor="middle" x="318.7906" y="-665.2156" font-family="Times,serif" font-size="14.00" fill="#000000">Prefix 0x51F</text>
+            <text text-anchor="middle" x="318.7906" y="-648.4156" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- 0x51C -->
+        <g id="node5" class="node">
+            <title>0x51C</title>
+            <ellipse fill="none" stroke="#000000" cx="318.7906" cy="-527" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="318.7906" y="-522.8" font-family="Times,serif" font-size="14.00" fill="#000000">0x51C</text>
+        </g>
+        <!-- 0x51F&#45;&gt;0x51C -->
+        <g id="edge3" class="edge">
+            <title>0x51F&#45;&gt;0x51C</title>
+            <path fill="none" stroke="#000000" d="M318.7906,-631.2732C318.7906,-608.6091 318.7906,-577.6052 318.7906,-555.3044"/>
+            <polygon fill="#000000" stroke="#000000" points="322.2907,-555.1305 318.7906,-545.1306 315.2907,-555.1306 322.2907,-555.1305"/>
+            <text text-anchor="middle" x="322.8709" y="-600" font-family="Times,serif" font-size="14.00" fill="#000000">ε</text>
+        </g>
+        <!-- 0x53C -->
+        <g id="node6" class="node">
+            <title>0x53C</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="237.7906" cy="-396.2" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="237.7906" y="-392" font-family="Times,serif" font-size="14.00" fill="#000000">0x53C</text>
+        </g>
+        <!-- 0x51C&#45;&gt;0x53C -->
+        <g id="edge4" class="edge">
+            <title>0x51C&#45;&gt;0x53C</title>
+            <path fill="none" stroke="#000000" d="M284.1131,-520.6804C267.7647,-515.4937 251.7906,-506.1624 251.7906,-489.5 251.7906,-489.5 251.7906,-489.5 251.7906,-433.7 251.7906,-430.2948 251.2777,-426.8193 250.4482,-423.4252"/>
+            <polygon fill="#000000" stroke="#000000" points="253.6734,-422.0307 247.2065,-413.6385 247.0285,-424.2318 253.6734,-422.0307"/>
+            <text text-anchor="middle" x="262.2906" y="-457.4" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;00</text>
+        </g>
+        <!-- 0x57C -->
+        <g id="node8" class="node">
+            <title>0x57C</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="325.7906" cy="-396.2" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="325.7906" y="-392" font-family="Times,serif" font-size="14.00" fill="#000000">0x57C</text>
+        </g>
+        <!-- 0x51C&#45;&gt;0x57C -->
+        <g id="edge5" class="edge">
+            <title>0x51C&#45;&gt;0x57C</title>
+            <path fill="none" stroke="#000000" d="M320.8414,-508.7129C321.3687,-502.6109 321.7906,-495.77 321.7906,-489.5 321.7906,-489.5 321.7906,-489.5 321.7906,-433.7 321.7906,-430.7537 321.9142,-427.6845 322.118,-424.6266"/>
+            <polygon fill="#000000" stroke="#000000" points="325.6215,-424.7486 323.0562,-414.469 318.6512,-424.1046 325.6215,-424.7486"/>
+            <text text-anchor="middle" x="332.2906" y="-457.4" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;01</text>
+        </g>
+        <!-- 0x55C -->
+        <g id="node7" class="node">
+            <title>0x55C</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="237.7906" cy="-265.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="237.7906" y="-261.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x55C</text>
+        </g>
+        <!-- 0x53C&#45;&gt;0x55C -->
+        <g id="edge6" class="edge">
+            <title>0x53C&#45;&gt;0x55C</title>
+            <path fill="none" stroke="#000000" d="M237.7906,-377.8895C237.7906,-371.7859 237.7906,-364.95 237.7906,-358.7 237.7906,-358.7 237.7906,-358.7 237.7906,-302.9 237.7906,-299.9703 237.7906,-296.9119 237.7906,-293.8605"/>
+            <polygon fill="#000000" stroke="#000000" points="241.2907,-293.7105 237.7906,-283.7105 234.2907,-293.7106 241.2907,-293.7105"/>
+            <text text-anchor="middle" x="251.7906" y="-326.6" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;110</text>
+        </g>
+        <!-- 0x41B -->
+        <g id="node11" class="node">
+            <title>0x41B</title>
+            <ellipse fill="none" stroke="#000000" cx="36.7906" cy="-159.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="36.7906" y="-155.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x41B</text>
+        </g>
+        <!-- 0x55C&#45;&gt;0x41B -->
+        <g id="edge9" class="edge">
+            <title>0x55C&#45;&gt;0x41B</title>
+            <path fill="none" stroke="#000000" d="M212.5449,-252.7617C191.8449,-242.3434 161.8208,-227.105 135.7906,-213.4 113.9245,-201.8874 89.5312,-188.6361 70.4296,-178.1656"/>
+            <polygon fill="#000000" stroke="#000000" points="71.9619,-175.014 61.5118,-173.2686 68.5926,-181.1498 71.9619,-175.014"/>
+            <text text-anchor="middle" x="149.7906" y="-200.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;011</text>
+        </g>
+        <!-- 0x33B -->
+        <g id="node12" class="node">
+            <title>0x33B</title>
+            <ellipse fill="none" stroke="#000000" cx="128.7906" cy="-159.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="128.7906" y="-155.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x33B</text>
+        </g>
+        <!-- 0x55C&#45;&gt;0x33B -->
+        <g id="edge10" class="edge">
+            <title>0x55C&#45;&gt;0x33B</title>
+            <path fill="none" stroke="#000000" d="M221.2359,-249.3313C202.9895,-231.6206 173.4782,-202.9757 152.7171,-182.824"/>
+            <polygon fill="#000000" stroke="#000000" points="155.1145,-180.2735 145.5012,-175.82 150.2391,-185.2964 155.1145,-180.2735"/>
+            <text text-anchor="middle" x="197.7906" y="-200.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;101</text>
+        </g>
+        <!-- 0x59C -->
+        <g id="node9" class="node">
+            <title>0x59C</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="325.7906" cy="-265.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="325.7906" y="-261.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x59C</text>
+        </g>
+        <!-- 0x57C&#45;&gt;0x59C -->
+        <g id="edge7" class="edge">
+            <title>0x57C&#45;&gt;0x59C</title>
+            <path fill="none" stroke="#000000" d="M325.7906,-377.8895C325.7906,-371.7859 325.7906,-364.95 325.7906,-358.7 325.7906,-358.7 325.7906,-358.7 325.7906,-302.9 325.7906,-299.9703 325.7906,-296.9119 325.7906,-293.8605"/>
+            <polygon fill="#000000" stroke="#000000" points="329.2907,-293.7105 325.7906,-283.7105 322.2907,-293.7106 329.2907,-293.7105"/>
+            <text text-anchor="middle" x="339.7906" y="-326.6" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;000</text>
+        </g>
+        <!-- 0x5BC -->
+        <g id="node10" class="node">
+            <title>0x5BC</title>
+            <ellipse fill="none" stroke="#000000" stroke-dasharray="5,2" cx="416.7906" cy="-265.4" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="416.7906" y="-261.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x5BC</text>
+        </g>
+        <!-- 0x57C&#45;&gt;0x5BC -->
+        <g id="edge8" class="edge">
+            <title>0x57C&#45;&gt;0x5BC</title>
+            <path fill="none" stroke="#000000" d="M356.6658,-387.2435C378.0878,-380.079 402.7906,-369.4801 402.7906,-358.7 402.7906,-358.7 402.7906,-358.7 402.7906,-302.9 402.7906,-299.4948 403.3035,-296.0193 404.133,-292.6252"/>
+            <polygon fill="#000000" stroke="#000000" points="407.5528,-293.4318 407.3747,-282.8385 400.9078,-291.2307 407.5528,-293.4318"/>
+            <text text-anchor="middle" x="416.7906" y="-326.6" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;010</text>
+        </g>
+        <!-- 0x59C&#45;&gt;0x13A -->
+        <g id="edge13" class="edge">
+            <title>0x59C&#45;&gt;0x13A</title>
+            <path fill="none" stroke="#000000" d="M342.4972,-249.3313C360.911,-231.6206 390.693,-202.9757 411.6447,-182.824"/>
+            <polygon fill="#000000" stroke="#000000" points="414.1457,-185.2747 418.9268,-175.82 409.2931,-180.2296 414.1457,-185.2747"/>
+            <text text-anchor="middle" x="410.7906" y="-200.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;011</text>
+        </g>
+        <!-- 0x238 -->
+        <g id="node13" class="node">
+            <title>0x238</title>
+            <ellipse fill="none" stroke="#000000" cx="218.7906" cy="-159.6" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="218.7906" y="-155.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x238</text>
+        </g>
+        <!-- 0x59C&#45;&gt;0x238 -->
+        <g id="edge11" class="edge">
+            <title>0x59C&#45;&gt;0x238</title>
+            <path fill="none" stroke="#000000" d="M309.5396,-249.3313C291.7067,-231.6983 262.9122,-203.2267 242.5467,-183.0897"/>
+            <polygon fill="#000000" stroke="#000000" points="244.7663,-180.3623 235.1945,-175.82 239.8445,-185.3399 244.7663,-180.3623"/>
+            <text text-anchor="middle" x="286.7906" y="-200.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;001</text>
+        </g>
+        <!-- 0x35C -->
+        <g id="node14" class="node">
+            <title>0x35C</title>
+            <ellipse fill="none" stroke="#000000" cx="325.7906" cy="-159.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="325.7906" y="-155.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x35C</text>
+        </g>
+        <!-- 0x59C&#45;&gt;0x35C -->
+        <g id="edge12" class="edge">
+            <title>0x59C&#45;&gt;0x35C</title>
+            <path fill="none" stroke="#000000" d="M325.7906,-246.971C325.7906,-230.6622 325.7906,-206.6111 325.7906,-187.8698"/>
+            <polygon fill="#000000" stroke="#000000" points="329.2907,-187.8177 325.7906,-177.8178 322.2907,-187.8178 329.2907,-187.8177"/>
+            <text text-anchor="middle" x="339.7906" y="-200.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;010</text>
+        </g>
+        <!-- 0x37A -->
+        <g id="node15" class="node">
+            <title>0x37A</title>
+            <ellipse fill="none" stroke="#000000" cx="528.7906" cy="-159.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="528.7906" y="-155.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x37A</text>
+        </g>
+        <!-- 0x5BC&#45;&gt;0x37A -->
+        <g id="edge14" class="edge">
+            <title>0x5BC&#45;&gt;0x37A</title>
+            <path fill="none" stroke="#000000" d="M433.801,-249.3313C452.5495,-231.6206 482.8731,-202.9757 504.2057,-182.824"/>
+            <polygon fill="#000000" stroke="#000000" points="506.7542,-185.2313 511.6201,-175.82 501.9472,-180.1427 506.7542,-185.2313"/>
+            <text text-anchor="middle" x="502.7906" y="-200.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;000</text>
+        </g>
+        <!-- 0x455 -->
+        <g id="node16" class="node">
+            <title>0x455</title>
+            <ellipse fill="none" stroke="#000000" cx="619.7906" cy="-159.6" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="619.7906" y="-155.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x455</text>
+        </g>
+        <!-- 0x5BC&#45;&gt;0x455 -->
+        <g id="edge15" class="edge">
+            <title>0x5BC&#45;&gt;0x455</title>
+            <path fill="none" stroke="#000000" d="M443.4197,-252.397C464.5065,-242.0254 494.6583,-227.0286 520.7906,-213.4 542.9996,-201.8175 567.7596,-188.3619 586.9624,-177.8047"/>
+            <polygon fill="#000000" stroke="#000000" points="588.838,-180.7674 595.9072,-172.8758 585.4597,-174.6365 588.838,-180.7674"/>
+            <text text-anchor="middle" x="564.7906" y="-200.8" font-family="Times,serif" font-size="14.00" fill="#000000"> &#160;001</text>
+        </g>
+        <!-- 0x238&#45;&gt;0x35C -->
+        <!-- 0x35C&#45;&gt;0x13A -->
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m1.svg
new file mode 100644
index 0000000000..ff928a44db
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m1.svg
@@ -0,0 +1,349 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+# embedded DOT (plantuml) works in IDEA preview
+# but not on GitHub
+```plantuml
+digraph G {
+    { rank=same root -> start [style=invis] }
+    start [label="start/end"]
+
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"]
+
+    tree [label = "contentArray[1]"]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    root -> t [label = " t"]
+    t -> tr [label = " r"]
+    tr -> tra [label = " a"]
+    tra -> trac [label = " c"]
+    trac -> tract [label = " t"]
+    tract -> tracto [label = " o"]
+    tracto -> tractor [label = " r"]
+
+    tr -> tre [label = " e"]
+    tre -> tree [label = " e"]
+
+    tr -> tri [label = " i"]
+    tri -> trie [label = " e"]
+
+    // {rank=same tra -> tre -> tri [style=invis]}
+    {rank=same trac -> tree -> trie [style=invis]}
+
+    subgraph path {
+        edge [color = "lightblue"; fontcolor="blue"; arrowhead="vee"]
+        node [color = "lightblue"; fontcolor="blue"]
+
+        start -> root
+
+        root -> t [label = " t"]
+        t -> tr [label = " r"]
+        tr -> tra [label = " a"]
+        tra -> trav [label = " v"]
+        trav -> trave [label = " e"]
+        trave -> traver [label = " r"]
+        traver -> travers [label = " s"]
+        travers -> traverse [label = " e"]
+
+        trav [label = "NONE"]
+        trave [label = "NONE"]
+        traver [label = "NONE"]
+        travers [label = "NONE"]
+        traverse [label = "NONE"]
+    }
+}
+```
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="575pt" height="823pt"
+     viewBox="0.00 0.00 575.47 822.89" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 818.8939)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-818.8939 571.4738,-818.8939 571.4738,4 -4,4"/>
+        <!-- root -->
+        <g id="node1" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="279.2369" cy="-785.4782" rx="33.1337" ry="29.3315"/>
+            <text text-anchor="middle" x="279.2369" y="-789.6782" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="279.2369" y="-772.8782" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- start -->
+        <g id="node2" class="node">
+            <title>start</title>
+            <ellipse fill="none" stroke="#000000" cx="392.2369" cy="-785.4782" rx="44.0775" ry="18"/>
+            <text text-anchor="middle" x="392.2369" y="-781.2782" font-family="Times,serif" font-size="14.00" fill="#000000">start/end</text>
+        </g>
+        <!-- root&#45;&gt;start -->
+        <!-- t -->
+        <g id="node13" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="279.2369" cy="-685.2626" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="279.2369" y="-681.0626" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge2" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#000000" d="M269.3368,-757.0824C266.6104,-745.9368 264.8794,-733.0211 266.8463,-721.2626 267.3268,-718.39 268.009,-715.4398 268.8091,-712.5238"/>
+            <polygon fill="#000000" stroke="#000000" points="272.1732,-713.494 271.8627,-702.9037 265.5012,-711.3762 272.1732,-713.494"/>
+            <text text-anchor="middle" x="270.9322" y="-725.4626" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge16" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#add8e6" d="M279.2369,-755.9522C279.2369,-742.4671 279.2369,-726.5749 279.2369,-713.2779"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="279.2369,-703.2659 283.737,-713.2659 279.2369,-708.2659 279.237,-713.2659 279.237,-713.2659 279.237,-713.2659 279.2369,-708.2659 274.737,-713.2659 279.2369,-703.2659 279.2369,-703.2659"/>
+            <text text-anchor="middle" x="282.9322" y="-725.4626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> t</text>
+        </g>
+        <!-- start&#45;&gt;root -->
+        <g id="edge15" class="edge">
+            <title>start&#45;&gt;root</title>
+            <path fill="none" stroke="#add8e6" d="M367.6112,-800.5185C350.7348,-807.9537 333.8585,-809.4839 316.9821,-805.109"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="307.0455,-801.8508 317.9498,-800.6907 311.7966,-803.4087 316.5477,-804.9667 316.5477,-804.9667 316.5477,-804.9667 311.7966,-803.4087 315.1456,-809.2427 307.0455,-801.8508 307.0455,-801.8508"/>
+        </g>
+        <!-- tractor -->
+        <g id="node3" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tracto -->
+        <g id="node4" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="91.2369" cy="-195.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="91.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge8" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#000000" d="M87.5943,-177.63C85.1221,-165.4338 81.802,-149.0543 78.9772,-135.1187"/>
+            <polygon fill="#000000" stroke="#000000" points="82.3528,-134.1534 76.9358,-125.0481 75.4923,-135.5441 82.3528,-134.1534"/>
+            <text text-anchor="middle" x="88.3172" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- tract -->
+        <g id="node5" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="91.2369" cy="-284.4" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="91.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge7" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#000000" d="M91.2369,-266.0006C91.2369,-253.8949 91.2369,-237.8076 91.2369,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="94.737,-223.672 91.2369,-213.672 87.737,-223.6721 94.737,-223.672"/>
+            <text text-anchor="middle" x="96.4869" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> o</text>
+        </g>
+        <!-- trac -->
+        <g id="node6" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="92.2369" cy="-373.2" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="92.2369" y="-369" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge6" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#000000" d="M92.0297,-354.8006C91.8934,-342.6949 91.7122,-326.6076 91.5575,-312.8674"/>
+            <polygon fill="#000000" stroke="#000000" points="95.0529,-312.4319 91.4404,-302.472 88.0533,-312.5108 95.0529,-312.4319"/>
+            <text text-anchor="middle" x="95.9322" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- tree -->
+        <g id="node8" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#000000" cx="312.2369" cy="-373.2" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="312.2369" y="-369" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- trac&#45;&gt;tree -->
+        <!-- tra -->
+        <g id="node7" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#000000" cx="203.2369" cy="-473.4156" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="203.2369" y="-477.6156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="203.2369" y="-460.8156" font-family="Times,serif" font-size="14.00" fill="#000000">0x018</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge5" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#000000" d="M178.8775,-451.4229C160.5618,-434.8867 135.5561,-412.3105 117.1156,-395.6616"/>
+            <polygon fill="#000000" stroke="#000000" points="119.421,-393.0276 109.6531,-388.9241 114.7301,-398.2233 119.421,-393.0276"/>
+            <text text-anchor="middle" x="153.0942" y="-413.4" font-family="Times,serif" font-size="14.00" fill="#000000"> c</text>
+        </g>
+        <!-- trav -->
+        <g id="node14" class="node">
+            <title>trav</title>
+            <ellipse fill="none" stroke="#add8e6" cx="183.2369" cy="-373.2" rx="37.7006" ry="18"/>
+            <text text-anchor="middle" x="183.2369" y="-369" font-family="Times,serif" font-size="14.00" fill="#0000ff">NONE</text>
+        </g>
+        <!-- tra&#45;&gt;trav -->
+        <g id="edge19" class="edge">
+            <title>tra&#45;&gt;trav</title>
+            <path fill="none" stroke="#add8e6" d="M197.3983,-444.1597C194.7054,-430.666 191.5224,-414.7168 188.8568,-401.3604"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="186.8493,-391.3011 193.2195,-400.227 187.8279,-396.2044 188.8065,-401.1078 188.8065,-401.1078 188.8065,-401.1078 187.8279,-396.2044 184.3935,-401.9885 186.8493,-391.3011 186.8493,-391.3011"/>
+            <text text-anchor="middle" x="198.4869" y="-413.4" font-family="Times,serif" font-size="14.00" fill="#0000ff"> v</text>
+        </g>
+        <!-- trie -->
+        <g id="node10" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="494.2369" cy="-373.2" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="494.2369" y="-369" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tree&#45;&gt;trie -->
+        <!-- tre -->
+        <g id="node9" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="304.2369" cy="-473.4156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="304.2369" y="-477.6156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="304.2369" y="-460.8156" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge10" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#000000" d="M306.5939,-443.8897C307.6704,-430.4046 308.939,-414.5123 310.0005,-401.2153"/>
+            <polygon fill="#000000" stroke="#000000" points="313.4928,-401.4502 310.7997,-391.2033 306.515,-400.8931 313.4928,-401.4502"/>
+            <text text-anchor="middle" x="314.0942" y="-413.4" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tri -->
+        <g id="node11" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="446.2369" cy="-473.4156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="446.2369" y="-477.6156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="446.2369" y="-460.8156" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge12" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#000000" d="M459.4802,-445.766C466.2441,-431.6441 474.4452,-414.5215 481.1822,-400.456"/>
+            <polygon fill="#000000" stroke="#000000" points="484.4134,-401.812 485.5766,-391.2812 478.1002,-398.7881 484.4134,-401.812"/>
+            <text text-anchor="middle" x="480.0942" y="-413.4" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr -->
+        <g id="node12" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="279.2369" cy="-585.0469" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="279.2369" y="-589.2469" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="279.2369" y="-572.4469" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge4" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#000000" d="M252.4549,-564.0846C243.9686,-556.4495 235.0956,-547.3003 228.5223,-537.6313 223.1796,-529.7724 218.7204,-520.6194 215.11,-511.7284"/>
+            <polygon fill="#000000" stroke="#000000" points="218.3382,-510.37 211.5239,-502.2573 211.7918,-512.8487 218.3382,-510.37"/>
+            <text text-anchor="middle" x="234.0942" y="-525.0313" font-family="Times,serif" font-size="14.00" fill="#000000"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge18" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#add8e6" d="M262.877,-558.5399C255.5232,-546.8584 246.6268,-533.0409 238.2369,-520.8313 235.1223,-516.2987 231.7727,-511.5792 228.424,-506.9511"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="222.4485,-498.7793 231.9836,-504.1952 225.3998,-502.8154 228.3512,-506.8514 228.3512,-506.8514 228.3512,-506.8514 225.3998,-502.8154 224.7187,-509.5077 222.4485,-498.7793 222.4485,-498.7793"/>
+            <text text-anchor="middle" x="254.0942" y="-525.0313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge9" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#000000" d="M285.7401,-556.0085C288.6971,-542.8046 292.2409,-526.9807 295.4318,-512.7327"/>
+            <polygon fill="#000000" stroke="#000000" points="298.9093,-513.2196 297.6794,-502.6964 292.0785,-511.6898 298.9093,-513.2196"/>
+            <text text-anchor="middle" x="298.0942" y="-525.0313" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge11" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#000000" d="M307.9452,-565.8568C335.7854,-547.247 378.1751,-518.9116 408.6826,-498.5188"/>
+            <polygon fill="#000000" stroke="#000000" points="410.9757,-501.196 417.3443,-492.7289 407.0856,-495.3764 410.9757,-501.196"/>
+            <text text-anchor="middle" x="376.9322" y="-525.0313" font-family="Times,serif" font-size="14.00" fill="#000000"> i</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge3" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#000000" d="M271.4066,-667.6591C269.2309,-661.9119 267.1669,-655.422 266.0763,-649.2626 264.5952,-640.8975 265.0969,-631.9507 266.5509,-623.4728"/>
+            <polygon fill="#000000" stroke="#000000" points="270.0289,-623.9493 268.7277,-613.4347 263.1879,-622.4658 270.0289,-623.9493"/>
+            <text text-anchor="middle" x="270.3172" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge17" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#add8e6" d="M279.2369,-666.8815C279.2369,-655.1502 279.2369,-639.4774 279.2369,-624.9885"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="279.2369,-614.7209 283.737,-624.7208 279.2369,-619.7209 279.237,-624.7209 279.237,-624.7209 279.237,-624.7209 279.2369,-619.7209 274.737,-624.7209 279.2369,-614.7209 279.2369,-614.7209"/>
+            <text text-anchor="middle" x="283.3172" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- trave -->
+        <g id="node15" class="node">
+            <title>trave</title>
+            <ellipse fill="none" stroke="#add8e6" cx="188.2369" cy="-284.4" rx="37.7006" ry="18"/>
+            <text text-anchor="middle" x="188.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#0000ff">NONE</text>
+        </g>
+        <!-- trav&#45;&gt;trave -->
+        <g id="edge20" class="edge">
+            <title>trav&#45;&gt;trave</title>
+            <path fill="none" stroke="#add8e6" d="M184.2729,-354.8006C184.9613,-342.575 185.8783,-326.2887 186.6569,-312.4599"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="187.2193,-302.472 191.1499,-312.7092 186.9382,-307.4641 186.6571,-312.4562 186.6571,-312.4562 186.6571,-312.4562 186.9382,-307.4641 182.1642,-312.2032 187.2193,-302.472 187.2193,-302.472"/>
+            <text text-anchor="middle" x="191.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- traver -->
+        <g id="node16" class="node">
+            <title>traver</title>
+            <ellipse fill="none" stroke="#add8e6" cx="193.2369" cy="-195.6" rx="37.7006" ry="18"/>
+            <text text-anchor="middle" x="193.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#0000ff">NONE</text>
+        </g>
+        <!-- trave&#45;&gt;traver -->
+        <g id="edge21" class="edge">
+            <title>trave&#45;&gt;traver</title>
+            <path fill="none" stroke="#add8e6" d="M189.2729,-266.0006C189.9613,-253.775 190.8783,-237.4887 191.6569,-223.6599"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="192.2193,-213.672 196.1499,-223.9092 191.9382,-218.6641 191.6571,-223.6562 191.6571,-223.6562 191.6571,-223.6562 191.9382,-218.6641 187.1642,-223.4032 192.2193,-213.672 192.2193,-213.672"/>
+            <text text-anchor="middle" x="195.3172" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- travers -->
+        <g id="node17" class="node">
+            <title>travers</title>
+            <ellipse fill="none" stroke="#add8e6" cx="202.2369" cy="-106.8" rx="37.7006" ry="18"/>
+            <text text-anchor="middle" x="202.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#0000ff">NONE</text>
+        </g>
+        <!-- traver&#45;&gt;travers -->
+        <g id="edge22" class="edge">
+            <title>traver&#45;&gt;travers</title>
+            <path fill="none" stroke="#add8e6" d="M195.1017,-177.2006C196.3408,-164.975 197.9914,-148.6887 199.393,-134.8599"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="200.4053,-124.872 203.8739,-135.2748 199.901,-129.8465 199.3968,-134.821 199.3968,-134.821 199.3968,-134.821 199.901,-129.8465 194.9198,-134.3673 200.4053,-124.872 200.4053,-124.872"/>
+            <text text-anchor="middle" x="203.7092" y="-147" font-family="Times,serif" font-size="14.00" fill="#0000ff"> s</text>
+        </g>
+        <!-- traverse -->
+        <g id="node18" class="node">
+            <title>traverse</title>
+            <ellipse fill="none" stroke="#add8e6" cx="202.2369" cy="-18" rx="37.7006" ry="18"/>
+            <text text-anchor="middle" x="202.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#0000ff">NONE</text>
+        </g>
+        <!-- travers&#45;&gt;traverse -->
+        <g id="edge23" class="edge">
+            <title>travers&#45;&gt;traverse</title>
+            <path fill="none" stroke="#add8e6" d="M202.2369,-88.4006C202.2369,-76.2949 202.2369,-60.2076 202.2369,-46.4674"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="202.2369,-36.072 206.737,-46.072 202.2369,-41.072 202.237,-46.072 202.237,-46.072 202.237,-46.072 202.2369,-41.072 197.737,-46.0721 202.2369,-36.072 202.2369,-36.072"/>
+            <text text-anchor="middle" x="207.0942" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m2.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m2.svg
new file mode 100644
index 0000000000..ba33dd1f22
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m2.svg
@@ -0,0 +1,430 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+# embedded DOT (plantuml) works in IDEA preview
+# but not on GitHub
+```plantuml
+digraph G {
+    { rank=same root -> start [style=invis] }
+    start [label="start/end"]
+
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"]
+
+    tree [label = "contentArray[1]"]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    root -> t [label = " t"]
+    t -> tr [label = " r"]
+    tr -> tra [label = " a"]
+    tra -> trac [label = " c"]
+    trac -> tract [label = " t"]
+    tract -> tracto [label = " o"]
+    tracto -> tractor [label = " r"]
+
+    tr -> tre [label = " e"]
+    tre -> tree [label = " e"]
+
+    tr -> tri [label = " i"]
+    tri -> trie [label = " e"]
+
+    // {rank=same tra -> tre -> tri [style=invis]}
+    {rank=same trac -> tree -> trie [style=invis]}
+
+    subgraph path {
+        edge [color = "lightblue"; fontcolor="blue"; arrowhead="vee"]
+        node [color = "blue"; fontcolor="blue"]
+
+        start -> root
+
+        root -> t [label = " t"]
+        t -> tr [label = " r"]
+        tr -> tra [label = " a"]
+        tra -> trav [label = " v"]
+        trav -> trave [label = " e"]
+        trave -> traver [label = " r"]
+        traver -> travers [label = " s"]
+        travers -> traverse [label = " e"]
+
+        trav [label = "0x0B8"]
+        trave [label = "0x0B9"]
+        traver [label = "0x0BA"]
+        travers [label = "0x0BB"]
+        traverse [label = "contentArray[3]"]
+    }
+
+    {
+        edge [color = "blue"]
+        trav -> trave [label = " e"]
+        trave -> traver [label = " r"]
+        traver -> travers [label = " s"]
+        travers -> traverse [label = " e"]
+    }
+
+    subgraph back {
+        edge [color = "pink"; fontcolor="red"; arrowhead="vee"; constrain="false"]
+
+        traverse -> travers [label = " ~3"]
+        travers -> traver [label = "0x0BB"]
+        traver -> trave [label = "0x0BA"]
+        trave -> trav [label = "0x0B9"]
+        trav -> tra [label = "0x0B8"]
+    }
+}
+```
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="574pt" height="823pt"
+     viewBox="0.00 0.00 574.47 822.89" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 818.8939)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-818.8939 570.4738,-818.8939 570.4738,4 -4,4"/>
+        <!-- root -->
+        <g id="node1" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="289.2369" cy="-785.4782" rx="33.1337" ry="29.3315"/>
+            <text text-anchor="middle" x="289.2369" y="-789.6782" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="289.2369" y="-772.8782" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- start -->
+        <g id="node2" class="node">
+            <title>start</title>
+            <ellipse fill="none" stroke="#000000" cx="402.2369" cy="-785.4782" rx="44.0775" ry="18"/>
+            <text text-anchor="middle" x="402.2369" y="-781.2782" font-family="Times,serif" font-size="14.00" fill="#000000">start/end</text>
+        </g>
+        <!-- root&#45;&gt;start -->
+        <!-- t -->
+        <g id="node13" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="289.2369" cy="-685.2626" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="289.2369" y="-681.0626" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge2" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#000000" d="M279.3368,-757.0824C276.6104,-745.9368 274.8794,-733.0211 276.8463,-721.2626 277.3268,-718.39 278.009,-715.4398 278.8091,-712.5238"/>
+            <polygon fill="#000000" stroke="#000000" points="282.1732,-713.494 281.8627,-702.9037 275.5012,-711.3762 282.1732,-713.494"/>
+            <text text-anchor="middle" x="280.9322" y="-725.4626" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge16" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#add8e6" d="M289.2369,-755.9522C289.2369,-742.4671 289.2369,-726.5749 289.2369,-713.2779"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="289.2369,-703.2659 293.737,-713.2659 289.2369,-708.2659 289.237,-713.2659 289.237,-713.2659 289.237,-713.2659 289.2369,-708.2659 284.737,-713.2659 289.2369,-703.2659 289.2369,-703.2659"/>
+            <text text-anchor="middle" x="292.9322" y="-725.4626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> t</text>
+        </g>
+        <!-- start&#45;&gt;root -->
+        <g id="edge15" class="edge">
+            <title>start&#45;&gt;root</title>
+            <path fill="none" stroke="#add8e6" d="M377.6112,-800.5185C360.7348,-807.9537 343.8585,-809.4839 326.9821,-805.109"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="317.0455,-801.8508 327.9498,-800.6907 321.7966,-803.4087 326.5477,-804.9667 326.5477,-804.9667 326.5477,-804.9667 321.7966,-803.4087 325.1456,-809.2427 317.0455,-801.8508 317.0455,-801.8508"/>
+        </g>
+        <!-- tractor -->
+        <g id="node3" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tracto -->
+        <g id="node4" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="91.2369" cy="-195.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="91.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge8" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#000000" d="M87.5943,-177.63C85.1221,-165.4338 81.802,-149.0543 78.9772,-135.1187"/>
+            <polygon fill="#000000" stroke="#000000" points="82.3528,-134.1534 76.9358,-125.0481 75.4923,-135.5441 82.3528,-134.1534"/>
+            <text text-anchor="middle" x="88.3172" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- tract -->
+        <g id="node5" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="92.2369" cy="-284.4" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="92.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge7" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#000000" d="M92.0297,-266.0006C91.8934,-253.8949 91.7122,-237.8076 91.5575,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="95.0529,-223.6319 91.4404,-213.672 88.0533,-223.7108 95.0529,-223.6319"/>
+            <text text-anchor="middle" x="97.4869" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> o</text>
+        </g>
+        <!-- trac -->
+        <g id="node6" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="93.2369" cy="-373.2" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="93.2369" y="-369" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge6" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#000000" d="M93.0297,-354.8006C92.8934,-342.6949 92.7122,-326.6076 92.5575,-312.8674"/>
+            <polygon fill="#000000" stroke="#000000" points="96.0529,-312.4319 92.4404,-302.472 89.0533,-312.5108 96.0529,-312.4319"/>
+            <text text-anchor="middle" x="96.9322" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- tree -->
+        <g id="node8" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#000000" cx="311.2369" cy="-373.2" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="311.2369" y="-369" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- trac&#45;&gt;tree -->
+        <!-- tra -->
+        <g id="node7" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#000000" cx="219.2369" cy="-473.4156" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="219.2369" y="-477.6156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="219.2369" y="-460.8156" font-family="Times,serif" font-size="14.00" fill="#000000">0x018</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge5" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#000000" d="M191.1461,-454.472C179.0394,-446.0482 164.8584,-435.8286 152.5223,-426 140.7539,-416.6237 128.241,-405.6422 117.7589,-396.1298"/>
+            <polygon fill="#000000" stroke="#000000" points="119.9234,-393.366 110.185,-389.1928 115.1954,-398.5281 119.9234,-393.366"/>
+            <text text-anchor="middle" x="158.0942" y="-413.4" font-family="Times,serif" font-size="14.00" fill="#000000"> c</text>
+        </g>
+        <!-- trav -->
+        <g id="node14" class="node">
+            <title>trav</title>
+            <ellipse fill="none" stroke="#0000ff" cx="183.2369" cy="-373.2" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="183.2369" y="-369" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0B8</text>
+        </g>
+        <!-- tra&#45;&gt;trav -->
+        <g id="edge19" class="edge">
+            <title>tra&#45;&gt;trav</title>
+            <path fill="none" stroke="#add8e6" d="M217.5652,-443.8618C216.1848,-432.6862 213.6979,-420.0532 209.2369,-409.2 207.6981,-405.4564 205.697,-401.7342 203.4879,-398.182"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="197.713,-389.7873 207.0881,-395.4757 200.5468,-393.9067 203.3807,-398.0261 203.3807,-398.0261 203.3807,-398.0261 200.5468,-393.9067 199.6732,-400.5766 197.713,-389.7873 197.713,-389.7873"/>
+            <text text-anchor="middle" x="219.4869" y="-413.4" font-family="Times,serif" font-size="14.00" fill="#0000ff"> v</text>
+        </g>
+        <!-- trie -->
+        <g id="node10" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="493.2369" cy="-373.2" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="493.2369" y="-369" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tree&#45;&gt;trie -->
+        <!-- tre -->
+        <g id="node9" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="311.2369" cy="-473.4156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="311.2369" y="-477.6156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="311.2369" y="-460.8156" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge10" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#000000" d="M311.2369,-443.8897C311.2369,-430.4046 311.2369,-414.5123 311.2369,-401.2153"/>
+            <polygon fill="#000000" stroke="#000000" points="314.737,-401.2033 311.2369,-391.2033 307.737,-401.2034 314.737,-401.2033"/>
+            <text text-anchor="middle" x="316.0942" y="-413.4" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tri -->
+        <g id="node11" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="448.2369" cy="-473.4156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="448.2369" y="-477.6156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="448.2369" y="-460.8156" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge12" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#000000" d="M460.7719,-445.5C467.1109,-431.383 474.7723,-414.3209 481.0603,-400.3175"/>
+            <polygon fill="#000000" stroke="#000000" points="484.2575,-401.7415 485.1609,-391.1852 477.8717,-398.874 484.2575,-401.7415"/>
+            <text text-anchor="middle" x="480.0942" y="-413.4" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr -->
+        <g id="node12" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="289.2369" cy="-585.0469" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="289.2369" y="-589.2469" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="289.2369" y="-572.4469" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge4" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#000000" d="M262.3516,-564.0201C254.086,-556.4444 245.5803,-547.3475 239.5223,-537.6313 234.7133,-529.9184 230.9544,-520.9679 228.0512,-512.2428"/>
+            <polygon fill="#000000" stroke="#000000" points="231.37,-511.1256 225.1267,-502.5657 224.6692,-513.1506 231.37,-511.1256"/>
+            <text text-anchor="middle" x="245.0942" y="-525.0313" font-family="Times,serif" font-size="14.00" fill="#000000"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge18" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#add8e6" d="M272.7366,-558.4643C265.5384,-546.8917 256.9887,-533.179 249.2369,-520.8313 246.7152,-516.8145 244.0675,-512.6105 241.4331,-508.4359"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="236.0886,-499.9773 245.2344,-506.0275 238.7594,-504.2043 241.4302,-508.4312 241.4302,-508.4312 241.4302,-508.4312 238.7594,-504.2043 237.6259,-510.8349 236.0886,-499.9773 236.0886,-499.9773"/>
+            <text text-anchor="middle" x="264.0942" y="-525.0313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge9" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#000000" d="M295.0171,-555.7171C297.6192,-542.5138 300.7286,-526.7364 303.5256,-512.5441"/>
+            <polygon fill="#000000" stroke="#000000" points="306.9955,-513.0376 305.4953,-502.5496 300.1276,-511.6841 306.9955,-513.0376"/>
+            <text text-anchor="middle" x="307.0942" y="-525.0313" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge11" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#000000" d="M317.2969,-565.3465C343.5549,-546.9111 382.9334,-519.2642 411.6478,-499.1043"/>
+            <polygon fill="#000000" stroke="#000000" points="414.0173,-501.7172 420.1904,-493.1066 409.995,-495.9882 414.0173,-501.7172"/>
+            <text text-anchor="middle" x="381.9322" y="-525.0313" font-family="Times,serif" font-size="14.00" fill="#000000"> i</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge3" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#000000" d="M281.4066,-667.6591C279.2309,-661.9119 277.1669,-655.422 276.0763,-649.2626 274.5952,-640.8975 275.0969,-631.9507 276.5509,-623.4728"/>
+            <polygon fill="#000000" stroke="#000000" points="280.0289,-623.9493 278.7277,-613.4347 273.1879,-622.4658 280.0289,-623.9493"/>
+            <text text-anchor="middle" x="280.3172" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge17" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#add8e6" d="M289.2369,-666.8815C289.2369,-655.1502 289.2369,-639.4774 289.2369,-624.9885"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="289.2369,-614.7209 293.737,-624.7208 289.2369,-619.7209 289.237,-624.7209 289.237,-624.7209 289.237,-624.7209 289.2369,-619.7209 284.737,-624.7209 289.2369,-614.7209 289.2369,-614.7209"/>
+            <text text-anchor="middle" x="293.3172" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- trav&#45;&gt;tra -->
+        <g id="edge32" class="edge">
+            <title>trav&#45;&gt;tra</title>
+            <path fill="none" stroke="#ffc0cb" d="M172.1325,-390.7196C166.9576,-401.1938 162.925,-414.6319 167.9003,-426 171.4315,-434.0686 177.0881,-441.3739 183.3808,-447.7"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="190.9707,-454.6637 180.5599,-451.219 187.2864,-451.2834 183.6021,-447.9031 183.6021,-447.9031 183.6021,-447.9031 187.2864,-451.2834 186.6444,-444.5873 190.9707,-454.6637 190.9707,-454.6637"/>
+            <text text-anchor="middle" x="186.9052" y="-413.4" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0B8</text>
+        </g>
+        <!-- trave -->
+        <g id="node15" class="node">
+            <title>trave</title>
+            <ellipse fill="none" stroke="#0000ff" cx="189.2369" cy="-284.4" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="189.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0B9</text>
+        </g>
+        <!-- trav&#45;&gt;trave -->
+        <g id="edge20" class="edge">
+            <title>trav&#45;&gt;trave</title>
+            <path fill="none" stroke="#add8e6" d="M206.004,-359.0012C213.2194,-353.2102 220.3301,-345.8294 224.2369,-337.2 228.9378,-326.8164 224.0959,-316.1864 216.6675,-307.2114"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="209.6485,-299.798 219.7915,-303.9657 213.0861,-303.4288 216.5238,-307.0596 216.5238,-307.0596 216.5238,-307.0596 213.0861,-303.4288 213.2561,-310.1535 209.6485,-299.798 209.6485,-299.798"/>
+            <text text-anchor="middle" x="231.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- trav&#45;&gt;trave -->
+        <g id="edge24" class="edge">
+            <title>trav&#45;&gt;trave</title>
+            <path fill="none" stroke="#0000ff" d="M165.9239,-357.0271C157.1626,-346.8113 149.3672,-333.1725 154.5223,-320.4 156.3746,-315.8107 159.123,-311.4803 162.2767,-307.5211"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="165.0223,-309.7042 169.162,-299.9515 159.844,-304.994 165.0223,-309.7042"/>
+            <text text-anchor="middle" x="160.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- trave&#45;&gt;trav -->
+        <g id="edge31" class="edge">
+            <title>trave&#45;&gt;trav</title>
+            <path fill="none" stroke="#ffc0cb" d="M185.3926,-302.6325C184.3695,-308.2985 183.4083,-314.5837 182.9003,-320.4 182.2094,-328.3107 182.0246,-336.9134 182.0825,-344.8339"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="182.2872,-355.0829 177.5883,-345.1748 182.1873,-350.0839 182.0874,-345.0849 182.0874,-345.0849 182.0874,-345.0849 182.1873,-350.0839 186.5865,-344.995 182.2872,-355.0829 182.2872,-355.0829"/>
+            <text text-anchor="middle" x="201.9052" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0B9</text>
+        </g>
+        <!-- traver -->
+        <g id="node16" class="node">
+            <title>traver</title>
+            <ellipse fill="none" stroke="#0000ff" cx="194.2369" cy="-195.6" rx="38.8671" ry="18"/>
+            <text text-anchor="middle" x="194.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0BA</text>
+        </g>
+        <!-- trave&#45;&gt;traver -->
+        <g id="edge21" class="edge">
+            <title>trave&#45;&gt;traver</title>
+            <path fill="none" stroke="#add8e6" d="M213.0898,-270.4154C220.6549,-264.6446 228.1173,-257.2217 232.2369,-248.4 237.2769,-237.6073 231.7142,-226.7456 223.4297,-217.7036"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="216.0035,-210.5778 226.3347,-214.2545 219.6113,-214.0396 223.219,-217.5015 223.219,-217.5015 223.219,-217.5015 219.6113,-214.0396 220.1034,-220.7485 216.0035,-210.5778 216.0035,-210.5778"/>
+            <text text-anchor="middle" x="238.3172" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- trave&#45;&gt;traver -->
+        <g id="edge25" class="edge">
+            <title>trave&#45;&gt;traver</title>
+            <path fill="none" stroke="#0000ff" d="M172.8782,-268.1851C164.5976,-257.9534 157.2248,-244.3128 162.0763,-231.6 163.7001,-227.3449 166.0838,-223.2741 168.8315,-219.5011"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="171.5668,-221.6859 175.2747,-211.761 166.1869,-217.2074 171.5668,-221.6859"/>
+            <text text-anchor="middle" x="166.3172" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- traver&#45;&gt;trave -->
+        <g id="edge30" class="edge">
+            <title>traver&#45;&gt;trave</title>
+            <path fill="none" stroke="#ffc0cb" d="M190.328,-213.8298C189.2877,-219.4957 188.3103,-225.7815 187.7937,-231.6 187.0912,-239.5125 187.0294,-248.1157 187.2551,-256.0363"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="187.7079,-266.2849 182.7708,-256.4934 187.4871,-261.2898 187.2664,-256.2947 187.2664,-256.2947 187.2664,-256.2947 187.4871,-261.2898 191.762,-256.096 187.7079,-266.2849 187.7079,-266.2849"/>
+            <text text-anchor="middle" x="208.4585" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0BA</text>
+        </g>
+        <!-- travers -->
+        <g id="node17" class="node">
+            <title>travers</title>
+            <ellipse fill="none" stroke="#0000ff" cx="203.2369" cy="-106.8" rx="38.305" ry="18"/>
+            <text text-anchor="middle" x="203.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0BB</text>
+        </g>
+        <!-- traver&#45;&gt;travers -->
+        <g id="edge22" class="edge">
+            <title>traver&#45;&gt;travers</title>
+            <path fill="none" stroke="#add8e6" d="M219.6485,-181.4864C227.3996,-175.7839 234.9896,-168.4371 239.2369,-159.6 244.2682,-149.1317 239.1644,-138.3954 231.3732,-129.361"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="224.3601,-122.2156 234.5764,-126.2003 227.8624,-125.784 231.3648,-129.3524 231.3648,-129.3524 231.3648,-129.3524 227.8624,-125.784 228.1532,-132.5046 224.3601,-122.2156 224.3601,-122.2156"/>
+            <text text-anchor="middle" x="245.7092" y="-147" font-family="Times,serif" font-size="14.00" fill="#0000ff"> s</text>
+        </g>
+        <!-- traver&#45;&gt;travers -->
+        <g id="edge26" class="edge">
+            <title>traver&#45;&gt;travers</title>
+            <path fill="none" stroke="#0000ff" d="M177.7403,-179.2722C169.4246,-168.9981 162.0986,-155.3524 167.2923,-142.8 169.2677,-138.0258 172.2173,-133.5536 175.5926,-129.4948"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="178.1364,-131.8988 182.4642,-122.2282 173.0503,-127.0892 178.1364,-131.8988"/>
+            <text text-anchor="middle" x="172.7092" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> s</text>
+        </g>
+        <!-- travers&#45;&gt;traver -->
+        <g id="edge29" class="edge">
+            <title>travers&#45;&gt;traver</title>
+            <path fill="none" stroke="#ffc0cb" d="M198.6734,-125.0051C197.4382,-130.6699 196.2533,-136.9614 195.5637,-142.8 194.632,-150.6883 194.1789,-159.2838 193.9873,-167.2046"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="193.8675,-177.4577 189.4847,-167.4058 193.9259,-172.4581 193.9844,-167.4584 193.9844,-167.4584 193.9844,-167.4584 193.9259,-172.4581 198.4841,-167.511 193.8675,-177.4577 193.8675,-177.4577"/>
+            <text text-anchor="middle" x="216.0735" y="-147" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0BB</text>
+        </g>
+        <!-- traverse -->
+        <g id="node18" class="node">
+            <title>traverse</title>
+            <ellipse fill="none" stroke="#0000ff" cx="203.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="203.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#0000ff">contentArray[3]</text>
+        </g>
+        <!-- travers&#45;&gt;traverse -->
+        <g id="edge23" class="edge">
+            <title>travers&#45;&gt;traverse</title>
+            <path fill="none" stroke="#add8e6" d="M184.8676,-90.9569C179.5135,-85.2132 174.3474,-78.2805 171.5223,-70.8 168.8843,-63.8149 168.8843,-60.9851 171.5223,-54 172.8668,-50.4399 174.7415,-47.004 176.9144,-43.7612"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="183.203,-35.676 180.6155,-46.3323 180.1332,-39.6227 177.0634,-43.5695 177.0634,-43.5695 177.0634,-43.5695 180.1332,-39.6227 173.5114,-40.8067 183.203,-35.676 183.203,-35.676"/>
+            <text text-anchor="middle" x="177.0942" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- travers&#45;&gt;traverse -->
+        <g id="edge27" class="edge">
+            <title>travers&#45;&gt;traverse</title>
+            <path fill="none" stroke="#0000ff" d="M203.2369,-88.4006C203.2369,-76.2949 203.2369,-60.2076 203.2369,-46.4674"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="206.737,-46.072 203.2369,-36.072 199.737,-46.0721 206.737,-46.072"/>
+            <text text-anchor="middle" x="208.0942" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- traverse&#45;&gt;travers -->
+        <g id="edge28" class="edge">
+            <title>traverse&#45;&gt;travers</title>
+            <path fill="none" stroke="#ffc0cb" d="M209.9024,-36.0722C211.6782,-41.7317 213.3487,-48.0535 214.2369,-54 215.3399,-61.3847 215.3399,-63.4153 214.2369,-70.8 213.8344,-73.4945 213.2713,-76.2661 212.6106,-79.0181"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="209.9024,-88.7278 208.2546,-77.8864 211.2458,-83.9116 212.5891,-79.0954 212.5891,-79.0954 212.5891,-79.0954 211.2458,-83.9116 216.9237,-80.3044 209.9024,-88.7278 209.9024,-88.7278"/>
+            <text text-anchor="middle" x="224.2732" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#ff0000"> ~3</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m3.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m3.svg
new file mode 100644
index 0000000000..e71114f892
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m3.svg
@@ -0,0 +1,500 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+# embedded DOT (plantuml) works in IDEA preview
+# but not on GitHub
+```plantuml
+digraph G {
+    { rank=same root -> start [style=invis] }
+    start [label="start/end"]
+
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"; color = "lightgrey"; fontcolor = lightgray]
+
+    tree [label = "contentArray[1]"]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    root -> t [label = " t"]
+    t -> tr [label = " r"]
+    trac -> tract [label = " t"]
+    tract -> tracto [label = " o"]
+    tracto -> tractor [label = " r"]
+
+    tr -> tra [label = " a"; color = "lightgrey"; fontcolor = lightgray]
+    tra -> trac [label = " c"; color = "lightgrey"; fontcolor = lightgray]
+
+    tr -> tre [label = " e"]
+    tre -> tree [label = " e"]
+
+    tr -> tri [label = " i"]
+    tri -> trie [label = " e"]
+
+    subgraph path {
+        edge [color = "lightblue"; fontcolor="blue"; arrowhead="vee"]
+        node [color = "blue"; fontcolor="blue"]
+
+        start -> root
+
+        root -> t [label = " t"]
+        t -> tr [label = " r"]
+        tr -> tra [label = " a"]
+        tra -> trav [label = " v"]
+        trav -> trave [label = " e"]
+        trave -> traver [label = " r"]
+        traver -> travers [label = " s"]
+        travers -> traverse [label = " e"]
+
+        tra2 [label = "Sparse\n0x0DE"]
+        trav [label = "Chain\n0x0B8"]
+        trave [label = "0x0B9"]
+        traver [label = "0x0BA"]
+        travers [label = "0x0BB"]
+        traverse [label = "contentArray[3]"]
+    }
+
+    {rank=same tra -> tra2 -> tre -> tri [style=invis]}
+    {rank=same trac -> trav -> tree -> trie [style=invis]}
+
+    {
+        edge [color = "blue"]
+        tr -> tra2 [label = " a"]
+        tra2 -> trac [label = " c"]
+        tra2 -> trav [label = " v"]
+        trav -> trave [label = " e"]
+        trave -> traver [label = " r"]
+        traver -> travers [label = " s"]
+        travers -> traverse [label = " e"]
+    }
+
+    subgraph back {
+        edge [color = "pink"; fontcolor="red"; arrowhead="vee"; constrain="false"]
+
+        traverse -> travers [label = " ~3"]
+        travers -> traver [label = "0x0BB"]
+        traver -> trave [label = "0x0BA"]
+        trave -> trav [label = "0x0B9"]
+        trav -> tra2 [label = "0x0B8"]
+        tra2 -> tr [label = "0x0DE"]
+        tr -> t [label = "0x07E"]
+        t -> root [label = "0x09B"]
+        root -> start [label = "0x09A"]
+    }
+}
+```
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="611pt" height="846pt"
+     viewBox="0.00 0.00 611.47 845.73" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 841.7251)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-841.7251 607.4738,-841.7251 607.4738,4 -4,4"/>
+        <!-- root -->
+        <g id="node1" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="230.2369" cy="-808.3095" rx="33.1337" ry="29.3315"/>
+            <text text-anchor="middle" x="230.2369" y="-812.5095" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="230.2369" y="-795.7095" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- start -->
+        <g id="node2" class="node">
+            <title>start</title>
+            <ellipse fill="none" stroke="#000000" cx="363.2369" cy="-808.3095" rx="44.0775" ry="18"/>
+            <text text-anchor="middle" x="363.2369" y="-804.1095" font-family="Times,serif" font-size="14.00" fill="#000000">start/end</text>
+        </g>
+        <!-- root&#45;&gt;start -->
+        <!-- root&#45;&gt;start -->
+        <g id="edge43" class="edge">
+            <title>root&#45;&gt;start</title>
+            <path fill="none" stroke="#ffc0cb" d="M263.7984,-808.3095C277.3037,-808.3095 293.2339,-808.3095 308.3398,-808.3095"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="318.6261,-808.3095 308.6262,-812.8096 313.6261,-808.3095 308.6261,-808.3096 308.6261,-808.3096 308.6261,-808.3096 313.6261,-808.3095 308.6261,-803.8096 318.6261,-808.3095 318.6261,-808.3095"/>
+            <text text-anchor="middle" x="291.2509" y="-815.5095" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x09A</text>
+        </g>
+        <!-- t -->
+        <g id="node13" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="230.2369" cy="-708.0939" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="230.2369" y="-703.8939" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge2" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#000000" d="M210.9564,-784.2612C203.7393,-772.3485 198.2768,-757.6462 202.8463,-744.0939 204.1636,-740.1869 206.0926,-736.3684 208.3269,-732.7663"/>
+            <polygon fill="#000000" stroke="#000000" points="211.3844,-734.5068 214.3357,-724.3313 205.683,-730.4454 211.3844,-734.5068"/>
+            <text text-anchor="middle" x="206.9322" y="-748.2939" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge14" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#add8e6" d="M230.2369,-778.7835C230.2369,-765.2984 230.2369,-749.4062 230.2369,-736.1092"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="230.2369,-726.0972 234.737,-736.0971 230.2369,-731.0972 230.237,-736.0972 230.237,-736.0972 230.237,-736.0972 230.2369,-731.0972 225.737,-736.0972 230.2369,-726.0972 230.2369,-726.0972"/>
+            <text text-anchor="middle" x="233.9322" y="-748.2939" font-family="Times,serif" font-size="14.00" fill="#0000ff"> t</text>
+        </g>
+        <!-- start&#45;&gt;root -->
+        <g id="edge13" class="edge">
+            <title>start&#45;&gt;root</title>
+            <path fill="none" stroke="#add8e6" d="M319.4042,-805.6333C307.2938,-805.1818 294.1479,-804.9584 282.0184,-805.3095 279.3264,-805.3874 276.5568,-805.4881 273.7643,-805.6051"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="263.6504,-806.089 273.4239,-801.1161 268.6447,-805.85 273.639,-805.611 273.639,-805.611 273.639,-805.611 268.6447,-805.85 273.8541,-810.1058 263.6504,-806.089 263.6504,-806.089"/>
+        </g>
+        <!-- tractor -->
+        <g id="node3" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tracto -->
+        <g id="node4" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="74.2369" cy="-195.6" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="74.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge6" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#000000" d="M74.0297,-177.2006C73.8934,-165.0949 73.7122,-149.0076 73.5575,-135.2674"/>
+            <polygon fill="#000000" stroke="#000000" points="77.0529,-134.8319 73.4404,-124.872 70.0533,-134.9108 77.0529,-134.8319"/>
+            <text text-anchor="middle" x="78.3172" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- tract -->
+        <g id="node5" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="78.2369" cy="-284.4" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="78.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge5" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#000000" d="M77.4081,-266.0006C76.8628,-253.8949 76.1381,-237.8076 75.5192,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="78.9975,-223.5044 75.0509,-213.672 72.0046,-223.8194 78.9975,-223.5044"/>
+            <text text-anchor="middle" x="82.4869" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> o</text>
+        </g>
+        <!-- trac -->
+        <g id="node6" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="94.2369" cy="-384.6156" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="94.2369" y="-380.4156" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge4" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#000000" d="M91.3022,-366.2345C88.9115,-351.2603 85.4955,-329.8643 82.756,-312.7055"/>
+            <polygon fill="#000000" stroke="#000000" points="86.1858,-311.9877 81.1529,-302.6646 79.2734,-313.0913 86.1858,-311.9877"/>
+            <text text-anchor="middle" x="89.9322" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- trav -->
+        <g id="node14" class="node">
+            <title>trav</title>
+            <ellipse fill="none" stroke="#0000ff" cx="202.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="202.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#0000ff">Chain</text>
+            <text text-anchor="middle" x="202.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0B8</text>
+        </g>
+        <!-- trac&#45;&gt;trav -->
+        <!-- tra -->
+        <g id="node7" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#d3d3d3" cx="124.2369" cy="-496.2469" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="124.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#d3d3d3">Chain</text>
+            <text text-anchor="middle" x="124.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#d3d3d3">0x018</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge8" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#d3d3d3" d="M108.792,-469.6104C105.5657,-463.008 102.5364,-455.818 100.5223,-448.8313 97.2068,-437.3303 95.5641,-424.2429 94.7752,-412.8969"/>
+            <polygon fill="#d3d3d3" stroke="#d3d3d3" points="98.2581,-412.4701 94.2539,-402.6611 91.2672,-412.8262 98.2581,-412.4701"/>
+            <text text-anchor="middle" x="106.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#d3d3d3"> c</text>
+        </g>
+        <!-- tra&#45;&gt;trav -->
+        <g id="edge17" class="edge">
+            <title>tra&#45;&gt;trav</title>
+            <path fill="none" stroke="#add8e6" d="M124.8024,-466.7151C126.1832,-455.1695 129.2555,-442.2712 135.7369,-432.0313 142.2539,-421.7351 151.8894,-413.0083 161.8152,-405.9576"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="170.1591,-400.4315 164.3065,-409.7051 165.9904,-403.1924 161.8218,-405.9533 161.8218,-405.9533 161.8218,-405.9533 165.9904,-403.1924 159.337,-402.2015 170.1591,-400.4315 170.1591,-400.4315"/>
+            <text text-anchor="middle" x="141.4869" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> v</text>
+        </g>
+        <!-- tra2 -->
+        <g id="node19" class="node">
+            <title>tra2</title>
+            <ellipse fill="none" stroke="#0000ff" cx="235.2369" cy="-496.2469" rx="39.2145" ry="29.3315"/>
+            <text text-anchor="middle" x="235.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#0000ff">Sparse</text>
+            <text text-anchor="middle" x="235.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0DE</text>
+        </g>
+        <!-- tra&#45;&gt;tra2 -->
+        <!-- tree -->
+        <g id="node8" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#000000" cx="348.2369" cy="-384.6156" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="348.2369" y="-380.4156" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- trie -->
+        <g id="node10" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="530.2369" cy="-384.6156" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="530.2369" y="-380.4156" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tree&#45;&gt;trie -->
+        <!-- tre -->
+        <g id="node9" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="348.2369" cy="-496.2469" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="348.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="348.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge10" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#000000" d="M348.2369,-466.6249C348.2369,-449.9873 348.2369,-429.334 348.2369,-412.9163"/>
+            <polygon fill="#000000" stroke="#000000" points="351.737,-412.8239 348.2369,-402.8239 344.737,-412.8239 351.737,-412.8239"/>
+            <text text-anchor="middle" x="353.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tri -->
+        <g id="node11" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="459.2369" cy="-496.2469" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="459.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="459.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- tre&#45;&gt;tri -->
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge12" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#000000" d="M476.0629,-469.7919C487.3072,-452.1128 502.0457,-428.9399 513.3106,-411.2283"/>
+            <polygon fill="#000000" stroke="#000000" points="516.4345,-412.8384 518.848,-402.5221 510.5279,-409.0817 516.4345,-412.8384"/>
+            <text text-anchor="middle" x="503.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr -->
+        <g id="node12" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="230.2369" cy="-607.8782" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="230.2369" y="-612.0782" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="230.2369" y="-595.2782" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge7" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#d3d3d3" d="M195.7438,-595.1807C178.8708,-587.4308 159.461,-575.9889 146.5223,-560.4626 140.4195,-553.1393 135.9491,-544.0436 132.6887,-535.0233"/>
+            <polygon fill="#d3d3d3" stroke="#d3d3d3" points="135.9777,-533.8168 129.6057,-525.3523 129.3084,-535.943 135.9777,-533.8168"/>
+            <text text-anchor="middle" x="152.0942" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#d3d3d3"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge16" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#add8e6" d="M207.757,-584.2041C191.9107,-567.5159 170.5426,-545.0127 153.387,-526.9457"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="146.3209,-519.5041 156.4699,-523.6571 149.7638,-523.13 153.2067,-526.7558 153.2067,-526.7558 153.2067,-526.7558 149.7638,-523.13 149.9435,-529.8544 146.3209,-519.5041 146.3209,-519.5041"/>
+            <text text-anchor="middle" x="189.0942" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge9" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#000000" d="M254.4007,-585.0186C272.3773,-568.0122 297.076,-544.6465 316.6218,-526.1556"/>
+            <polygon fill="#000000" stroke="#000000" points="319.2463,-528.4909 324.1054,-519.076 314.4357,-523.4058 319.2463,-528.4909"/>
+            <text text-anchor="middle" x="302.0942" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge11" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#000000" d="M262.5613,-592.1209C303.3074,-572.2583 373.3978,-538.0911 417.9723,-516.3623"/>
+            <polygon fill="#000000" stroke="#000000" points="419.6217,-519.452 427.0769,-511.924 416.5544,-513.1598 419.6217,-519.452"/>
+            <text text-anchor="middle" x="361.9322" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> i</text>
+        </g>
+        <!-- tr&#45;&gt;t -->
+        <g id="edge41" class="edge">
+            <title>tr&#45;&gt;t</title>
+            <path fill="none" stroke="#ffc0cb" d="M228.1399,-637.287C227.605,-648.2159 227.2816,-660.7205 227.6843,-672.0939 227.7703,-674.5242 227.889,-677.0451 228.0283,-679.5733"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="228.6875,-689.7931 223.5531,-680.1035 228.3656,-684.8034 228.0438,-679.8138 228.0438,-679.8138 228.0438,-679.8138 228.3656,-684.8034 232.5344,-679.5241 228.6875,-689.7931 228.6875,-689.7931"/>
+            <text text-anchor="middle" x="246.5132" y="-659.4939" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x07E</text>
+        </g>
+        <!-- tr&#45;&gt;tra2 -->
+        <g id="edge28" class="edge">
+            <title>tr&#45;&gt;tra2</title>
+            <path fill="none" stroke="#0000ff" d="M218.5572,-579.8196C215.264,-568.6049 213.1491,-555.5518 215.5223,-543.6626 216.1971,-540.2816 217.118,-536.8402 218.1951,-533.4336"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="221.4953,-534.5993 221.57,-524.0047 214.9047,-532.2403 221.4953,-534.5993"/>
+            <text text-anchor="middle" x="221.0942" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> a</text>
+        </g>
+        <!-- t&#45;&gt;root -->
+        <g id="edge42" class="edge">
+            <title>t&#45;&gt;root</title>
+            <path fill="none" stroke="#ffc0cb" d="M235.0886,-726.2786C236.3802,-731.9426 237.5942,-738.2388 238.2369,-744.0939 239.1247,-752.1817 238.87,-760.8327 238.048,-769.081"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="236.7469,-779.2407 233.5537,-768.7501 237.3821,-774.2812 238.0173,-769.3217 238.0173,-769.3217 238.0173,-769.3217 237.3821,-774.2812 242.4808,-769.8934 236.7469,-779.2407 236.7469,-779.2407"/>
+            <text text-anchor="middle" x="257.9052" y="-748.2939" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x09B</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge3" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#000000" d="M251.7729,-693.2308C258.3244,-687.4682 264.7179,-680.2751 268.2369,-672.0939 271.1872,-665.2348 270.7115,-662.3385 268.2369,-655.2939 266.2659,-649.6828 263.3202,-644.2436 259.9247,-639.1727"/>
+            <polygon fill="#000000" stroke="#000000" points="262.6469,-636.9677 253.8782,-631.0211 257.0247,-641.138 262.6469,-636.9677"/>
+            <text text-anchor="middle" x="275.3172" y="-659.4939" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge15" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#add8e6" d="M213.324,-692.0138C208.3995,-686.2543 203.6543,-679.3722 201.0763,-672.0939 197.4827,-661.9481 199.7954,-651.1983 204.382,-641.4556"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="209.3576,-632.4758 208.4471,-643.4038 206.9342,-636.8493 204.5109,-641.2228 204.5109,-641.2228 204.5109,-641.2228 206.9342,-636.8493 200.5747,-639.0418 209.3576,-632.4758 209.3576,-632.4758"/>
+            <text text-anchor="middle" x="205.3172" y="-659.4939" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- trav&#45;&gt;tree -->
+        <!-- trave -->
+        <g id="node15" class="node">
+            <title>trave</title>
+            <ellipse fill="none" stroke="#0000ff" cx="202.2369" cy="-284.4" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="202.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0B9</text>
+        </g>
+        <!-- trav&#45;&gt;trave -->
+        <g id="edge18" class="edge">
+            <title>trav&#45;&gt;trave</title>
+            <path fill="none" stroke="#add8e6" d="M181.4692,-359.4476C176.9833,-352.6281 172.8815,-344.9707 170.5223,-337.2 168.3531,-330.0554 167.8843,-327.3851 170.5223,-320.4 172.1555,-316.0753 174.5712,-311.9338 177.3533,-308.0992"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="183.8676,-300.2431 180.9485,-310.8134 180.6761,-304.0921 177.4845,-307.941 177.4845,-307.941 177.4845,-307.941 180.6761,-304.0921 174.0205,-305.0686 183.8676,-300.2431 183.8676,-300.2431"/>
+            <text text-anchor="middle" x="176.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- trav&#45;&gt;trave -->
+        <g id="edge31" class="edge">
+            <title>trav&#45;&gt;trave</title>
+            <path fill="none" stroke="#0000ff" d="M202.2369,-355.0897C202.2369,-341.6046 202.2369,-325.7123 202.2369,-312.4153"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="205.737,-312.4033 202.2369,-302.4033 198.737,-312.4034 205.737,-312.4033"/>
+            <text text-anchor="middle" x="207.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- trav&#45;&gt;tra2 -->
+        <g id="edge39" class="edge">
+            <title>trav&#45;&gt;tra2</title>
+            <path fill="none" stroke="#ffc0cb" d="M210.8211,-413.654C214.805,-427.1305 219.5957,-443.3363 223.8744,-457.8102"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="226.7386,-467.4991 219.5882,-459.1851 225.3211,-462.7042 223.9036,-457.9093 223.9036,-457.9093 223.9036,-457.9093 225.3211,-462.7042 228.219,-456.6336 226.7386,-467.4991 226.7386,-467.4991"/>
+            <text text-anchor="middle" x="239.9052" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0B8</text>
+        </g>
+        <!-- trave&#45;&gt;trav -->
+        <g id="edge38" class="edge">
+            <title>trave&#45;&gt;trav</title>
+            <path fill="none" stroke="#ffc0cb" d="M208.9024,-302.4722C210.6782,-308.1317 212.3487,-314.4535 213.2369,-320.4 214.4724,-328.6713 214.0766,-337.5213 212.8888,-345.926"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="211.1036,-355.888 208.4382,-345.2509 211.9856,-350.9664 212.8676,-346.0448 212.8676,-346.0448 212.8676,-346.0448 211.9856,-350.9664 217.297,-346.8386 211.1036,-355.888 211.1036,-355.888"/>
+            <text text-anchor="middle" x="232.9052" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0B9</text>
+        </g>
+        <!-- traver -->
+        <g id="node16" class="node">
+            <title>traver</title>
+            <ellipse fill="none" stroke="#0000ff" cx="202.2369" cy="-195.6" rx="38.8671" ry="18"/>
+            <text text-anchor="middle" x="202.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0BA</text>
+        </g>
+        <!-- trave&#45;&gt;traver -->
+        <g id="edge19" class="edge">
+            <title>trave&#45;&gt;traver</title>
+            <path fill="none" stroke="#add8e6" d="M185.324,-268.32C180.3995,-262.5605 175.6543,-255.6784 173.0763,-248.4 170.5834,-241.3618 170.5834,-238.6382 173.0763,-231.6 174.4962,-227.5912 176.5735,-223.7026 178.9762,-220.0556"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="185.0164,-212.0415 182.5911,-222.7358 182.007,-216.0344 178.9975,-220.0273 178.9975,-220.0273 178.9975,-220.0273 182.007,-216.0344 175.4039,-217.3188 185.0164,-212.0415 185.0164,-212.0415"/>
+            <text text-anchor="middle" x="177.3172" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- trave&#45;&gt;traver -->
+        <g id="edge32" class="edge">
+            <title>trave&#45;&gt;traver</title>
+            <path fill="none" stroke="#0000ff" d="M202.2369,-266.0006C202.2369,-253.8949 202.2369,-237.8076 202.2369,-224.0674"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="205.737,-223.672 202.2369,-213.672 198.737,-223.6721 205.737,-223.672"/>
+            <text text-anchor="middle" x="206.3172" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- traver&#45;&gt;trave -->
+        <g id="edge37" class="edge">
+            <title>traver&#45;&gt;trave</title>
+            <path fill="none" stroke="#ffc0cb" d="M207.6937,-213.7509C209.1467,-219.4136 210.5127,-225.7174 211.2369,-231.6 212.1492,-239.0107 212.1492,-240.9893 211.2369,-248.4 210.9201,-250.9736 210.4804,-253.6279 209.9643,-256.2738"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="207.6937,-266.2491 205.5255,-255.4997 208.8035,-261.3738 209.9132,-256.4985 209.9132,-256.4985 209.9132,-256.4985 208.8035,-261.3738 214.301,-257.4973 207.6937,-266.2491 207.6937,-266.2491"/>
+            <text text-anchor="middle" x="232.4585" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0BA</text>
+        </g>
+        <!-- travers -->
+        <g id="node17" class="node">
+            <title>travers</title>
+            <ellipse fill="none" stroke="#0000ff" cx="203.2369" cy="-106.8" rx="38.305" ry="18"/>
+            <text text-anchor="middle" x="203.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#0000ff">0x0BB</text>
+        </g>
+        <!-- traver&#45;&gt;travers -->
+        <g id="edge20" class="edge">
+            <title>traver&#45;&gt;travers</title>
+            <path fill="none" stroke="#add8e6" d="M183.9854,-179.6658C174.6965,-169.5397 166.3137,-155.9116 171.2923,-142.8 172.9385,-138.4645 175.3723,-134.3169 178.1749,-130.4793"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="184.7367,-122.6213 181.7812,-133.1814 181.5319,-126.4592 178.3271,-130.2971 178.3271,-130.2971 178.3271,-130.2971 181.5319,-126.4592 174.873,-127.4128 184.7367,-122.6213 184.7367,-122.6213"/>
+            <text text-anchor="middle" x="176.7092" y="-147" font-family="Times,serif" font-size="14.00" fill="#0000ff"> s</text>
+        </g>
+        <!-- traver&#45;&gt;travers -->
+        <g id="edge33" class="edge">
+            <title>traver&#45;&gt;travers</title>
+            <path fill="none" stroke="#0000ff" d="M202.4441,-177.2006C202.5804,-165.0949 202.7616,-149.0076 202.9163,-135.2674"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="206.4205,-134.9108 203.0334,-124.872 199.4209,-134.8319 206.4205,-134.9108"/>
+            <text text-anchor="middle" x="207.7092" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> s</text>
+        </g>
+        <!-- travers&#45;&gt;traver -->
+        <g id="edge36" class="edge">
+            <title>travers&#45;&gt;traver</title>
+            <path fill="none" stroke="#ffc0cb" d="M209.0666,-124.8376C211.6638,-135.0186 213.8193,-147.9908 212.2369,-159.6 211.8717,-162.2793 211.3602,-165.0407 210.7599,-167.7863"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="208.2983,-177.4866 206.3963,-166.6869 209.5282,-172.6402 210.758,-167.7938 210.758,-167.7938 210.758,-167.7938 209.5282,-172.6402 215.1198,-168.9007 208.2983,-177.4866 208.2983,-177.4866"/>
+            <text text-anchor="middle" x="233.0735" y="-147" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0BB</text>
+        </g>
+        <!-- traverse -->
+        <g id="node18" class="node">
+            <title>traverse</title>
+            <ellipse fill="none" stroke="#0000ff" cx="203.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="203.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#0000ff">contentArray[3]</text>
+        </g>
+        <!-- travers&#45;&gt;traverse -->
+        <g id="edge21" class="edge">
+            <title>travers&#45;&gt;traverse</title>
+            <path fill="none" stroke="#add8e6" d="M184.8676,-90.9569C179.5135,-85.2132 174.3474,-78.2805 171.5223,-70.8 168.8843,-63.8149 168.8843,-60.9851 171.5223,-54 172.8668,-50.4399 174.7415,-47.004 176.9144,-43.7612"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="183.203,-35.676 180.6155,-46.3323 180.1332,-39.6227 177.0634,-43.5695 177.0634,-43.5695 177.0634,-43.5695 180.1332,-39.6227 173.5114,-40.8067 183.203,-35.676 183.203,-35.676"/>
+            <text text-anchor="middle" x="177.0942" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- travers&#45;&gt;traverse -->
+        <g id="edge34" class="edge">
+            <title>travers&#45;&gt;traverse</title>
+            <path fill="none" stroke="#0000ff" d="M203.2369,-88.4006C203.2369,-76.2949 203.2369,-60.2076 203.2369,-46.4674"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="206.737,-46.072 203.2369,-36.072 199.737,-46.0721 206.737,-46.072"/>
+            <text text-anchor="middle" x="208.0942" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- traverse&#45;&gt;travers -->
+        <g id="edge35" class="edge">
+            <title>traverse&#45;&gt;travers</title>
+            <path fill="none" stroke="#ffc0cb" d="M209.9024,-36.0722C211.6782,-41.7317 213.3487,-48.0535 214.2369,-54 215.3399,-61.3847 215.3399,-63.4153 214.2369,-70.8 213.8344,-73.4945 213.2713,-76.2661 212.6106,-79.0181"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="209.9024,-88.7278 208.2546,-77.8864 211.2458,-83.9116 212.5891,-79.0954 212.5891,-79.0954 212.5891,-79.0954 211.2458,-83.9116 216.9237,-80.3044 209.9024,-88.7278 209.9024,-88.7278"/>
+            <text text-anchor="middle" x="224.2732" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#ff0000"> ~3</text>
+        </g>
+        <!-- tra2&#45;&gt;trac -->
+        <g id="edge29" class="edge">
+            <title>tra2&#45;&gt;trac</title>
+            <path fill="none" stroke="#0000ff" d="M208.0507,-474.7233C183.1741,-455.0282 146.5501,-426.0326 121.6137,-406.2901"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="123.6518,-403.4396 113.6389,-399.9764 119.3067,-408.9278 123.6518,-403.4396"/>
+            <text text-anchor="middle" x="178.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> c</text>
+        </g>
+        <!-- tra2&#45;&gt;tre -->
+        <!-- tra2&#45;&gt;tr -->
+        <g id="edge40" class="edge">
+            <title>tra2&#45;&gt;tr</title>
+            <path fill="none" stroke="#ffc0cb" d="M233.9039,-526.0074C233.3268,-538.8926 232.6427,-554.1663 232.0226,-568.0109"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="231.5637,-578.2562 227.5157,-568.0648 231.7874,-573.2612 232.0112,-568.2662 232.0112,-568.2662 232.0112,-568.2662 231.7874,-573.2612 236.5067,-568.4676 231.5637,-578.2562 231.5637,-578.2562"/>
+            <text text-anchor="middle" x="253.0665" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0DE</text>
+        </g>
+        <!-- tra2&#45;&gt;trav -->
+        <g id="edge30" class="edge">
+            <title>tra2&#45;&gt;trav</title>
+            <path fill="none" stroke="#0000ff" d="M214.7335,-471.0228C210.2756,-464.2044 206.1695,-456.5615 203.7369,-448.8313 201.3127,-441.1275 200.14,-432.6433 199.7048,-424.4366"/>
+            <polygon fill="#0000ff" stroke="#0000ff" points="203.201,-424.2005 199.5092,-414.2697 196.2023,-424.3352 203.201,-424.2005"/>
+            <text text-anchor="middle" x="209.4869" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> v</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.p1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.p1.svg
new file mode 100644
index 0000000000..c89c085820
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.p1.svg
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+digraph G {
+    { rank=same root -> start [style=invis] }
+    newrank = true
+    start [label="start/end"]
+
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"]
+
+    tree [label = "contentArray[1]"; color = "lightgrey"; fontcolor = lightgray]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    root -> t [label = " t"]
+    t -> tr [label = " r"]
+    tr -> tra [label = " a"]
+    tra -> trac [label = " c"]
+    trac -> tract [label = " t"]
+    tract -> tracto [label = " o"]
+    tracto -> tractor [label = " r"]
+
+    tr -> tre [label = " e"]
+    tre -> tree [label = " e"; color = "lightgrey"; fontcolor = lightgray]
+
+    tr -> tri [label = " i"]
+    tri -> trie [label = " e"]
+
+    {
+        ranksep = 0.1
+        tree2 [label = "Chain\n0x0BB"]
+        tree2p [label = "Prefix\n0x0BF\ncontentArray[1]"]
+        tree2p -> tree2 [label = " &epsilon;"]
+    }
+
+    tre -> tree2p [label = " e"]
+    tree2 -> trees [label = " s"]
+
+    {rank=same tra -> tre -> tri [style=invis]}
+    {rank=same trac -> tree -> tree2p -> trie [style=invis]}
+
+    subgraph path {
+        edge [color = "lightblue"; fontcolor="blue"; arrowhead="vee"]
+        node [color = "lightblue"; fontcolor="blue"]
+
+        start -> root
+
+        root -> t [label = " t"]
+        t -> tr [label = " r"]
+        tr -> tre [label = " e"]
+        tre -> tree [label = " e"]
+        tree -> trees [label = " s"]
+
+        trees [label = "contentArray[3]"; constraint = false]
+    }
+
+
+    subgraph back {
+        edge [color = "pink"; fontcolor="red"; arrowhead="vee"; constraint="false"]
+
+        trees -> tree2 [label = " ~3"]
+        tree2 -> tree2p [label = "0x0BB"]
+        tree2p -> tre [label = "0x0BF"]
+        tre -> tr [label = "0x03B"]
+        tr -> t [label = "0x07E"]
+        t -> root [label = "0x09B"]
+        root -> start [label = "0x09A"]
+    }
+}
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="666pt" height="678pt"
+     viewBox="0.00 0.00 666.47 677.52" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 673.5152)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-673.5152 662.4738,-673.5152 662.4738,4 -4,4"/>
+        <!-- root -->
+        <g id="node1" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="264.2369" cy="-640.0996" rx="33.1337" ry="29.3315"/>
+            <text text-anchor="middle" x="264.2369" y="-644.2996" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="264.2369" y="-627.4996" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- start -->
+        <g id="node2" class="node">
+            <title>start</title>
+            <ellipse fill="none" stroke="#000000" cx="397.2369" cy="-640.0996" rx="44.0775" ry="18"/>
+            <text text-anchor="middle" x="397.2369" y="-635.8996" font-family="Times,serif" font-size="14.00" fill="#000000">start/end</text>
+        </g>
+        <!-- root&#45;&gt;start -->
+        <!-- root&#45;&gt;start -->
+        <g id="edge33" class="edge">
+            <title>root&#45;&gt;start</title>
+            <path fill="none" stroke="#ffc0cb" d="M297.7984,-640.0996C311.3037,-640.0996 327.2339,-640.0996 342.3398,-640.0996"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="352.6261,-640.0996 342.6262,-644.5997 347.6261,-640.0996 342.6261,-640.0997 342.6261,-640.0997 342.6261,-640.0997 347.6261,-640.0996 342.6261,-635.5997 352.6261,-640.0996 352.6261,-640.0996"/>
+            <text text-anchor="middle" x="325.2509" y="-647.2996" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x09A</text>
+        </g>
+        <!-- t -->
+        <g id="node13" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="264.2369" cy="-557.8839" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="264.2369" y="-553.6839" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge2" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#000000" d="M253.1918,-612.3067C251.4263,-605.2505 250.368,-597.5734 252.6036,-585.3125"/>
+            <polygon fill="#000000" stroke="#000000" points="256.0259,-586.0473 254.8506,-575.5179 249.2031,-584.4821 256.0259,-586.0473"/>
+            <text text-anchor="middle" x="255.9322" y="-589.0839" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge22" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#add8e6" d="M264.2369,-610.4178C264.2369,-602.5756 264.2369,-594.1307 264.2369,-586.3542"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="264.2369,-576.2835 268.737,-586.2835 264.2369,-581.2835 264.237,-586.2835 264.237,-586.2835 264.237,-586.2835 264.2369,-581.2835 259.737,-586.2836 264.2369,-576.2835 264.2369,-576.2835"/>
+            <text text-anchor="middle" x="267.9322" y="-589.0839" font-family="Times,serif" font-size="14.00" fill="#0000ff"> t</text>
+        </g>
+        <!-- start&#45;&gt;root -->
+        <g id="edge21" class="edge">
+            <title>start&#45;&gt;root</title>
+            <path fill="none" stroke="#add8e6" d="M353.4042,-637.4233C341.2938,-636.9719 328.1479,-636.7485 316.0184,-637.0996 313.3264,-637.1775 310.5568,-637.2781 307.7643,-637.3951"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="297.6504,-637.8791 307.4239,-632.9062 302.6447,-637.6401 307.639,-637.401 307.639,-637.401 307.639,-637.401 302.6447,-637.6401 307.8541,-641.8959 297.6504,-637.8791 297.6504,-637.8791"/>
+        </g>
+        <!-- tractor -->
+        <g id="node3" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tracto -->
+        <g id="node4" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-88.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-84.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge8" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-70.5672C73.2369,-63.2743 73.2369,-54.6987 73.2369,-46.6137"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-46.417 73.2369,-36.417 69.737,-46.4171 76.737,-46.417"/>
+            <text text-anchor="middle" x="77.3172" y="-49.2" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- tract -->
+        <g id="node5" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-171.0156" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-166.8156" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge7" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-152.7693C73.2369,-142.338 73.2369,-129.027 73.2369,-117.2514"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-116.9953 73.2369,-106.9953 69.737,-116.9953 76.737,-116.9953"/>
+            <text text-anchor="middle" x="78.4869" y="-120" font-family="Times,serif" font-size="14.00" fill="#000000"> o</text>
+        </g>
+        <!-- trac -->
+        <g id="node6" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-276.5263" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-272.3263" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge6" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-258.1478C73.2369,-241.8835 73.2369,-217.8982 73.2369,-199.2081"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-199.1836 73.2369,-189.1836 69.737,-199.1836 76.737,-199.1836"/>
+            <text text-anchor="middle" x="76.9322" y="-213.6313" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- tree -->
+        <g id="node8" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#d3d3d3" cx="217.2369" cy="-276.5263" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="217.2369" y="-272.3263" font-family="Times,serif" font-size="14.00" fill="#d3d3d3">contentArray[1]</text>
+        </g>
+        <!-- trac&#45;&gt;tree -->
+        <!-- tra -->
+        <g id="node7" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#000000" cx="141.2369" cy="-382.037" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="141.2369" y="-386.237" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="141.2369" y="-369.437" font-family="Times,serif" font-size="14.00" fill="#000000">0x018</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge5" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#000000" d="M124.4279,-355.9557C113.8753,-339.582 100.3244,-318.5561 89.8158,-302.2506"/>
+            <polygon fill="#000000" stroke="#000000" points="92.6883,-300.2467 84.329,-293.7372 86.8044,-304.0388 92.6883,-300.2467"/>
+            <text text-anchor="middle" x="121.0942" y="-331.0214" font-family="Times,serif" font-size="14.00" fill="#000000"> c</text>
+        </g>
+        <!-- tre -->
+        <g id="node9" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="250.2369" cy="-382.037" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="250.2369" y="-386.237" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="250.2369" y="-369.437" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tra&#45;&gt;tre -->
+        <!-- tree2p -->
+        <g id="node15" class="node">
+            <title>tree2p</title>
+            <ellipse fill="none" stroke="#000000" cx="401.2369" cy="-276.5263" rx="75.1528" ry="41.0911"/>
+            <text text-anchor="middle" x="401.2369" y="-289.1263" font-family="Times,serif" font-size="14.00" fill="#000000">Prefix</text>
+            <text text-anchor="middle" x="401.2369" y="-272.3263" font-family="Times,serif" font-size="14.00" fill="#000000">0x0BF</text>
+            <text text-anchor="middle" x="401.2369" y="-255.5263" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- tree&#45;&gt;tree2p -->
+        <!-- trees -->
+        <g id="node16" class="node">
+            <title>trees</title>
+            <ellipse fill="none" stroke="#000000" cx="295.2369" cy="-88.8" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="295.2369" y="-84.6" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[3]</text>
+        </g>
+        <!-- tree&#45;&gt;trees -->
+        <g id="edge26" class="edge">
+            <title>tree&#45;&gt;trees</title>
+            <path fill="none" stroke="#add8e6" d="M224.7159,-258.5263C238.4018,-225.5876 267.5561,-155.4207 283.7655,-116.4088"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="287.7806,-106.7455 288.0991,-117.7068 285.8621,-111.3628 283.9435,-115.9801 283.9435,-115.9801 283.9435,-115.9801 285.8621,-111.3628 279.788,-114.2534 287.7806,-106.7455 287.7806,-106.7455"/>
+            <text text-anchor="middle" x="277.7092" y="-166.8156" font-family="Times,serif" font-size="14.00" fill="#0000ff"> s</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge10" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#d3d3d3" d="M227.0349,-358.8106C223.5725,-354.1203 220.5044,-348.9742 218.5223,-343.6214 214.0163,-331.453 213.1878,-317.1731 213.6897,-304.9221"/>
+            <polygon fill="#d3d3d3" stroke="#d3d3d3" points="217.1925,-305.0065 214.4144,-294.7824 210.2104,-304.5074 217.1925,-305.0065"/>
+            <text text-anchor="middle" x="224.0942" y="-331.0214" font-family="Times,serif" font-size="14.00" fill="#d3d3d3"> e</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge25" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#add8e6" d="M241.2195,-353.2057C236.4749,-338.0358 230.683,-319.5176 225.9798,-304.4799"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="222.9239,-294.7093 230.2038,-302.9101 224.4164,-299.4813 225.909,-304.2534 225.909,-304.2534 225.909,-304.2534 224.4164,-299.4813 221.6142,-305.5967 222.9239,-294.7093 222.9239,-294.7093"/>
+            <text text-anchor="middle" x="243.0942" y="-331.0214" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- tri -->
+        <g id="node11" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="361.2369" cy="-382.037" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="361.2369" y="-386.237" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="361.2369" y="-369.437" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- tre&#45;&gt;tri -->
+        <!-- tr -->
+        <g id="node12" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="264.2369" cy="-475.6683" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="264.2369" y="-479.8683" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="264.2369" y="-463.0683" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- tre&#45;&gt;tr -->
+        <g id="edge30" class="edge">
+            <title>tre&#45;&gt;tr</title>
+            <path fill="none" stroke="#ffc0cb" d="M220.1451,-399.8185C206.9606,-410.0222 196.0135,-423.5087 203.5777,-437.2526 207.8544,-445.0234 214.4401,-451.4422 221.7208,-456.6604"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="230.4902,-462.2435 219.638,-460.6689 226.2725,-459.5582 222.0547,-456.8729 222.0547,-456.8729 222.0547,-456.8729 226.2725,-459.5582 224.4715,-453.0769 230.4902,-462.2435 230.4902,-462.2435"/>
+            <text text-anchor="middle" x="224.0665" y="-424.6526" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x03B</text>
+        </g>
+        <!-- tre&#45;&gt;tree2p -->
+        <g id="edge14" class="edge">
+            <title>tre&#45;&gt;tree2p</title>
+            <path fill="none" stroke="#000000" d="M281.9441,-365.9135C292.0842,-361.18 303.4736,-356.2938 314.2369,-352.6214 331.0728,-346.877 338.4386,-353.4933 353.2369,-343.6214 360.9367,-338.4848 367.8078,-331.6659 373.7781,-324.3413"/>
+            <polygon fill="#000000" stroke="#000000" points="376.657,-326.3371 379.8994,-316.2506 371.0746,-322.1136 376.657,-326.3371"/>
+            <text text-anchor="middle" x="377.0942" y="-331.0214" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- trie -->
+        <g id="node10" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="585.2369" cy="-276.5263" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="585.2369" y="-272.3263" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge12" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#000000" d="M393.8084,-366.6948C432.9219,-348.2712 498.6628,-317.3053 541.999,-296.8927"/>
+            <polygon fill="#000000" stroke="#000000" points="543.6019,-300.0065 551.1571,-292.5789 540.619,-293.6739 543.6019,-300.0065"/>
+            <text text-anchor="middle" x="478.0942" y="-331.0214" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge4" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#000000" d="M230.3862,-462.3473C216.3498,-455.9318 200.378,-447.4106 187.5223,-437.2526 179.7381,-431.102 172.3346,-423.4626 165.8229,-415.8473"/>
+            <polygon fill="#000000" stroke="#000000" points="168.1714,-413.1902 159.1314,-407.6649 162.7526,-417.6217 168.1714,-413.1902"/>
+            <text text-anchor="middle" x="193.0942" y="-424.6526" font-family="Times,serif" font-size="14.00" fill="#000000"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge9" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#000000" d="M262.3234,-446.0575C261.7354,-439.4979 260.9854,-432.4951 259.2059,-421.283"/>
+            <polygon fill="#000000" stroke="#000000" points="262.641,-420.6036 257.5414,-411.3168 255.7366,-421.7567 262.641,-420.6036"/>
+            <text text-anchor="middle" x="267.0942" y="-424.6526" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge24" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#add8e6" d="M251.4368,-447.9439C250.2405,-444.4076 249.2174,-440.7879 248.5223,-437.2526 247.53,-432.2063 247.0287,-426.8513 246.8562,-421.5383"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="246.8871,-411.4654 251.3563,-421.4792 246.8717,-416.4654 246.8563,-421.4654 246.8563,-421.4654 246.8563,-421.4654 246.8717,-416.4654 242.3563,-421.4516 246.8871,-411.4654 246.8871,-411.4654"/>
+            <text text-anchor="middle" x="254.0942" y="-424.6526" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge11" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#000000" d="M288.2144,-452.5235C300.8059,-440.3693 316.39,-425.3264 329.9234,-412.263"/>
+            <polygon fill="#000000" stroke="#000000" points="332.4349,-414.7033 337.199,-405.2401 327.5733,-409.6669 332.4349,-414.7033"/>
+            <text text-anchor="middle" x="325.9322" y="-424.6526" font-family="Times,serif" font-size="14.00" fill="#000000"> i</text>
+        </g>
+        <!-- tr&#45;&gt;t -->
+        <g id="edge31" class="edge">
+            <title>tr&#45;&gt;t</title>
+            <path fill="none" stroke="#ffc0cb" d="M264.2369,-505.1649C264.2369,-513.0799 264.2369,-521.6204 264.2369,-529.4751"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="264.2369,-539.6376 259.737,-529.6376 264.2369,-534.6376 264.237,-529.6376 264.237,-529.6376 264.237,-529.6376 264.2369,-534.6376 268.737,-529.6377 264.2369,-539.6376 264.2369,-539.6376"/>
+            <text text-anchor="middle" x="282.5132" y="-518.2839" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x07E</text>
+        </g>
+        <!-- t&#45;&gt;root -->
+        <g id="edge32" class="edge">
+            <title>t&#45;&gt;root</title>
+            <path fill="none" stroke="#ffc0cb" d="M272.5745,-575.5942C273.6774,-578.6326 274.6336,-581.8043 275.2369,-584.8839 276.3056,-590.3394 276.3564,-596.0962 275.789,-601.7512"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="274.1862,-611.733 271.3286,-601.146 274.9789,-606.7962 275.7717,-601.8595 275.7717,-601.8595 275.7717,-601.8595 274.9789,-606.7962 280.2148,-602.573 274.1862,-611.733 274.1862,-611.733"/>
+            <text text-anchor="middle" x="294.9052" y="-589.0839" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x09B</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge3" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#000000" d="M254.2705,-540.2954C252.9499,-537.2423 251.8032,-534.0343 251.0763,-530.8839 249.8023,-525.3626 249.7646,-519.536 250.472,-513.8197"/>
+            <polygon fill="#000000" stroke="#000000" points="253.9589,-514.2255 252.4441,-503.7395 247.0891,-512.8814 253.9589,-514.2255"/>
+            <text text-anchor="middle" x="255.3172" y="-518.2839" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge23" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#add8e6" d="M296.3684,-548.8052C304.9495,-544.7547 313.2223,-538.9976 318.2369,-530.8839 325.8698,-518.5336 317.3847,-506.7525 305.051,-497.2687"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="296.521,-491.3854 307.3079,-493.3588 300.6369,-494.2243 304.7529,-497.0631 304.7529,-497.0631 304.7529,-497.0631 300.6369,-494.2243 302.1979,-500.7675 296.521,-491.3854 296.521,-491.3854"/>
+            <text text-anchor="middle" x="325.3172" y="-518.2839" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- tree2 -->
+        <g id="node14" class="node">
+            <title>tree2</title>
+            <ellipse fill="none" stroke="#000000" cx="354.2369" cy="-171.0156" rx="39.2342" ry="29.3315"/>
+            <text text-anchor="middle" x="354.2369" y="-175.2156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="354.2369" y="-158.4156" font-family="Times,serif" font-size="14.00" fill="#000000">0x0BB</text>
+        </g>
+        <!-- tree2&#45;&gt;tree2p -->
+        <g id="edge28" class="edge">
+            <title>tree2&#45;&gt;tree2p</title>
+            <path fill="none" stroke="#ffc0cb" d="M332.5998,-195.7225C326.8044,-205.3439 323.4112,-216.3994 328.5637,-226.2313 331.3469,-231.5422 334.9738,-236.4037 339.122,-240.8292"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="346.5383,-247.8677 336.1871,-244.2478 342.9116,-244.4257 339.2849,-240.9838 339.2849,-240.9838 339.2849,-240.9838 342.9116,-244.4257 342.3827,-237.7197 346.5383,-247.8677 346.5383,-247.8677"/>
+            <text text-anchor="middle" x="349.0735" y="-213.6313" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0BB</text>
+        </g>
+        <!-- tree2&#45;&gt;trees -->
+        <g id="edge15" class="edge">
+            <title>tree2&#45;&gt;trees</title>
+            <path fill="none" stroke="#000000" d="M335.5689,-145.0021C328.6935,-135.4213 320.9064,-124.5701 314.0208,-114.9751"/>
+            <polygon fill="#000000" stroke="#000000" points="316.6766,-112.6728 308.0027,-106.5889 310.9895,-116.7541 316.6766,-112.6728"/>
+            <text text-anchor="middle" x="331.7092" y="-120" font-family="Times,serif" font-size="14.00" fill="#000000"> s</text>
+        </g>
+        <!-- tree2p&#45;&gt;tre -->
+        <g id="edge29" class="edge">
+            <title>tree2p&#45;&gt;tre</title>
+            <path fill="none" stroke="#ffc0cb" d="M346.159,-304.6442C334.2868,-311.4197 322.0324,-318.9824 311.1177,-326.8214 300.7955,-334.2347 290.2869,-343.1729 280.9708,-351.6776"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="273.4561,-358.6766 277.7068,-348.5681 277.1149,-355.2689 280.7738,-351.8611 280.7738,-351.8611 280.7738,-351.8611 277.1149,-355.2689 283.8408,-355.1541 273.4561,-358.6766 273.4561,-358.6766"/>
+            <text text-anchor="middle" x="330.2965" y="-331.0214" font-family="Times,serif" font-size="14.00" fill="#ff0000">0x0BF</text>
+        </g>
+        <!-- tree2p&#45;&gt;trie -->
+        <!-- tree2p&#45;&gt;tree2 -->
+        <g id="edge13" class="edge">
+            <title>tree2p&#45;&gt;tree2</title>
+            <path fill="none" stroke="#000000" d="M383.3341,-236.3363C379.2138,-227.0866 374.8498,-217.2898 370.7898,-208.1754"/>
+            <polygon fill="#000000" stroke="#000000" points="373.9581,-206.6863 366.6918,-198.9757 367.5638,-209.5346 373.9581,-206.6863"/>
+            <text text-anchor="middle" x="383.0672" y="-213.6313" font-family="Times,serif" font-size="14.00" fill="#000000"> ε</text>
+        </g>
+        <!-- trees&#45;&gt;tree2 -->
+        <g id="edge27" class="edge">
+            <title>trees&#45;&gt;tree2</title>
+            <path fill="none" stroke="#ffc0cb" d="M290.8236,-106.8267C289.7499,-115.1419 289.9269,-124.8606 294.1643,-132.6 298.1354,-139.8531 304.124,-145.9366 310.807,-150.9671"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="319.5438,-156.7845 308.7261,-154.9878 315.382,-154.0133 311.2202,-151.2421 311.2202,-151.2421 311.2202,-151.2421 315.382,-154.0133 313.7143,-147.4965 319.5438,-156.7845 319.5438,-156.7845"/>
+            <text text-anchor="middle" x="303.2732" y="-120" font-family="Times,serif" font-size="14.00" fill="#ff0000"> ~3</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w1.svg
new file mode 100644
index 0000000000..1be94ae237
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w1.svg
@@ -0,0 +1,226 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+# embedded DOT (plantuml) works in IDEA preview
+# but not on GitHub
+``` plantuml
+digraph G {
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"]
+
+    tree [label = "contentArray[1]"]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    // edge [color="none", fontcolor="none"]
+    root -> t [label = " t"]
+    t -> tr [label = " r"]
+    tr -> tra [label = " a"]
+    tra -> trac [label = " c"]
+    trac -> tract [label = " t"]
+    tract -> tracto [label = " o"]
+    tracto -> tractor [label = " r"]
+
+    tr -> tre [label = " e"]
+    tre -> tree [label = " e"]
+
+    tr -> tri [label = " i"]
+    tri -> trie [label = " e"]
+
+    // {rank=same tra -> tre -> tri [style=invis]}
+    {rank=same trac -> tree -> trie [style=invis]}
+}
+```
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="480pt" height="734pt"
+     viewBox="0.00 0.00 480.47 734.09" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 730.0939)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-730.0939 476.4738,-730.0939 476.4738,4 -4,4"/>
+        <!-- tractor -->
+        <g id="node1" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tracto -->
+        <g id="node2" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge7" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-88.4006C73.2369,-76.2949 73.2369,-60.2076 73.2369,-46.4674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-46.072 73.2369,-36.072 69.737,-46.0721 76.737,-46.072"/>
+            <text text-anchor="middle" x="77.3172" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- tract -->
+        <g id="node3" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-195.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge6" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-177.2006C73.2369,-165.0949 73.2369,-149.0076 73.2369,-135.2674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-134.872 73.2369,-124.872 69.737,-134.8721 76.737,-134.872"/>
+            <text text-anchor="middle" x="78.4869" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> o</text>
+        </g>
+        <!-- trac -->
+        <g id="node4" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-284.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge5" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-266.0006C73.2369,-253.8949 73.2369,-237.8076 73.2369,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-223.672 73.2369,-213.672 69.737,-223.6721 76.737,-223.672"/>
+            <text text-anchor="middle" x="76.9322" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- tree -->
+        <g id="node6" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="217.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- trac&#45;&gt;tree -->
+        <!-- tra -->
+        <g id="node5" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#000000" cx="112.2369" cy="-384.6156" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="112.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="112.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x018</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge4" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#000000" d="M101.2694,-356.4333C95.8306,-342.4575 89.2887,-325.6472 83.8929,-311.7821"/>
+            <polygon fill="#000000" stroke="#000000" points="87.0872,-310.3393 80.1988,-302.2895 80.5638,-312.878 87.0872,-310.3393"/>
+            <text text-anchor="middle" x="98.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> c</text>
+        </g>
+        <!-- trie -->
+        <g id="node8" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="399.2369" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="399.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tree&#45;&gt;trie -->
+        <!-- tre -->
+        <g id="node7" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="217.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge9" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#000000" d="M217.2369,-355.0897C217.2369,-341.6046 217.2369,-325.7123 217.2369,-312.4153"/>
+            <polygon fill="#000000" stroke="#000000" points="220.737,-312.4033 217.2369,-302.4033 213.737,-312.4034 220.737,-312.4033"/>
+            <text text-anchor="middle" x="222.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tri -->
+        <g id="node9" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="354.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="354.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="354.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge11" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#000000" d="M366.7719,-356.7C373.1109,-342.583 380.7723,-325.5209 387.0603,-311.5175"/>
+            <polygon fill="#000000" stroke="#000000" points="390.2575,-312.9415 391.1609,-302.3852 383.8717,-310.074 390.2575,-312.9415"/>
+            <text text-anchor="middle" x="386.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr -->
+        <g id="node10" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-496.2469" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="217.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge3" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#000000" d="M194.7118,-472.2992C179.1047,-455.7065 158.1828,-433.4633 141.317,-415.5323"/>
+            <polygon fill="#000000" stroke="#000000" points="143.7653,-413.0267 134.3644,-408.1406 138.6664,-417.8227 143.7653,-413.0267"/>
+            <text text-anchor="middle" x="176.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge8" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#000000" d="M217.2369,-466.6249C217.2369,-453.7568 217.2369,-438.4867 217.2369,-424.6319"/>
+            <polygon fill="#000000" stroke="#000000" points="220.737,-424.3761 217.2369,-414.3761 213.737,-424.3762 220.737,-424.3761"/>
+            <text text-anchor="middle" x="222.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge10" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#000000" d="M243.328,-474.9872C264.9956,-457.3319 295.9963,-432.0716 319.7169,-412.7435"/>
+            <polygon fill="#000000" stroke="#000000" points="322.2348,-415.2066 327.7763,-406.1765 317.813,-409.78 322.2348,-415.2066"/>
+            <text text-anchor="middle" x="296.9322" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> i</text>
+        </g>
+        <!-- t -->
+        <g id="node11" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-596.4626" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="217.2369" y="-592.2626" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge2" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#000000" d="M217.2369,-578.0815C217.2369,-566.3502 217.2369,-550.6774 217.2369,-536.1885"/>
+            <polygon fill="#000000" stroke="#000000" points="220.737,-535.9208 217.2369,-525.9209 213.737,-535.9209 220.737,-535.9208"/>
+            <text text-anchor="middle" x="221.3172" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- root -->
+        <g id="node12" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-696.6782" rx="33.1337" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-700.8782" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="217.2369" y="-684.0782" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge1" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#000000" d="M217.2369,-667.1522C217.2369,-653.6671 217.2369,-637.7749 217.2369,-624.4779"/>
+            <polygon fill="#000000" stroke="#000000" points="220.737,-624.4659 217.2369,-614.4659 213.737,-624.4659 220.737,-624.4659"/>
+            <text text-anchor="middle" x="220.9322" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w2.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w2.svg
new file mode 100644
index 0000000000..9d8ab22e69
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w2.svg
@@ -0,0 +1,326 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+```plantuml
+digraph G {
+    { rank=same root -> start [style=invis] }
+    start [label="start/end"]
+
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"]
+
+    tree [label = "contentArray[1]"]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    // {rank=same tra -> tre -> tri [style=invis]}
+    {rank=same trac -> tree -> trie [style=invis]}
+
+    subgraph path {
+        edge [color = "lightblue"; fontcolor="blue"; arrowhead="vee"]
+
+        start -> root
+
+    root -> t [label = " t"]
+    t -> tr [label = " r"]
+    tr -> tra [label = " a"]
+    tra -> trac [label = " c"]
+    trac -> tract [label = " t"]
+    tract -> tracto [label = " o"]
+    tracto -> tractor [label = " r"]
+
+    tr -> tre [label = " e"]
+    tre -> tree [label = " e"]
+
+    tr -> tri [label = " i"]
+    tri -> trie [label = " e"]
+    }
+
+    subgraph back {
+        edge [color = "pink"; fontcolor="blue"; constraint="false"; arrowhead="vee"]
+
+        tractor -> tracto -> tract -> trac -> tra -> tr
+        tree -> tre -> tr
+        trie -> tri -> tr -> t -> root -> start
+    }
+}
+
+```
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="480pt" height="734pt"
+     viewBox="0.00 0.00 480.47 734.09" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 730.0939)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-730.0939 476.4738,-730.0939 476.4738,4 -4,4"/>
+        <!-- root -->
+        <g id="node1" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-696.6782" rx="33.1337" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-700.8782" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="217.2369" y="-684.0782" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- start -->
+        <g id="node2" class="node">
+            <title>start</title>
+            <ellipse fill="none" stroke="#000000" cx="330.2369" cy="-696.6782" rx="44.0775" ry="18"/>
+            <text text-anchor="middle" x="330.2369" y="-692.4782" font-family="Times,serif" font-size="14.00" fill="#000000">start/end</text>
+        </g>
+        <!-- root&#45;&gt;start -->
+        <!-- root&#45;&gt;start -->
+        <g id="edge27" class="edge">
+            <title>root&#45;&gt;start</title>
+            <path fill="none" stroke="#ffc0cb" d="M245.0455,-713.0508C262.0796,-719.8061 279.1137,-720.5455 296.1478,-715.2689"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="305.6112,-711.7185 297.8292,-719.4444 300.9299,-713.4748 296.2485,-715.2312 296.2485,-715.2312 296.2485,-715.2312 300.9299,-713.4748 294.6678,-711.0179 305.6112,-711.7185 305.6112,-711.7185"/>
+        </g>
+        <!-- t -->
+        <g id="node13" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-596.4626" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="217.2369" y="-592.2626" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge5" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#add8e6" d="M217.2369,-667.1522C217.2369,-653.6671 217.2369,-637.7749 217.2369,-624.4779"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="217.2369,-614.4659 221.737,-624.4659 217.2369,-619.4659 217.237,-624.4659 217.237,-624.4659 217.237,-624.4659 217.2369,-619.4659 212.737,-624.4659 217.2369,-614.4659 217.2369,-614.4659"/>
+            <text text-anchor="middle" x="220.9322" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> t</text>
+        </g>
+        <!-- start&#45;&gt;root -->
+        <g id="edge4" class="edge">
+            <title>start&#45;&gt;root</title>
+            <path fill="none" stroke="#add8e6" d="M285.8549,-696.6782C277.5437,-696.6782 269.2326,-696.6782 260.9215,-696.6782"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="250.7838,-696.6782 260.7838,-692.1783 255.7838,-696.6783 260.7838,-696.6783 260.7838,-696.6783 260.7838,-696.6783 255.7838,-696.6783 260.7837,-701.1783 250.7838,-696.6782 250.7838,-696.6782"/>
+        </g>
+        <!-- tractor -->
+        <g id="node3" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tracto -->
+        <g id="node4" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tractor&#45;&gt;tracto -->
+        <g id="edge16" class="edge">
+            <title>tractor&#45;&gt;tracto</title>
+            <path fill="none" stroke="#ffc0cb" d="M78.6937,-36.1509C80.1467,-41.8136 81.5127,-48.1174 82.2369,-54 83.1492,-61.4107 83.1492,-63.3893 82.2369,-70.8 81.9201,-73.3736 81.4804,-76.0279 80.9643,-78.6738"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="78.6937,-88.6491 76.5255,-77.8997 79.8035,-83.7738 80.9132,-78.8985 80.9132,-78.8985 80.9132,-78.8985 79.8035,-83.7738 85.301,-79.8973 78.6937,-88.6491 78.6937,-88.6491"/>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge11" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#add8e6" d="M73.2369,-88.4006C73.2369,-76.2949 73.2369,-60.2076 73.2369,-46.4674"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="73.2369,-36.072 77.737,-46.072 73.2369,-41.072 73.237,-46.072 73.237,-46.072 73.237,-46.072 73.2369,-41.072 68.737,-46.0721 73.2369,-36.072 73.2369,-36.072"/>
+            <text text-anchor="middle" x="77.3172" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- tract -->
+        <g id="node5" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-195.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tracto&#45;&gt;tract -->
+        <g id="edge17" class="edge">
+            <title>tracto&#45;&gt;tract</title>
+            <path fill="none" stroke="#ffc0cb" d="M80.3796,-124.4595C82.3635,-130.2097 84.2449,-136.6872 85.2369,-142.8 86.4329,-150.1703 86.4329,-152.2297 85.2369,-159.6 84.797,-162.3109 84.1821,-165.0936 83.461,-167.8527"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="80.5061,-177.5725 79.1093,-166.6959 81.9604,-172.7886 83.4148,-168.0048 83.4148,-168.0048 83.4148,-168.0048 81.9604,-172.7886 87.7202,-169.3137 80.5061,-177.5725 80.5061,-177.5725"/>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge10" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#add8e6" d="M73.2369,-177.2006C73.2369,-165.0949 73.2369,-149.0076 73.2369,-135.2674"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="73.2369,-124.872 77.737,-134.872 73.2369,-129.872 73.237,-134.872 73.237,-134.872 73.237,-134.872 73.2369,-129.872 68.737,-134.8721 73.2369,-124.872 73.2369,-124.872"/>
+            <text text-anchor="middle" x="78.4869" y="-147" font-family="Times,serif" font-size="14.00" fill="#0000ff"> o</text>
+        </g>
+        <!-- trac -->
+        <g id="node6" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-284.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- tract&#45;&gt;trac -->
+        <g id="edge18" class="edge">
+            <title>tract&#45;&gt;trac</title>
+            <path fill="none" stroke="#ffc0cb" d="M78.0886,-213.7847C79.3802,-219.4487 80.5942,-225.7449 81.2369,-231.6 82.0516,-239.0221 82.0516,-240.9779 81.2369,-248.4 80.9557,-250.9616 80.5652,-253.6076 80.1066,-256.2482"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="78.0886,-266.2153 75.6626,-255.5211 79.0808,-261.3147 80.0731,-256.4142 80.0731,-256.4142 80.0731,-256.4142 79.0808,-261.3147 84.4836,-257.3072 78.0886,-266.2153 78.0886,-266.2153"/>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge9" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#add8e6" d="M73.2369,-266.0006C73.2369,-253.8949 73.2369,-237.8076 73.2369,-224.0674"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="73.2369,-213.672 77.737,-223.672 73.2369,-218.672 73.237,-223.672 73.237,-223.672 73.237,-223.672 73.2369,-218.672 68.737,-223.6721 73.2369,-213.672 73.2369,-213.672"/>
+            <text text-anchor="middle" x="76.9322" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#0000ff"> t</text>
+        </g>
+        <!-- tra -->
+        <g id="node7" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#000000" cx="112.2369" cy="-384.6156" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="112.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="112.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x018</text>
+        </g>
+        <!-- trac&#45;&gt;tra -->
+        <g id="edge19" class="edge">
+            <title>trac&#45;&gt;tra</title>
+            <path fill="none" stroke="#ffc0cb" d="M92.2023,-299.8107C98.0398,-305.6172 103.8226,-312.698 107.2369,-320.4 110.6198,-328.0312 112.455,-336.6721 113.3574,-345.0849"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="114.0369,-355.1505 108.8735,-345.4763 113.7001,-350.1618 113.3633,-345.1732 113.3633,-345.1732 113.3633,-345.1732 113.7001,-350.1618 117.853,-344.87 114.0369,-355.1505 114.0369,-355.1505"/>
+        </g>
+        <!-- tree -->
+        <g id="node8" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="217.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- trac&#45;&gt;tree -->
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge8" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#add8e6" d="M101.2694,-356.4333C95.8306,-342.4575 89.2887,-325.6472 83.8929,-311.7821"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="80.1988,-302.2895 88.0191,-309.9766 82.0121,-306.9491 83.8255,-311.6087 83.8255,-311.6087 83.8255,-311.6087 82.0121,-306.9491 79.6319,-313.2407 80.1988,-302.2895 80.1988,-302.2895"/>
+            <text text-anchor="middle" x="98.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> c</text>
+        </g>
+        <!-- tr -->
+        <g id="node12" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-496.2469" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="217.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- tra&#45;&gt;tr -->
+        <g id="edge20" class="edge">
+            <title>tra&#45;&gt;tr</title>
+            <path fill="none" stroke="#ffc0cb" d="M134.8244,-407.6467C142.3198,-415.356 150.6782,-424.0253 158.2369,-432.0313 168.0928,-442.4704 178.7426,-453.987 188.2457,-464.3504"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="195.2572,-472.0136 185.1867,-467.6734 191.882,-468.3247 188.5068,-464.6357 188.5068,-464.6357 188.5068,-464.6357 191.882,-468.3247 191.8268,-461.598 195.2572,-472.0136 195.2572,-472.0136"/>
+        </g>
+        <!-- tre -->
+        <g id="node9" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="217.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tree&#45;&gt;tre -->
+        <g id="edge21" class="edge">
+            <title>tree&#45;&gt;tre</title>
+            <path fill="none" stroke="#ffc0cb" d="M225.7116,-302.3279C227.9713,-307.9814 230.0993,-314.3363 231.2369,-320.4 232.8247,-328.8637 232.2596,-337.9179 230.673,-346.479"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="228.409,-356.2376 226.2855,-345.4793 229.5391,-351.3669 230.6691,-346.4963 230.6691,-346.4963 230.6691,-346.4963 229.5391,-351.3669 235.0527,-347.5133 228.409,-356.2376 228.409,-356.2376"/>
+        </g>
+        <!-- trie -->
+        <g id="node10" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="399.2369" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="399.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tree&#45;&gt;trie -->
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge13" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#add8e6" d="M217.2369,-355.0897C217.2369,-341.6046 217.2369,-325.7123 217.2369,-312.4153"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="217.2369,-302.4033 221.737,-312.4033 217.2369,-307.4033 217.237,-312.4033 217.237,-312.4033 217.237,-312.4033 217.2369,-307.4033 212.737,-312.4034 217.2369,-302.4033 217.2369,-302.4033"/>
+            <text text-anchor="middle" x="222.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- tre&#45;&gt;tr -->
+        <g id="edge22" class="edge">
+            <title>tre&#45;&gt;tr</title>
+            <path fill="none" stroke="#ffc0cb" d="M227.8923,-412.9079C230.9037,-424.04 232.9509,-436.9736 231.2369,-448.8313 230.8027,-451.835 230.2214,-454.9136 229.5418,-457.989"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="227.0968,-467.7057 225.1732,-456.9099 228.317,-462.8569 229.5371,-458.008 229.5371,-458.008 229.5371,-458.008 228.317,-462.8569 233.9011,-459.1062 227.0968,-467.7057 227.0968,-467.7057"/>
+        </g>
+        <!-- tri -->
+        <g id="node11" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="326.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="326.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="326.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- trie&#45;&gt;tri -->
+        <g id="edge23" class="edge">
+            <title>trie&#45;&gt;tri</title>
+            <path fill="none" stroke="#ffc0cb" d="M393.8563,-302.4822C390.1724,-313.1558 384.6144,-326.606 377.2369,-337.2 372.5248,-343.9665 366.7036,-350.493 360.6987,-356.4272"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="353.0638,-363.6005 357.2704,-353.4736 356.7078,-360.1768 360.3518,-356.7531 360.3518,-356.7531 360.3518,-356.7531 356.7078,-360.1768 363.4331,-360.0327 353.0638,-363.6005 353.0638,-363.6005"/>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge15" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#add8e6" d="M342.2386,-357.7635C349.5562,-346.0182 358.5767,-332.2461 367.5223,-320.4 370.1199,-316.9602 372.9633,-313.4399 375.8469,-310.0124"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="382.5518,-302.2644 379.4108,-312.7708 379.2799,-306.0453 376.0081,-309.8261 376.0081,-309.8261 376.0081,-309.8261 379.2799,-306.0453 372.6053,-306.8815 382.5518,-302.2644 382.5518,-302.2644"/>
+            <text text-anchor="middle" x="372.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- tri&#45;&gt;tr -->
+        <g id="edge24" class="edge">
+            <title>tri&#45;&gt;tr</title>
+            <path fill="none" stroke="#ffc0cb" d="M314.1697,-412.7968C308.1382,-424.7883 300.0242,-438.3853 290.2369,-448.8313 280.9326,-458.7618 269.1419,-467.5728 257.7651,-474.8231"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="249.1939,-480.056 255.3841,-471.0044 253.4614,-477.4506 257.729,-474.8451 257.729,-474.8451 257.729,-474.8451 253.4614,-477.4506 260.0739,-478.6859 249.1939,-480.056 249.1939,-480.056"/>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge7" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#add8e6" d="M186.1766,-479.0539C173.479,-471.0083 159.2794,-460.5991 148.5223,-448.8313 141.1239,-440.7377 134.615,-430.8736 129.2283,-421.329"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="124.4153,-412.3441 133.1041,-419.0341 126.7763,-416.7516 129.1373,-421.159 129.1373,-421.159 129.1373,-421.159 126.7763,-416.7516 125.1706,-423.284 124.4153,-412.3441 124.4153,-412.3441"/>
+            <text text-anchor="middle" x="154.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge12" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#add8e6" d="M217.2369,-466.6249C217.2369,-453.7568 217.2369,-438.4867 217.2369,-424.6319"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="217.2369,-414.3761 221.737,-424.3761 217.2369,-419.3761 217.237,-424.3761 217.237,-424.3761 217.237,-424.3761 217.2369,-419.3761 212.737,-424.3762 217.2369,-414.3761 217.2369,-414.3761"/>
+            <text text-anchor="middle" x="222.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge14" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#add8e6" d="M240.0869,-472.8454C256.2553,-456.2866 278.1111,-433.9032 295.7628,-415.8254"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="303.0416,-408.3709 299.275,-418.6696 299.5485,-411.9483 296.0553,-415.5258 296.0553,-415.5258 296.0553,-415.5258 299.5485,-411.9483 292.8356,-412.382 303.0416,-408.3709 303.0416,-408.3709"/>
+            <text text-anchor="middle" x="282.9322" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> i</text>
+        </g>
+        <!-- tr&#45;&gt;t -->
+        <g id="edge25" class="edge">
+            <title>tr&#45;&gt;t</title>
+            <path fill="none" stroke="#ffc0cb" d="M224.5566,-525.3096C226.4734,-536.3112 227.6536,-548.9542 226.2369,-560.4626 225.9201,-563.0362 225.4804,-565.6905 224.9643,-568.3364"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="222.6937,-578.3117 220.5255,-567.5623 223.8035,-573.4364 224.9132,-568.5611 224.9132,-568.5611 224.9132,-568.5611 223.8035,-573.4364 229.301,-569.5599 222.6937,-578.3117 222.6937,-578.3117"/>
+        </g>
+        <!-- t&#45;&gt;root -->
+        <g id="edge26" class="edge">
+            <title>t&#45;&gt;root</title>
+            <path fill="none" stroke="#ffc0cb" d="M222.0886,-614.6473C223.3802,-620.3113 224.5942,-626.6075 225.2369,-632.4626 226.1247,-640.5504 225.87,-649.2014 225.048,-657.4497"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="223.7469,-667.6094 220.5537,-657.1188 224.3821,-662.6499 225.0173,-657.6904 225.0173,-657.6904 225.0173,-657.6904 224.3821,-662.6499 229.4808,-658.2621 223.7469,-667.6094 223.7469,-667.6094"/>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge6" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#add8e6" d="M217.2369,-578.0815C217.2369,-566.3502 217.2369,-550.6774 217.2369,-536.1885"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="217.2369,-525.9209 221.737,-535.9208 217.2369,-530.9209 217.237,-535.9209 217.237,-535.9209 217.237,-535.9209 217.2369,-530.9209 212.737,-535.9209 217.2369,-525.9209 217.2369,-525.9209"/>
+            <text text-anchor="middle" x="221.3172" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w3.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w3.svg
new file mode 100644
index 0000000000..2202ade193
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w3.svg
@@ -0,0 +1,269 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+# embedded DOT (plantuml) works in IDEA preview
+# but not on GitHub
+digraph G {
+    { rank=same root -> start [style=invis] }
+    start [label="start/end"]
+
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"]
+
+    tree [label = "contentArray[1]"]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    subgraph path {
+        edge [color = "lightblue"; fontcolor="blue"; arrowhead="vee"]
+
+        start -> root
+        root -> t [label = " t"]
+        t -> tr [label = " r"]
+        tr -> tra [label = " a"]
+        tra -> trac [label = " c"]
+        trac -> tract [label = " t"]
+        tract -> tracto [label = " o"]
+        tracto -> tractor [label = " r"]
+
+        tr -> tre [label = " e"]
+        tre -> tree [label = " e"]
+
+        tr -> tri [label = " i"]
+        tri -> trie [label = " e"]
+
+        // {rank=same tra -> tre -> tri [style=invis]}
+        {rank=same trac -> tree -> trie [style=invis]}
+    }
+
+    subgraph back {
+        edge [color = "pink"; fontcolor="blue"; constraint="false"; arrowhead="vee"]
+        tractor -> tr
+        tree -> tr
+        trie -> start
+    }
+}
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="480pt" height="734pt"
+     viewBox="0.00 0.00 480.47 734.09" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 730.0939)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-730.0939 476.4738,-730.0939 476.4738,4 -4,4"/>
+        <!-- root -->
+        <g id="node1" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-696.6782" rx="34.9213" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-700.8782" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="217.2369" y="-684.0782" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- start -->
+        <g id="node2" class="node">
+            <title>start</title>
+            <ellipse fill="none" stroke="#000000" cx="332.2369" cy="-696.6782" rx="44.0775" ry="18"/>
+            <text text-anchor="middle" x="332.2369" y="-692.4782" font-family="Times,serif" font-size="14.00" fill="#000000">start/end</text>
+        </g>
+        <!-- root&#45;&gt;start -->
+        <!-- t -->
+        <g id="node13" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-596.4626" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="217.2369" y="-592.2626" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge3" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#add8e6" d="M217.2369,-667.1522C217.2369,-653.6671 217.2369,-637.7749 217.2369,-624.4779"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="217.2369,-614.4659 221.737,-624.4659 217.2369,-619.4659 217.237,-624.4659 217.237,-624.4659 217.237,-624.4659 217.2369,-619.4659 212.737,-624.4659 217.2369,-614.4659 217.2369,-614.4659"/>
+            <text text-anchor="middle" x="220.9322" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> t</text>
+        </g>
+        <!-- start&#45;&gt;root -->
+        <g id="edge2" class="edge">
+            <title>start&#45;&gt;root</title>
+            <path fill="none" stroke="#add8e6" d="M307.4351,-711.6056C290.438,-719.0227 273.4409,-720.6565 256.4438,-716.507"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="246.4361,-713.3955 257.3213,-712.0674 251.2107,-714.88 255.9852,-716.3645 255.9852,-716.3645 255.9852,-716.3645 251.2107,-714.88 254.6492,-720.6616 246.4361,-713.3955 246.4361,-713.3955"/>
+        </g>
+        <!-- tractor -->
+        <g id="node3" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tr -->
+        <g id="node12" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-496.2469" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="217.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- tractor&#45;&gt;tr -->
+        <g id="edge16" class="edge">
+            <title>tractor&#45;&gt;tr</title>
+            <path fill="none" stroke="#ffc0cb" d="M57.5135,-35.7428C46.6571,-49.2423 33.1507,-68.8431 27.2369,-88.8 26.2429,-92.1544 24.6138,-277.4611 29.2369,-302.4 32.9143,-322.2376 56.637,-420.9537 90.2369,-448.8313 113.2626,-467.9355 144.7374,-479.7585 170.592,-486.8207"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="180.558,-489.3874 169.7517,-491.2511 175.716,-488.1403 170.874,-486.8933 170.874,-486.8933 170.874,-486.8933 175.716,-488.1403 171.9963,-482.5355 180.558,-489.3874 180.558,-489.3874"/>
+        </g>
+        <!-- tracto -->
+        <g id="node4" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge9" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#add8e6" d="M73.2369,-88.4006C73.2369,-76.2949 73.2369,-60.2076 73.2369,-46.4674"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="73.2369,-36.072 77.737,-46.072 73.2369,-41.072 73.237,-46.072 73.237,-46.072 73.237,-46.072 73.2369,-41.072 68.737,-46.0721 73.2369,-36.072 73.2369,-36.072"/>
+            <text text-anchor="middle" x="77.3172" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+        <!-- tract -->
+        <g id="node5" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-195.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge8" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#add8e6" d="M73.2369,-177.2006C73.2369,-165.0949 73.2369,-149.0076 73.2369,-135.2674"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="73.2369,-124.872 77.737,-134.872 73.2369,-129.872 73.237,-134.872 73.237,-134.872 73.237,-134.872 73.2369,-129.872 68.737,-134.8721 73.2369,-124.872 73.2369,-124.872"/>
+            <text text-anchor="middle" x="78.4869" y="-147" font-family="Times,serif" font-size="14.00" fill="#0000ff"> o</text>
+        </g>
+        <!-- trac -->
+        <g id="node6" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-284.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge7" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#add8e6" d="M73.2369,-266.0006C73.2369,-253.8949 73.2369,-237.8076 73.2369,-224.0674"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="73.2369,-213.672 77.737,-223.672 73.2369,-218.672 73.237,-223.672 73.237,-223.672 73.237,-223.672 73.2369,-218.672 68.737,-223.6721 73.2369,-213.672 73.2369,-213.672"/>
+            <text text-anchor="middle" x="76.9322" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#0000ff"> t</text>
+        </g>
+        <!-- tree -->
+        <g id="node8" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="217.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- trac&#45;&gt;tree -->
+        <!-- tra -->
+        <g id="node7" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#000000" cx="124.2369" cy="-384.6156" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="124.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="124.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x018</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge6" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#add8e6" d="M110.3009,-357.2313C103.0163,-342.9169 94.1342,-325.4635 86.8914,-311.2313"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="82.1788,-301.9709 90.7249,-308.8422 84.4465,-306.4271 86.7143,-310.8832 86.7143,-310.8832 86.7143,-310.8832 84.4465,-306.4271 82.7038,-312.9242 82.1788,-301.9709 82.1788,-301.9709"/>
+            <text text-anchor="middle" x="104.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> c</text>
+        </g>
+        <!-- trie -->
+        <g id="node10" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="399.2369" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="399.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tree&#45;&gt;trie -->
+        <!-- tree&#45;&gt;tr -->
+        <g id="edge17" class="edge">
+            <title>tree&#45;&gt;tr</title>
+            <path fill="none" stroke="#ffc0cb" d="M158.3327,-295.1942C129.7342,-301.5466 99.3858,-310.3083 89.2369,-320.4 77.8256,-331.747 81.481,-339.2647 79.2369,-355.2 75.5906,-381.0917 66.3187,-391.2981 79.2369,-414.0313 98.9655,-448.7494 139.7974,-470.3508 171.9533,-482.6472"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="181.6212,-486.1688 170.6849,-486.9744 176.9232,-484.4574 172.2251,-482.7461 172.2251,-482.7461 172.2251,-482.7461 176.9232,-484.4574 173.7653,-478.5179 181.6212,-486.1688 181.6212,-486.1688"/>
+        </g>
+        <!-- tre -->
+        <g id="node9" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="217.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge11" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#add8e6" d="M217.2369,-355.0897C217.2369,-341.6046 217.2369,-325.7123 217.2369,-312.4153"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="217.2369,-302.4033 221.737,-312.4033 217.2369,-307.4033 217.237,-312.4033 217.237,-312.4033 217.237,-312.4033 217.2369,-307.4033 212.737,-312.4034 217.2369,-302.4033 217.2369,-302.4033"/>
+            <text text-anchor="middle" x="222.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- trie&#45;&gt;start -->
+        <g id="edge18" class="edge">
+            <title>trie&#45;&gt;start</title>
+            <path fill="none" stroke="#ffc0cb" d="M398.5378,-302.5097C397.4092,-327.6379 394.7326,-374.4581 389.2369,-414.0313 376.1548,-508.2316 351.0965,-618.2848 339.0431,-668.779"/>
+            <polygon fill="#ffc0cb" stroke="#ffc0cb" points="336.6774,-678.6227 334.6388,-667.848 337.8458,-673.7612 339.0142,-668.8996 339.0142,-668.8996 339.0142,-668.8996 337.8458,-673.7612 343.3896,-669.9511 336.6774,-678.6227 336.6774,-678.6227"/>
+        </g>
+        <!-- tri -->
+        <g id="node11" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="342.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="342.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="342.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge13" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#add8e6" d="M351.1677,-355.794C355.3217,-344.3517 360.8359,-331.3387 367.5223,-320.4 369.7215,-316.8022 372.2911,-313.2194 374.9987,-309.7866"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="381.4573,-302.1119 378.4615,-312.6607 378.2379,-305.9375 375.0184,-309.7632 375.0184,-309.7632 375.0184,-309.7632 378.2379,-305.9375 371.5754,-306.8657 381.4573,-302.1119 381.4573,-302.1119"/>
+            <text text-anchor="middle" x="372.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge5" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#add8e6" d="M196.5972,-471.4723C183.2324,-455.43 165.6885,-434.3715 151.2227,-417.0077"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="144.8189,-409.321 154.6771,-414.1237 148.0193,-413.1625 151.2197,-417.0041 151.2197,-417.0041 151.2197,-417.0041 148.0193,-413.1625 147.7623,-419.8845 144.8189,-409.321 144.8189,-409.321"/>
+            <text text-anchor="middle" x="182.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge10" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#add8e6" d="M217.2369,-466.6249C217.2369,-453.7568 217.2369,-438.4867 217.2369,-424.6319"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="217.2369,-414.3761 221.737,-424.3761 217.2369,-419.3761 217.237,-424.3761 217.237,-424.3761 217.237,-424.3761 217.2369,-419.3761 212.737,-424.3762 217.2369,-414.3761 217.2369,-414.3761"/>
+            <text text-anchor="middle" x="222.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge12" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#add8e6" d="M241.0794,-473.3437C254.1439,-460.9437 270.7139,-445.4579 285.8463,-432.0313 293.0131,-425.6723 300.8138,-418.9751 308.2639,-412.682"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="316.1638,-406.0471 311.4004,-415.9244 312.335,-409.2628 308.5062,-412.4785 308.5062,-412.4785 308.5062,-412.4785 312.335,-409.2628 305.6121,-409.0326 316.1638,-406.0471 316.1638,-406.0471"/>
+            <text text-anchor="middle" x="288.9322" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> i</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge4" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#add8e6" d="M217.2369,-578.0815C217.2369,-566.3502 217.2369,-550.6774 217.2369,-536.1885"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="217.2369,-525.9209 221.737,-535.9208 217.2369,-530.9209 217.237,-535.9209 217.237,-535.9209 217.237,-535.9209 217.2369,-530.9209 212.737,-535.9209 217.2369,-525.9209 217.2369,-525.9209"/>
+            <text text-anchor="middle" x="221.3172" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> r</text>
+        </g>
+    </g>
+</svg>
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc1.svg
new file mode 100644
index 0000000000..ae595f3510
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc1.svg
@@ -0,0 +1,349 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  - Licensed to the Apache Software Foundation (ASF) under one
+  - or more contributor license agreements.  See the NOTICE file
+  - distributed with this work for additional information
+  - regarding copyright ownership.  The ASF licenses this file
+  - to you under the Apache License, Version 2.0 (the
+  - "License"); you may not use this file except in compliance
+  - with the License.  You may obtain a copy of the License at
+  -
+  -     http://www.apache.org/licenses/LICENSE-2.0
+  -
+  - Unless required by applicable law or agreed to in writing, software
+  - distributed under the License is distributed on an "AS IS" BASIS,
+  - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  - See the License for the specific language governing permissions and
+  - limitations under the License.
+  -->
+
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+digraph G {
+    { rank=same root -> start [style=invis] }
+    start [label="start/end"]
+
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"]
+
+    tree [label = "contentArray[1]"]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    root -> t [label = " t"]
+    t -> tr [label = " r"]
+    tr -> tra [label = " a"]
+    tra -> trac [label = " c"]
+    trac -> tract [label = " t"]
+    tract -> tracto [label = " o"]
+    tracto -> tractor [label = " r"]
+
+    tr -> tre [label = " e"]
+    tre -> tree [label = " e"]
+
+    tr -> tri [label = " i"]
+    tri -> trie [label = " e"]
+
+    // {rank=same tra -> tre -> tri [style=invis]}
+    {rank=same trac -> tree -> trie [style=invis]}
+
+   subgraph path {
+        edge [color = "lightblue"; fontcolor="blue"; arrowhead="vee"; constraint="false"]
+
+        start -> root [label = " 0, -1"]
+
+        root -> t [label = " 1, t"]
+        t -> tr [label = " 2, r"]
+        tr -> tra [label = " 3, a"]
+        tra -> trac [label = " 4, c"]
+        trac -> tract [label = " 5, t"]
+        tract -> tracto [label = " 6, o"]
+        tracto -> tractor [label = " 7, r"]
+
+        tractor -> tre [label = " 3, e"]
+
+        // tr -> tre [label = " e"]
+        tre -> tree [label = " 4, e"]
+
+        tree -> tri [label = "3, i"; ]
+
+        // tr -> tri [label = " i"]
+        tri -> trie [label = " 4, e"]
+
+        trie -> start [label = "-1, -1"]
+    }
+}
+
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="496pt" height="734pt"
+     viewBox="0.00 0.00 496.47 734.09" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 730.0939)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-730.0939 492.4673,-730.0939 492.4673,4 -4,4"/>
+        <!-- root -->
+        <g id="node1" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-696.6782" rx="34.9213" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-700.8782" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="217.2369" y="-684.0782" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- start -->
+        <g id="node2" class="node">
+            <title>start</title>
+            <ellipse fill="none" stroke="#000000" cx="343.2369" cy="-696.6782" rx="44.0775" ry="18"/>
+            <text text-anchor="middle" x="343.2369" y="-692.4782" font-family="Times,serif" font-size="14.00" fill="#000000">start/end</text>
+        </g>
+        <!-- root&#45;&gt;start -->
+        <!-- t -->
+        <g id="node13" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-596.4626" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="217.2369" y="-592.2626" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge2" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#000000" d="M217.2369,-667.1522C217.2369,-653.6671 217.2369,-637.7749 217.2369,-624.4779"/>
+            <polygon fill="#000000" stroke="#000000" points="220.737,-624.4659 217.2369,-614.4659 213.737,-624.4659 220.737,-624.4659"/>
+            <text text-anchor="middle" x="220.9322" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge16" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#add8e6" d="M226.1036,-667.9505C228.4851,-656.8764 229.9747,-644.0969 228.2369,-632.4626 227.8344,-629.7681 227.2713,-626.9965 226.6106,-624.2444"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="223.9024,-614.5348 230.9237,-622.9581 225.2458,-619.351 226.5891,-624.1672 226.5891,-624.1672 226.5891,-624.1672 225.2458,-619.351 222.2546,-625.3762 223.9024,-614.5348 223.9024,-614.5348"/>
+            <text text-anchor="middle" x="239.9322" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 1, t</text>
+        </g>
+        <!-- start&#45;&gt;root -->
+        <g id="edge15" class="edge">
+            <title>start&#45;&gt;root</title>
+            <path fill="none" stroke="#add8e6" d="M298.8722,-696.6782C287.0707,-696.6782 274.3106,-696.6782 262.4637,-696.6782"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="252.3349,-696.6782 262.335,-692.1783 257.3349,-696.6783 262.3349,-696.6783 262.3349,-696.6783 262.3349,-696.6783 257.3349,-696.6783 262.3349,-701.1783 252.3349,-696.6782 252.3349,-696.6782"/>
+            <text text-anchor="middle" x="275.5731" y="-703.8782" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 0, &#45;1</text>
+        </g>
+        <!-- tractor -->
+        <g id="node3" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tre -->
+        <g id="node9" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="217.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tractor&#45;&gt;tre -->
+        <g id="edge23" class="edge">
+            <title>tractor&#45;&gt;tre</title>
+            <path fill="none" stroke="#add8e6" d="M123.782,-31.2033C214.7179,-58.0052 403.6425,-128.3147 481.2369,-266.4 489.075,-280.3486 492.1649,-290.7134 481.2369,-302.4 440.4686,-345.9983 254.0052,-276.8017 213.2369,-320.4 206.7937,-327.2904 205.037,-336.6232 205.5759,-346.0371"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="206.9485,-356.1722 201.1471,-346.8666 206.2775,-351.2174 205.6064,-346.2627 205.6064,-346.2627 205.6064,-346.2627 206.2775,-351.2174 210.0657,-345.6587 206.9485,-356.1722 206.9485,-356.1722"/>
+            <text text-anchor="middle" x="453.0942" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 3, e</text>
+        </g>
+        <!-- tracto -->
+        <g id="node4" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-106.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge8" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-88.4006C73.2369,-76.2949 73.2369,-60.2076 73.2369,-46.4674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-46.072 73.2369,-36.072 69.737,-46.0721 76.737,-46.072"/>
+            <text text-anchor="middle" x="77.3172" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge22" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#add8e6" d="M78.6937,-88.6491C80.1467,-82.9864 81.5127,-76.6826 82.2369,-70.8 83.1492,-63.3893 83.1492,-61.4107 82.2369,-54 81.9201,-51.4264 81.4804,-48.7721 80.9643,-46.1262"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="78.6937,-36.1509 85.301,-44.9027 79.8035,-41.0262 80.9132,-45.9015 80.9132,-45.9015 80.9132,-45.9015 79.8035,-41.0262 76.5255,-46.9003 78.6937,-36.1509 78.6937,-36.1509"/>
+            <text text-anchor="middle" x="94.3172" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 7, r</text>
+        </g>
+        <!-- tract -->
+        <g id="node5" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-195.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge7" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-177.2006C73.2369,-165.0949 73.2369,-149.0076 73.2369,-135.2674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-134.872 73.2369,-124.872 69.737,-134.8721 76.737,-134.872"/>
+            <text text-anchor="middle" x="78.4869" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> o</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge21" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#add8e6" d="M80.5061,-177.5725C82.4432,-171.9148 84.266,-165.5828 85.2369,-159.6 86.4329,-152.2297 86.4329,-150.1703 85.2369,-142.8 84.7719,-139.9346 84.1115,-136.9891 83.3367,-134.0759"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="80.3796,-124.4595 87.6201,-132.6951 81.8492,-129.2387 83.3189,-134.0178 83.3189,-134.0178 83.3189,-134.0178 81.8492,-129.2387 79.0176,-135.3405 80.3796,-124.4595 80.3796,-124.4595"/>
+            <text text-anchor="middle" x="98.4869" y="-147" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 6, o</text>
+        </g>
+        <!-- trac -->
+        <g id="node6" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="73.2369" cy="-284.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="73.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge6" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#000000" d="M73.2369,-266.0006C73.2369,-253.8949 73.2369,-237.8076 73.2369,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="76.737,-223.672 73.2369,-213.672 69.737,-223.6721 76.737,-223.672"/>
+            <text text-anchor="middle" x="76.9322" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge20" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#add8e6" d="M79.7865,-266.6959C81.6052,-260.9434 83.3294,-254.4757 84.2369,-248.4 85.3399,-241.0153 85.3399,-238.9847 84.2369,-231.6 83.8344,-228.9055 83.2713,-226.1339 82.6106,-223.3819"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="79.9024,-213.6722 86.9237,-222.0956 81.2458,-218.4884 82.5891,-223.3046 82.5891,-223.3046 82.5891,-223.3046 81.2458,-218.4884 78.2546,-224.5136 79.9024,-213.6722 79.9024,-213.6722"/>
+            <text text-anchor="middle" x="95.9322" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 5, t</text>
+        </g>
+        <!-- tree -->
+        <g id="node8" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="217.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- trac&#45;&gt;tree -->
+        <!-- tra -->
+        <g id="node7" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#000000" cx="123.2369" cy="-384.6156" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="123.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="123.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x018</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge5" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#000000" d="M102.7638,-360.0748C97.5807,-353.0088 92.3929,-345.074 88.5223,-337.2 84.7334,-329.4922 81.7193,-320.678 79.4036,-312.4881"/>
+            <polygon fill="#000000" stroke="#000000" points="82.7327,-311.3819 76.8381,-302.5783 75.9561,-313.1364 82.7327,-311.3819"/>
+            <text text-anchor="middle" x="94.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> c</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge19" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#add8e6" d="M113.8009,-356.1612C109.6057,-344.7723 104.2515,-331.7001 98.2369,-320.4 96.3599,-316.8736 94.2013,-313.2795 91.9556,-309.7938"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="86.3046,-301.4527 95.6391,-307.2076 89.1091,-305.5922 91.9136,-309.7316 91.9136,-309.7316 91.9136,-309.7316 89.1091,-305.5922 88.1881,-312.2557 86.3046,-301.4527 86.3046,-301.4527"/>
+            <text text-anchor="middle" x="118.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 4, c</text>
+        </g>
+        <!-- trie -->
+        <g id="node10" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="399.2369" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="399.2369" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tree&#45;&gt;trie -->
+        <!-- tri -->
+        <g id="node11" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="333.2369" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="333.2369" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="333.2369" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- tree&#45;&gt;tri -->
+        <g id="edge25" class="edge">
+            <title>tree&#45;&gt;tri</title>
+            <path fill="none" stroke="#add8e6" d="M238.3195,-301.876C245.324,-307.7286 253.1441,-314.3129 260.2369,-320.4 273.6022,-331.8703 288.1999,-344.6585 300.805,-355.7846"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="308.3671,-362.4707 297.8947,-359.2181 304.6213,-359.1588 300.8755,-355.8469 300.8755,-355.8469 300.8755,-355.8469 304.6213,-359.1588 303.8562,-352.4756 308.3671,-362.4707 308.3671,-362.4707"/>
+            <text text-anchor="middle" x="288.1822" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff">3, i</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge10" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#000000" d="M217.2369,-355.0897C217.2369,-341.6046 217.2369,-325.7123 217.2369,-312.4153"/>
+            <polygon fill="#000000" stroke="#000000" points="220.737,-312.4033 217.2369,-302.4033 213.737,-312.4034 220.737,-312.4033"/>
+            <text text-anchor="middle" x="222.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge24" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#add8e6" d="M228.409,-356.2376C231.4877,-345.0943 233.446,-332.1755 231.2369,-320.4 230.7037,-317.5576 229.9528,-314.6513 229.0727,-311.7832"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="225.7116,-302.3279 233.3012,-310.243 227.3863,-307.0391 229.0611,-311.7503 229.0611,-311.7503 229.0611,-311.7503 227.3863,-307.0391 224.821,-313.2575 225.7116,-302.3279 225.7116,-302.3279"/>
+            <text text-anchor="middle" x="244.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 4, e</text>
+        </g>
+        <!-- trie&#45;&gt;start -->
+        <g id="edge27" class="edge">
+            <title>trie&#45;&gt;start</title>
+            <path fill="none" stroke="#add8e6" d="M405.2983,-302.5134C406.9127,-308.1745 408.4309,-314.487 409.2369,-320.4 410.2453,-327.7983 409.9138,-329.7641 409.2369,-337.2 397.8673,-462.0872 364.7169,-608.394 350.1917,-668.6161"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="347.7856,-678.5093 345.7763,-667.729 348.9672,-673.6509 350.1489,-668.7925 350.1489,-668.7925 350.1489,-668.7925 348.9672,-673.6509 354.5214,-669.856 347.7856,-678.5093 347.7856,-678.5093"/>
+            <text text-anchor="middle" x="406.3975" y="-492.0469" font-family="Times,serif" font-size="14.00" fill="#0000ff">&#45;1, &#45;1</text>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge12" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#000000" d="M343.5308,-356.2283C348.4068,-344.6002 354.8542,-331.3353 362.5223,-320.4 365.189,-316.597 368.2884,-312.8528 371.5344,-309.3005"/>
+            <polygon fill="#000000" stroke="#000000" points="374.2909,-311.4868 378.7636,-301.8824 369.2777,-306.6013 374.2909,-311.4868"/>
+            <text text-anchor="middle" x="367.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge26" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#add8e6" d="M354.9336,-360.2898C360.8766,-353.0969 367.0926,-345.0431 372.2369,-337.2 377.4883,-329.1936 382.4875,-320.0361 386.7247,-311.6206"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="391.1577,-302.5353 390.8167,-313.4959 388.9651,-307.0289 386.7725,-311.5225 386.7725,-311.5225 386.7725,-311.5225 388.9651,-307.0289 382.7282,-309.5492 391.1577,-302.5353 391.1577,-302.5353"/>
+            <text text-anchor="middle" x="393.0942" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 4, e</text>
+        </g>
+        <!-- tr -->
+        <g id="node12" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="217.2369" cy="-496.2469" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="217.2369" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="217.2369" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge4" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#000000" d="M182.6116,-484.1004C165.8889,-476.4961 147.0056,-465.018 135.5223,-448.8313 130.4241,-441.645 127.2882,-432.9125 125.3932,-424.2323"/>
+            <polygon fill="#000000" stroke="#000000" points="128.8093,-423.4413 123.6954,-414.1623 121.9067,-424.6051 128.8093,-423.4413"/>
+            <text text-anchor="middle" x="141.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge18" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#add8e6" d="M196.3752,-471.4723C182.7729,-455.3186 164.8877,-434.0788 150.2087,-416.6465"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="143.7168,-408.9369 153.6002,-413.6877 146.9374,-412.7616 150.158,-416.5862 150.158,-416.5862 150.158,-416.5862 146.9374,-412.7616 146.7158,-419.4848 143.7168,-408.9369 143.7168,-408.9369"/>
+            <text text-anchor="middle" x="188.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 3, a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge9" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#000000" d="M217.2369,-466.6249C217.2369,-453.7568 217.2369,-438.4867 217.2369,-424.6319"/>
+            <polygon fill="#000000" stroke="#000000" points="220.737,-424.3761 217.2369,-414.3761 213.737,-424.3762 220.737,-424.3761"/>
+            <text text-anchor="middle" x="222.0942" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge11" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#000000" d="M234.0049,-469.6873C242.4241,-457.4678 253.2833,-443.2328 264.8463,-432.0313 273.7641,-423.3922 284.4016,-415.2101 294.5991,-408.1516"/>
+            <polygon fill="#000000" stroke="#000000" points="296.6631,-410.9815 303.0143,-402.5013 292.761,-405.17 296.6631,-410.9815"/>
+            <text text-anchor="middle" x="268.9322" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> i</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge3" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#000000" d="M217.2369,-578.0815C217.2369,-566.3502 217.2369,-550.6774 217.2369,-536.1885"/>
+            <polygon fill="#000000" stroke="#000000" points="220.737,-535.9208 217.2369,-525.9209 213.737,-535.9209 220.737,-535.9208"/>
+            <text text-anchor="middle" x="221.3172" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge17" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#add8e6" d="M224.3796,-578.803C226.3635,-573.0529 228.2449,-566.5754 229.2369,-560.4626 230.5801,-552.1852 230.1465,-543.3325 228.8503,-534.9273"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="226.9029,-524.9655 233.2379,-533.9164 227.8622,-529.8726 228.8215,-534.7798 228.8215,-534.7798 228.8215,-534.7798 227.8622,-529.8726 224.4051,-535.6431 226.9029,-524.9655 226.9029,-524.9655"/>
+            <text text-anchor="middle" x="241.3172" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 2, r</text>
+        </g>
+    </g>
+</svg>
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc2.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc2.svg
new file mode 100644
index 0000000000..5f63519646
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc2.svg
@@ -0,0 +1,314 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  - Licensed to the Apache Software Foundation (ASF) under one
+  - or more contributor license agreements.  See the NOTICE file
+  - distributed with this work for additional information
+  - regarding copyright ownership.  The ASF licenses this file
+  - to you under the Apache License, Version 2.0 (the
+  - "License"); you may not use this file except in compliance
+  - with the License.  You may obtain a copy of the License at
+  -
+  -     http://www.apache.org/licenses/LICENSE-2.0
+  -
+  - Unless required by applicable law or agreed to in writing, software
+  - distributed under the License is distributed on an "AS IS" BASIS,
+  - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  - See the License for the specific language governing permissions and
+  - limitations under the License.
+  -->
+
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN"
+        "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!-- Generated by graphviz version 2.40.1 (20161225.0304)
+digraph G {
+    { rank=same root -> start [style=invis] }
+    start [label="start/end"]
+
+    tractor [label = "contentArray[0]"]
+    tracto [label = "0x01B"]
+    tract [label = "0x01A"]
+    trac [label = "0x019"]
+    tra [label = "Chain\n0x018"]
+
+    tree [label = "contentArray[1]"]
+    tre [label = "Chain\n0x03B"]
+    trie [label = "contentArray[2]"]
+    tri [label = "Chain\n0x05B"]
+
+    tr [label = "Sparse\n0x07E"]
+
+    t [label = "0x09B"]
+    root [label = "Chain\n0x9A"]
+
+    root -> t [label = " t"]
+    t -> tr [label = " r"]
+    tr -> tra [label = " a"]
+    tra -> trac [label = " c"]
+    trac -> tract [label = " t"]
+    tract -> tracto [label = " o"]
+    tracto -> tractor [label = " r"]
+
+    tr -> tre [label = " e"]
+    tre -> tree [label = " e"]
+
+    tr -> tri [label = " i"]
+    tri -> trie [label = " e"]
+
+    // {rank=same tra -> tre -> tri [style=invis]}
+    {rank=same trac -> tree -> trie [style=invis]}
+
+    subgraph path {
+        edge [color = "lightblue"; fontcolor="blue"; arrowhead="vee"; constraint="false"]
+
+        start -> root [label = " 0, -1"]
+
+        root -> tr [label = "t, 2, r "]
+        tr -> tra [label = "3, a"]
+        tra -> tractor [label = "cto, 7, r"]
+
+        // tr -> tre [label = " e"]
+        tre -> tree [label = " 4, e"]
+
+        // tr -> tri [label = " i"]
+        tri -> trie [label = " 4, e"]
+
+        tractor -> tre [label = " 3, e"]
+        tree -> tri [label = "3, i"; ]
+        trie -> start [label = "-1, -1"]
+    }
+}
+ -->
+<!-- Title: G Pages: 1 -->
+<svg width="515pt" height="734pt"
+     viewBox="0.00 0.00 514.97 734.09" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink">
+    <g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 730.0939)">
+        <title>G</title>
+        <polygon fill="#ffffff" stroke="transparent" points="-4,4 -4,-730.0939 510.9735,-730.0939 510.9735,4 -4,4"/>
+        <!-- root -->
+        <g id="node1" class="node">
+            <title>root</title>
+            <ellipse fill="none" stroke="#000000" cx="235.743" cy="-696.6782" rx="34.9213" ry="29.3315"/>
+            <text text-anchor="middle" x="235.743" y="-700.8782" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="235.743" y="-684.0782" font-family="Times,serif" font-size="14.00" fill="#000000">0x9A</text>
+        </g>
+        <!-- start -->
+        <g id="node2" class="node">
+            <title>start</title>
+            <ellipse fill="none" stroke="#000000" cx="361.743" cy="-696.6782" rx="44.0775" ry="18"/>
+            <text text-anchor="middle" x="361.743" y="-692.4782" font-family="Times,serif" font-size="14.00" fill="#000000">start/end</text>
+        </g>
+        <!-- root&#45;&gt;start -->
+        <!-- tr -->
+        <g id="node12" class="node">
+            <title>tr</title>
+            <ellipse fill="none" stroke="#000000" cx="235.743" cy="-496.2469" rx="37.9027" ry="29.3315"/>
+            <text text-anchor="middle" x="235.743" y="-500.4469" font-family="Times,serif" font-size="14.00" fill="#000000">Sparse</text>
+            <text text-anchor="middle" x="235.743" y="-483.6469" font-family="Times,serif" font-size="14.00" fill="#000000">0x07E</text>
+        </g>
+        <!-- root&#45;&gt;tr -->
+        <g id="edge16" class="edge">
+            <title>root&#45;&gt;tr</title>
+            <path fill="none" stroke="#add8e6" d="M208.9256,-677.804C190.37,-662.9875 167.1966,-640.5442 156.6918,-614.4626 142.7976,-579.9656 172.623,-545.1366 199.529,-522.3097"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="207.3667,-515.9125 202.4651,-525.7219 203.4931,-519.0741 199.6196,-522.2357 199.6196,-522.2357 199.6196,-522.2357 203.4931,-519.0741 196.7741,-518.7495 207.3667,-515.9125 207.3667,-515.9125"/>
+            <text text-anchor="middle" x="173.2686" y="-592.2626" font-family="Times,serif" font-size="14.00" fill="#0000ff">t, 2, r </text>
+        </g>
+        <!-- t -->
+        <g id="node13" class="node">
+            <title>t</title>
+            <ellipse fill="none" stroke="#000000" cx="235.743" cy="-596.4626" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="235.743" y="-592.2626" font-family="Times,serif" font-size="14.00" fill="#000000">0x09B</text>
+        </g>
+        <!-- root&#45;&gt;t -->
+        <g id="edge2" class="edge">
+            <title>root&#45;&gt;t</title>
+            <path fill="none" stroke="#000000" d="M235.743,-667.1522C235.743,-653.6671 235.743,-637.7749 235.743,-624.4779"/>
+            <polygon fill="#000000" stroke="#000000" points="239.2431,-624.4659 235.743,-614.4659 232.2431,-624.4659 239.2431,-624.4659"/>
+            <text text-anchor="middle" x="239.4383" y="-636.6626" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- start&#45;&gt;root -->
+        <g id="edge15" class="edge">
+            <title>start&#45;&gt;root</title>
+            <path fill="none" stroke="#add8e6" d="M317.3783,-696.6782C305.5769,-696.6782 292.8167,-696.6782 280.9698,-696.6782"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="270.841,-696.6782 280.8411,-692.1783 275.841,-696.6783 280.841,-696.6783 280.841,-696.6783 280.841,-696.6783 275.841,-696.6783 280.841,-701.1783 270.841,-696.6782 270.841,-696.6782"/>
+            <text text-anchor="middle" x="294.0792" y="-703.8782" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 0, &#45;1</text>
+        </g>
+        <!-- tractor -->
+        <g id="node3" class="node">
+            <title>tractor</title>
+            <ellipse fill="none" stroke="#000000" cx="91.743" cy="-18" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="91.743" y="-13.8" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[0]</text>
+        </g>
+        <!-- tre -->
+        <g id="node9" class="node">
+            <title>tre</title>
+            <ellipse fill="none" stroke="#000000" cx="235.743" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="235.743" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="235.743" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x03B</text>
+        </g>
+        <!-- tractor&#45;&gt;tre -->
+        <g id="edge21" class="edge">
+            <title>tractor&#45;&gt;tre</title>
+            <path fill="none" stroke="#add8e6" d="M142.2881,-31.2033C233.224,-58.0052 422.1486,-128.3147 499.743,-266.4 507.5812,-280.3486 510.6711,-290.7134 499.743,-302.4 458.9747,-345.9983 272.5113,-276.8017 231.743,-320.4 225.2999,-327.2904 223.5432,-336.6232 224.0821,-346.0371"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="225.4547,-356.1722 219.6532,-346.8666 224.7836,-351.2174 224.1125,-346.2627 224.1125,-346.2627 224.1125,-346.2627 224.7836,-351.2174 228.5718,-345.6587 225.4547,-356.1722 225.4547,-356.1722"/>
+            <text text-anchor="middle" x="471.6003" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 3, e</text>
+        </g>
+        <!-- tracto -->
+        <g id="node4" class="node">
+            <title>tracto</title>
+            <ellipse fill="none" stroke="#000000" cx="91.743" cy="-106.8" rx="36.5824" ry="18"/>
+            <text text-anchor="middle" x="91.743" y="-102.6" font-family="Times,serif" font-size="14.00" fill="#000000">0x01B</text>
+        </g>
+        <!-- tracto&#45;&gt;tractor -->
+        <g id="edge8" class="edge">
+            <title>tracto&#45;&gt;tractor</title>
+            <path fill="none" stroke="#000000" d="M91.743,-88.4006C91.743,-76.2949 91.743,-60.2076 91.743,-46.4674"/>
+            <polygon fill="#000000" stroke="#000000" points="95.2431,-46.072 91.743,-36.072 88.2431,-46.0721 95.2431,-46.072"/>
+            <text text-anchor="middle" x="95.8233" y="-58.2" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+        <!-- tract -->
+        <g id="node5" class="node">
+            <title>tract</title>
+            <ellipse fill="none" stroke="#000000" cx="91.743" cy="-195.6" rx="37.1443" ry="18"/>
+            <text text-anchor="middle" x="91.743" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#000000">0x01A</text>
+        </g>
+        <!-- tract&#45;&gt;tracto -->
+        <g id="edge7" class="edge">
+            <title>tract&#45;&gt;tracto</title>
+            <path fill="none" stroke="#000000" d="M91.743,-177.2006C91.743,-165.0949 91.743,-149.0076 91.743,-135.2674"/>
+            <polygon fill="#000000" stroke="#000000" points="95.2431,-134.872 91.743,-124.872 88.2431,-134.8721 95.2431,-134.872"/>
+            <text text-anchor="middle" x="96.993" y="-147" font-family="Times,serif" font-size="14.00" fill="#000000"> o</text>
+        </g>
+        <!-- trac -->
+        <g id="node6" class="node">
+            <title>trac</title>
+            <ellipse fill="none" stroke="#000000" cx="91.743" cy="-284.4" rx="35.3587" ry="18"/>
+            <text text-anchor="middle" x="91.743" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">0x019</text>
+        </g>
+        <!-- trac&#45;&gt;tract -->
+        <g id="edge6" class="edge">
+            <title>trac&#45;&gt;tract</title>
+            <path fill="none" stroke="#000000" d="M91.743,-266.0006C91.743,-253.8949 91.743,-237.8076 91.743,-224.0674"/>
+            <polygon fill="#000000" stroke="#000000" points="95.2431,-223.672 91.743,-213.672 88.2431,-223.6721 95.2431,-223.672"/>
+            <text text-anchor="middle" x="95.4383" y="-235.8" font-family="Times,serif" font-size="14.00" fill="#000000"> t</text>
+        </g>
+        <!-- tree -->
+        <g id="node8" class="node">
+            <title>tree</title>
+            <ellipse fill="none" stroke="#000000" cx="235.743" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="235.743" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[1]</text>
+        </g>
+        <!-- trac&#45;&gt;tree -->
+        <!-- tra -->
+        <g id="node7" class="node">
+            <title>tra</title>
+            <ellipse fill="none" stroke="#000000" cx="117.743" cy="-384.6156" rx="36.125" ry="29.3315"/>
+            <text text-anchor="middle" x="117.743" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="117.743" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x018</text>
+        </g>
+        <!-- tra&#45;&gt;tractor -->
+        <g id="edge18" class="edge">
+            <title>tra&#45;&gt;tractor</title>
+            <path fill="none" stroke="#add8e6" d="M94.3918,-361.848C65.2172,-331.3805 17.371,-274.1277 2.9772,-213.6 -12.3828,-149.0092 36.553,-79.6557 67.7845,-43.4033"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="74.5314,-35.7623 71.2857,-46.2369 71.2219,-39.5104 67.9125,-43.2584 67.9125,-43.2584 67.9125,-43.2584 71.2219,-39.5104 64.5393,-40.2799 74.5314,-35.7623 74.5314,-35.7623"/>
+            <text text-anchor="middle" x="25.1259" y="-191.4" font-family="Times,serif" font-size="14.00" fill="#0000ff">cto, 7, r</text>
+        </g>
+        <!-- tra&#45;&gt;trac -->
+        <g id="edge5" class="edge">
+            <title>tra&#45;&gt;trac</title>
+            <path fill="none" stroke="#000000" d="M110.2227,-355.6291C106.6613,-341.9016 102.4282,-325.5854 98.9126,-312.0348"/>
+            <polygon fill="#000000" stroke="#000000" points="102.2863,-311.1009 96.3871,-302.3003 95.5106,-312.8588 102.2863,-311.1009"/>
+            <text text-anchor="middle" x="109.6003" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> c</text>
+        </g>
+        <!-- trie -->
+        <g id="node10" class="node">
+            <title>trie</title>
+            <ellipse fill="none" stroke="#000000" cx="417.743" cy="-284.4" rx="73.4745" ry="18"/>
+            <text text-anchor="middle" x="417.743" y="-280.2" font-family="Times,serif" font-size="14.00" fill="#000000">contentArray[2]</text>
+        </g>
+        <!-- tree&#45;&gt;trie -->
+        <!-- tri -->
+        <g id="node11" class="node">
+            <title>tri</title>
+            <ellipse fill="none" stroke="#000000" cx="351.743" cy="-384.6156" rx="37.9306" ry="29.3315"/>
+            <text text-anchor="middle" x="351.743" y="-388.8156" font-family="Times,serif" font-size="14.00" fill="#000000">Chain</text>
+            <text text-anchor="middle" x="351.743" y="-372.0156" font-family="Times,serif" font-size="14.00" fill="#000000">0x05B</text>
+        </g>
+        <!-- tree&#45;&gt;tri -->
+        <g id="edge22" class="edge">
+            <title>tree&#45;&gt;tri</title>
+            <path fill="none" stroke="#add8e6" d="M256.2723,-301.9531C263.1097,-307.8109 270.7613,-314.3787 277.743,-320.4 291.1837,-331.9919 305.9615,-344.8025 318.7535,-355.9126"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="326.432,-362.5843 315.9318,-359.4223 322.6577,-359.3048 318.8834,-356.0254 318.8834,-356.0254 318.8834,-356.0254 322.6577,-359.3048 321.8349,-352.6285 326.432,-362.5843 326.432,-362.5843"/>
+            <text text-anchor="middle" x="304.6883" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff">3, i</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge10" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#000000" d="M235.743,-355.0897C235.743,-341.6046 235.743,-325.7123 235.743,-312.4153"/>
+            <polygon fill="#000000" stroke="#000000" points="239.2431,-312.4033 235.743,-302.4033 232.2431,-312.4034 239.2431,-312.4033"/>
+            <text text-anchor="middle" x="240.6003" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tre&#45;&gt;tree -->
+        <g id="edge19" class="edge">
+            <title>tre&#45;&gt;tree</title>
+            <path fill="none" stroke="#add8e6" d="M246.9152,-356.2376C249.9939,-345.0943 251.9521,-332.1755 249.743,-320.4 249.2098,-317.5576 248.4589,-314.6513 247.5789,-311.7832"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="244.2177,-302.3279 251.8073,-310.243 245.8925,-307.0391 247.5672,-311.7503 247.5672,-311.7503 247.5672,-311.7503 245.8925,-307.0391 243.3271,-313.2575 244.2177,-302.3279 244.2177,-302.3279"/>
+            <text text-anchor="middle" x="261.6003" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 4, e</text>
+        </g>
+        <!-- trie&#45;&gt;start -->
+        <g id="edge23" class="edge">
+            <title>trie&#45;&gt;start</title>
+            <path fill="none" stroke="#add8e6" d="M423.8044,-302.5134C425.4188,-308.1745 426.937,-314.487 427.743,-320.4 428.7515,-327.7983 428.42,-329.7641 427.743,-337.2 416.3734,-462.0872 383.223,-608.394 368.6978,-668.6161"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="366.2917,-678.5093 364.2825,-667.729 367.4734,-673.6509 368.655,-668.7925 368.655,-668.7925 368.655,-668.7925 367.4734,-673.6509 373.0275,-669.856 366.2917,-678.5093 366.2917,-678.5093"/>
+            <text text-anchor="middle" x="424.9036" y="-492.0469" font-family="Times,serif" font-size="14.00" fill="#0000ff">&#45;1, &#45;1</text>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge12" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#000000" d="M362.0369,-356.2283C366.9129,-344.6002 373.3603,-331.3353 381.0284,-320.4 383.6952,-316.597 386.7946,-312.8528 390.0406,-309.3005"/>
+            <polygon fill="#000000" stroke="#000000" points="392.7971,-311.4868 397.2698,-301.8824 387.7839,-306.6013 392.7971,-311.4868"/>
+            <text text-anchor="middle" x="385.6003" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tri&#45;&gt;trie -->
+        <g id="edge20" class="edge">
+            <title>tri&#45;&gt;trie</title>
+            <path fill="none" stroke="#add8e6" d="M373.4397,-360.2898C379.3828,-353.0969 385.5988,-345.0431 390.743,-337.2 395.9944,-329.1936 400.9936,-320.0361 405.2308,-311.6206"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="409.6638,-302.5353 409.3228,-313.4959 407.4712,-307.0289 405.2786,-311.5225 405.2786,-311.5225 405.2786,-311.5225 407.4712,-307.0289 401.2343,-309.5492 409.6638,-302.5353 409.6638,-302.5353"/>
+            <text text-anchor="middle" x="411.6003" y="-324.6" font-family="Times,serif" font-size="14.00" fill="#0000ff"> 4, e</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge4" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#000000" d="M201.7509,-482.8183C184.9175,-474.8752 165.2161,-463.4812 151.0284,-448.8313 143.6903,-441.2541 137.5841,-431.7356 132.6915,-422.3656"/>
+            <polygon fill="#000000" stroke="#000000" points="135.7196,-420.5888 128.2027,-413.1224 129.4228,-423.6468 135.7196,-420.5888"/>
+            <text text-anchor="middle" x="156.6003" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> a</text>
+        </g>
+        <!-- tr&#45;&gt;tra -->
+        <g id="edge17" class="edge">
+            <title>tr&#45;&gt;tra</title>
+            <path fill="none" stroke="#add8e6" d="M211.5792,-473.3873C193.4836,-456.2683 168.5764,-432.7055 148.9704,-414.1576"/>
+            <polygon fill="#add8e6" stroke="#add8e6" points="141.4713,-407.0632 151.8282,-410.6666 145.1035,-410.4994 148.7357,-413.9356 148.7357,-413.9356 148.7357,-413.9356 145.1035,-410.4994 145.6431,-417.2046 141.4713,-407.0632 141.4713,-407.0632"/>
+            <text text-anchor="middle" x="194.8503" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#0000ff">3, a</text>
+        </g>
+        <!-- tr&#45;&gt;tre -->
+        <g id="edge9" class="edge">
+            <title>tr&#45;&gt;tre</title>
+            <path fill="none" stroke="#000000" d="M235.743,-466.6249C235.743,-453.7568 235.743,-438.4867 235.743,-424.6319"/>
+            <polygon fill="#000000" stroke="#000000" points="239.2431,-424.3761 235.743,-414.3761 232.2431,-424.3762 239.2431,-424.3761"/>
+            <text text-anchor="middle" x="240.6003" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> e</text>
+        </g>
+        <!-- tr&#45;&gt;tri -->
+        <g id="edge11" class="edge">
+            <title>tr&#45;&gt;tri</title>
+            <path fill="none" stroke="#000000" d="M259.4973,-473.3873C277.0521,-456.4936 301.128,-433.3244 320.2814,-414.8924"/>
+            <polygon fill="#000000" stroke="#000000" points="322.8434,-417.2844 327.6219,-407.8283 317.9895,-412.2405 322.8434,-417.2844"/>
+            <text text-anchor="middle" x="303.4383" y="-436.2313" font-family="Times,serif" font-size="14.00" fill="#000000"> i</text>
+        </g>
+        <!-- t&#45;&gt;tr -->
+        <g id="edge3" class="edge">
+            <title>t&#45;&gt;tr</title>
+            <path fill="none" stroke="#000000" d="M235.743,-578.0815C235.743,-566.3502 235.743,-550.6774 235.743,-536.1885"/>
+            <polygon fill="#000000" stroke="#000000" points="239.2431,-535.9208 235.743,-525.9209 232.2431,-535.9209 239.2431,-535.9208"/>
+            <text text-anchor="middle" x="239.8233" y="-547.8626" font-family="Times,serif" font-size="14.00" fill="#000000"> r</text>
+        </g>
+    </g>
+</svg>
diff --git a/src/java/org/apache/cassandra/db/tries/MergeTrie.java b/src/java/org/apache/cassandra/db/tries/MergeTrie.java
new file mode 100644
index 0000000000..80abb5ed69
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/MergeTrie.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * A merged view of two tries.
+ *
+ * This is accomplished by walking the two cursors in parallel; the merged cursor takes the position and features of the
+ * smaller and advances with it; when the two cursors are equal, both are advanced.
+ *
+ * Crucial for the efficiency of this is the fact that when they are advanced like this, we can compare cursors'
+ * positions by their depth descending and then incomingTransition ascending.
+ *
+ * See Trie.md for further details.
+ */
+class MergeTrie<T> extends Trie<T>
+{
+    private final MergeResolver<T> resolver;
+    protected final Trie<T> t1;
+    protected final Trie<T> t2;
+
+    MergeTrie(MergeResolver<T> resolver, Trie<T> t1, Trie<T> t2)
+    {
+        this.resolver = resolver;
+        this.t1 = t1;
+        this.t2 = t2;
+    }
+
+    @Override
+    protected Cursor<T> cursor()
+    {
+        return new MergeCursor<>(resolver, t1, t2);
+    }
+
+    static class MergeCursor<T> implements Cursor<T>
+    {
+        private final MergeResolver<T> resolver;
+        private final Cursor<T> c1;
+        private final Cursor<T> c2;
+
+        boolean atC1;
+        boolean atC2;
+
+        MergeCursor(MergeResolver<T> resolver, Trie<T> t1, Trie<T> t2)
+        {
+            this.resolver = resolver;
+            this.c1 = t1.cursor();
+            this.c2 = t2.cursor();
+            assert c1.depth() == 0;
+            assert c2.depth() == 0;
+            atC1 = atC2 = true;
+        }
+
+        @Override
+        public int advance()
+        {
+            return checkOrder(atC1 ? c1.advance() : c1.depth(),
+                              atC2 ? c2.advance() : c2.depth());
+        }
+
+        @Override
+        public int skipChildren()
+        {
+            return checkOrder(atC1 ? c1.skipChildren() : c1.depth(),
+                              atC2 ? c2.skipChildren() : c2.depth());
+        }
+
+        @Override
+        public int advanceMultiple(TransitionsReceiver receiver)
+        {
+            // While we are on a shared position, we must descend one byte at a time to maintain the cursor ordering.
+            if (atC1 && atC2)
+                return checkOrder(c1.advance(), c2.advance());
+
+            // If we are in a branch that's only covered by one of the sources, we can use its advanceMultiple as it is
+            // only different from advance if it takes multiple steps down, which does not change the order of the
+            // cursors.
+            // Since it might ascend, we still have to check the order after the call.
+            if (atC1)
+                return checkOrder(c1.advanceMultiple(receiver), c2.depth());
+            else // atC2
+                return checkOrder(c1.depth(), c2.advanceMultiple(receiver));
+        }
+
+        private int checkOrder(int c1depth, int c2depth)
+        {
+            if (c1depth > c2depth)
+            {
+                atC1 = true;
+                atC2 = false;
+                return c1depth;
+            }
+            if (c1depth < c2depth)
+            {
+                atC1 = false;
+                atC2 = true;
+                return c2depth;
+            }
+            // c1depth == c2depth
+            int c1trans = c1.incomingTransition();
+            int c2trans = c2.incomingTransition();
+            atC1 = c1trans <= c2trans;
+            atC2 = c1trans >= c2trans;
+            assert atC1 | atC2;
+            return c1depth;
+        }
+
+        @Override
+        public int depth()
+        {
+            return atC1 ? c1.depth() : c2.depth();
+        }
+
+        @Override
+        public int incomingTransition()
+        {
+            return atC1 ? c1.incomingTransition() : c2.incomingTransition();
+        }
+
+        public T content()
+        {
+            T mc = atC2 ? c2.content() : null;
+            T nc = atC1 ? c1.content() : null;
+            if (mc == null)
+                return nc;
+            else if (nc == null)
+                return mc;
+            else
+                return resolver.resolve(nc, mc);
+        }
+    }
+
+    /**
+     * Special instance for sources that are guaranteed (by the caller) distinct. The main difference is that we can
+     * form unordered value list by concatenating sources.
+     */
+    static class Distinct<T> extends MergeTrie<T>
+    {
+        Distinct(Trie<T> input1, Trie<T> input2)
+        {
+            super(throwingResolver(), input1, input2);
+        }
+
+        @Override
+        public Iterable<T> valuesUnordered()
+        {
+            return Iterables.concat(t1.valuesUnordered(), t2.valuesUnordered());
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/SingletonTrie.java b/src/java/org/apache/cassandra/db/tries/SingletonTrie.java
new file mode 100644
index 0000000000..0336a851ff
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/SingletonTrie.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Singleton trie, mapping the given key to value.
+ */
+class SingletonTrie<T> extends Trie<T>
+{
+    private final ByteComparable key;
+    private final T value;
+
+    SingletonTrie(ByteComparable key, T value)
+    {
+        this.key = key;
+        this.value = value;
+    }
+
+    public Cursor cursor()
+    {
+        return new Cursor();
+    }
+
+    class Cursor implements Trie.Cursor<T>
+    {
+        private final ByteSource src = key.asComparableBytes(BYTE_COMPARABLE_VERSION);
+        private int currentDepth = 0;
+        private int currentTransition = -1;
+        private int nextTransition = src.next();
+
+        @Override
+        public int advance()
+        {
+            currentTransition = nextTransition;
+            if (currentTransition != ByteSource.END_OF_STREAM)
+            {
+                nextTransition = src.next();
+                return ++currentDepth;
+            }
+            else
+            {
+                return currentDepth = -1;
+            }
+        }
+
+        @Override
+        public int advanceMultiple(TransitionsReceiver receiver)
+        {
+            if (nextTransition == ByteSource.END_OF_STREAM)
+                return currentDepth = -1;
+            int current = nextTransition;
+            int depth = currentDepth;
+            int next = src.next();
+            while (next != ByteSource.END_OF_STREAM)
+            {
+                if (receiver != null)
+                    receiver.addPathByte(current);
+                current = next;
+                next = src.next();
+                ++depth;
+            }
+            currentTransition = current;
+            nextTransition = next;
+            return currentDepth = ++depth;
+        }
+
+        @Override
+        public int skipChildren()
+        {
+            return currentDepth = -1;  // no alternatives
+        }
+
+        @Override
+        public int depth()
+        {
+            return currentDepth;
+        }
+
+        @Override
+        public T content()
+        {
+            return nextTransition == ByteSource.END_OF_STREAM ? value : null;
+        }
+
+        @Override
+        public int incomingTransition()
+        {
+            return currentTransition;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/SlicedTrie.java b/src/java/org/apache/cassandra/db/tries/SlicedTrie.java
new file mode 100644
index 0000000000..75ae3df27e
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/SlicedTrie.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.tries;
+
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Represents a sliced view of a trie, i.e. the content within the given pair of bounds.
+ *
+ * Applied by advancing three tries in parallel: the left bound, the source and the right bound. While the source
+ * bound is smallest, we don't issue any content and skip over any children. As soon as the left bound becomes strictly
+ * smaller, we stop processing it (as it's a singleton trie it will remain smaller until it's exhausted) and start
+ * issuing the nodes and content from the source. As soon as the right bound becomes strictly smaller, we finish the
+ * walk.
+ *
+ * We don't explicitly construct tries for the two bounds; tracking the current depth (= prefix length) and transition
+ * as characters are requested from the key is sufficient as it is a trie with just a single descent path. Because we
+ * need the next character to tell if it's been exhausted, we keep these one position ahead. The source is always
+ * advanced, thus this gives us the thing to compare it against after the advance.
+ *
+ * We also track the current state to make some decisions a little simpler.
+ *
+ * See Trie.md for further details.
+ */
+public class SlicedTrie<T> extends Trie<T>
+{
+    private final Trie<T> source;
+
+    /** Left-side boundary. The characters of this are requested as we descend along the left-side boundary. */
+    private final ByteComparable left;
+
+    /** Right-side boundary. The characters of this are requested as we descend along the right-side boundary. */
+    private final ByteComparable right;
+
+    private final boolean includeLeft;
+    private final boolean includeRight;
+
+    public SlicedTrie(Trie<T> source, ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight)
+    {
+        this.source = source;
+        this.left = left;
+        this.right = right;
+        this.includeLeft = includeLeft;
+        this.includeRight = includeRight;
+    }
+
+    @Override
+    protected Cursor<T> cursor()
+    {
+        return new SlicedCursor<>(this);
+    }
+
+    private enum State
+    {
+        /** The cursor is still positioned on some prefix of the left bound. Content should not be produced. */
+        BEFORE_LEFT,
+        /** The cursor is positioned inside the range, i.e. beyond the left bound, possibly on a prefix of the right. */
+        INSIDE,
+        /** The cursor is positioned beyond the right bound. Exhaustion (depth -1) has been reported. */
+        AFTER_RIGHT
+    }
+
+    private static class SlicedCursor<T> implements Cursor<T>
+    {
+        private final ByteSource left;
+        private final ByteSource right;
+        private final boolean includeLeft;
+        private final boolean excludeRight;
+        private final Cursor<T> source;
+
+        private State state;
+        private int leftNext;
+        private int leftNextDepth;
+        private int rightNext;
+        private int rightNextDepth;
+
+        public SlicedCursor(SlicedTrie<T> slicedTrie)
+        {
+            source = slicedTrie.source.cursor();
+            if (slicedTrie.left != null)
+            {
+                left = slicedTrie.left.asComparableBytes(BYTE_COMPARABLE_VERSION);
+                includeLeft = slicedTrie.includeLeft;
+                leftNext = left.next();
+                leftNextDepth = 1;
+                if (leftNext == ByteSource.END_OF_STREAM && includeLeft)
+                    state = State.INSIDE;
+                else
+                    state = State.BEFORE_LEFT;
+            }
+            else
+            {
+                left = null;
+                includeLeft = true;
+                state = State.INSIDE;
+            }
+
+            if (slicedTrie.right != null)
+            {
+                right = slicedTrie.right.asComparableBytes(BYTE_COMPARABLE_VERSION);
+                excludeRight = !slicedTrie.includeRight;
+                rightNext = right.next();
+                rightNextDepth = 1;
+                if (rightNext == ByteSource.END_OF_STREAM && excludeRight)
+                    state = State.BEFORE_LEFT;  // This is a hack, we are after the right bound but we don't want to
+                                                // report depth -1 yet. So just make sure root's content is not reported.
+            }
+            else
+            {
+                right = null;
+                excludeRight = true;
+                rightNextDepth = 0;
+            }
+        }
+
+        @Override
+        public int advance()
+        {
+            assert (state != State.AFTER_RIGHT);
+
+            int newDepth = source.advance();
+            int transition = source.incomingTransition();
+
+            if (state == State.BEFORE_LEFT)
+            {
+                // Skip any transitions before the left bound
+                while (newDepth == leftNextDepth && transition < leftNext)
+                {
+                    newDepth = source.skipChildren();
+                    transition = source.incomingTransition();
+                }
+
+                // Check if we are still following the left bound
+                if (newDepth == leftNextDepth && transition == leftNext)
+                {
+                    assert leftNext != ByteSource.END_OF_STREAM;
+                    leftNext = left.next();
+                    ++leftNextDepth;
+                    if (leftNext == ByteSource.END_OF_STREAM && includeLeft)
+                        state = State.INSIDE; // report the content on the left bound
+                }
+                else // otherwise we are beyond it
+                    state = State.INSIDE;
+            }
+
+            return checkRightBound(newDepth, transition);
+        }
+
+        private int markDone()
+        {
+            state = State.AFTER_RIGHT;
+            return -1;
+        }
+
+        private int checkRightBound(int newDepth, int transition)
+        {
+            // Cursor positions compare by depth descending and transition ascending.
+            if (newDepth > rightNextDepth)
+                return newDepth;
+            if (newDepth < rightNextDepth)
+                return markDone();
+            // newDepth == rightDepth
+            if (transition < rightNext)
+                return newDepth;
+            if (transition > rightNext)
+                return markDone();
+
+            // Following right bound
+            rightNext = right.next();
+            ++rightNextDepth;
+            if (rightNext == ByteSource.END_OF_STREAM && excludeRight)
+                return markDone();  // do not report any content on the right bound
+            return newDepth;
+        }
+
+        @Override
+        public int advanceMultiple(TransitionsReceiver receiver)
+        {
+            switch (state)
+            {
+                case BEFORE_LEFT:
+                    return advance();   // descend only one level to be able to compare cursors correctly
+                case INSIDE:
+                    int depth = source.depth();
+                    if (depth == rightNextDepth - 1)  // this is possible because right is already advanced;
+                        return advance();   // we need to check next byte against right boundary in this case
+                    int newDepth = source.advanceMultiple(receiver);
+                    if (newDepth > depth)
+                        return newDepth;    // successfully advanced
+                    // we ascended, check if we are still within boundaries
+                    return checkRightBound(newDepth, source.incomingTransition());
+                default:
+                    throw new AssertionError();
+            }
+        }
+
+        @Override
+        public int skipChildren()
+        {
+            assert (state != State.AFTER_RIGHT);
+
+            // We are either inside or following the left bound. In the latter case ascend takes us beyond it.
+            state = State.INSIDE;
+            return checkRightBound(source.skipChildren(), source.incomingTransition());
+        }
+
+        @Override
+        public int depth()
+        {
+            return state == State.AFTER_RIGHT ? -1 : source.depth();
+        }
+
+        @Override
+        public int incomingTransition()
+        {
+            return source.incomingTransition();
+        }
+
+        @Override
+        public T content()
+        {
+            return state == State.INSIDE ? source.content() : null;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java
new file mode 100644
index 0000000000..ebe912df36
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/Trie.java
@@ -0,0 +1,622 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import com.google.common.collect.ImmutableList;
+
+import org.agrona.DirectBuffer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Base class for tries.
+ *
+ * Normal users of tries will only use the public methods, which provide various transformations of the trie, conversion
+ * of its content to other formats (e.g. iterable of values), and several forms of processing.
+ *
+ * For any unimplemented data extraction operations one can build on the {@link TrieEntriesWalker} (for-each processing)
+ * and {@link TrieEntriesIterator} (to iterator) base classes, which provide the necessary mechanisms to handle walking
+ * the trie.
+ *
+ * The internal representation of tries using this interface is defined in the {@link Cursor} interface.
+ *
+ * Cursors are a method of presenting the internal structure of a trie without representing nodes as objects, which is
+ * still useful for performing the basic operations on tries (iteration, slicing/intersection and merging). A cursor
+ * will list the nodes of a trie in order, together with information about the path that was taken to reach them.
+ *
+ * To begin traversal over a trie, one must retrieve a cursor by calling {@link #cursor()}. Because cursors are
+ * stateful, the traversal must always proceed from one thread. Should concurrent reads be required, separate calls to
+ * {@link #cursor()} must be made. Any modification that has completed before the construction of a cursor must be
+ * visible, but any later concurrent modifications may be presented fully, partially or not at all; this also means that
+ * if multiple are made, the cursor may see any part of any subset of them.
+ *
+ * Note: This model only supports depth-first traversals. We do not currently have a need for breadth-first walks.
+ *
+ * See Trie.md for further description of the trie representation model.
+ *
+ * @param <T> The content type of the trie.
+ */
+public abstract class Trie<T>
+{
+    /**
+     * A trie cursor.
+     *
+     * This is the internal representation of the trie, which enables efficient walks and basic operations (merge,
+     * slice) on tries.
+     *
+     * The cursor represents the state of a walk over the nodes of trie. It provides three main features:
+     * - the current "depth" or descend-depth in the trie;
+     * - the "incomingTransition", i.e. the byte that was used to reach the current point;
+     * - the "content" associated with the current node,
+     * and provides methods for advancing to the next position.  This is enough information to extract all paths, and
+     * also to easily compare cursors over different tries that are advanced together. Advancing is always done in
+     * order; if one imagines the set of nodes in the trie with their associated paths, a cursor may only advance from a
+     * node with a lexicographically smaller path to one with bigger. The "advance" operation moves to the immediate
+     * next, it is also possible to skip over some items e.g. all children of the current node ("skipChildren").
+     *
+     * Moving to the immediate next position in the lexicographic order is accomplished by:
+     * - if the current node has children, moving to its first child;
+     * - otherwise, ascend the parent chain and return the next child of the closest parent that still has any.
+     * As long as the trie is not exhausted, advancing always takes one step down, from the current node, or from a node
+     * on the parent chain. By comparing the new depth (which "advance" also returns) with the one before the advance,
+     * one can tell if the former was the case (if newDepth == oldDepth + 1) and how many steps up we had to take
+     * (oldDepth + 1 - newDepth). When following a path down, the cursor will stop on all prefixes.
+     *
+     * When it is created the cursor is placed on the root node with depth() = 0, incomingTransition() = -1. Since
+     * tries can have mappings for empty, content() can possibly be non-null. It is not allowed for a cursor to start
+     * in exhausted state (i.e. with depth() = -1).
+     *
+     * For example, the following trie:
+     *  t
+     *   r
+     *    e
+     *     e *
+     *    i
+     *     e *
+     *     p *
+     *  w
+     *   i
+     *    n  *
+     * has nodes reachable with the paths
+     *  "", t, tr, tre, tree*, tri, trie*, trip*, w, wi, win*
+     * and the cursor will list them with the following (depth, incomingTransition) pairs:
+     *  (0, -1), (1, t), (2, r), (3, e), (4, e)*, (3, i), (4, e)*, (4, p)*, (1, w), (2, i), (3, n)*
+     *
+     * Because we exhaust transitions on bigger depths before we go the next transition on the smaller ones, when
+     * cursors are advanced together their positions can be easily compared using only the depth and incomingTransition:
+     * - one that is higher in depth is before one that is lower;
+     * - for equal depths, the one with smaller incomingTransition is first.
+     *
+     * If we consider walking the trie above in parallel with this:
+     *  t
+     *   r
+     *    i
+     *     c
+     *      k *
+     *  u
+     *   p *
+     * the combined iteration will proceed as follows:
+     *  (0, -1)+    (0, -1)+               cursors equal, advance both
+     *  (1, t)+     (1, t)+        t       cursors equal, advance both
+     *  (2, r)+     (2, r)+        tr      cursors equal, advance both
+     *  (3, e)+  <  (3, i)         tre     cursors not equal, advance smaller (3 = 3, e < i)
+     *  (4, e)+  <  (3, i)         tree*   cursors not equal, advance smaller (4 > 3)
+     *  (3, i)+     (3, i)+        tri     cursors equal, advance both
+     *  (4, e)   >  (4, c)+        tric    cursors not equal, advance smaller (4 = 4, e > c)
+     *  (4, e)   >  (5, k)+        trick*  cursors not equal, advance smaller (4 < 5)
+     *  (4, e)+  <  (1, u)         trie*   cursors not equal, advance smaller (4 > 1)
+     *  (4, p)+  <  (1, u)         trip*   cursors not equal, advance smaller (4 > 1)
+     *  (1, w)   >  (1, u)+        u       cursors not equal, advance smaller (1 = 1, w > u)
+     *  (1, w)   >  (2, p)+        up*     cursors not equal, advance smaller (1 < 2)
+     *  (1, w)+  <  (-1, -1)       w       cursors not equal, advance smaller (1 > -1)
+     *  (2, i)+  <  (-1, -1)       wi      cursors not equal, advance smaller (2 > -1)
+     *  (3, n)+  <  (-1, -1)       win*    cursors not equal, advance smaller (3 > -1)
+     *  (-1, -1)    (-1, -1)               both exhasted
+     */
+    protected interface Cursor<T>
+    {
+
+        /**
+         * @return the current descend-depth; 0, if the cursor has just been created and is positioned on the root,
+         *         and -1, if the trie has been exhausted.
+         */
+        int depth();
+
+        /**
+         * @return the last transition taken; if positioned on the root, return -1
+         */
+        int incomingTransition();
+
+        /**
+         * @return the content associated with the current node. This may be non-null for any presented node, including
+         *         the root.
+         */
+        T content();
+
+        /**
+         * Advance one position to the node whose associated path is next lexicographically.
+         * This can be either:
+         * - descending one level to the first child of the current node,
+         * - ascending to the closest parent that has remaining children, and then descending one level to its next
+         *   child.
+         *
+         * It is an error to call this after the trie has already been exhausted (i.e. when depth() == -1);
+         * for performance reasons we won't always check this.
+         *
+         * @return depth (can be prev+1 or <=prev), -1 means that the trie is exhausted
+         */
+        int advance();
+
+        /**
+         * Advance, descending multiple levels if the cursor can do this for the current position without extra work
+         * (e.g. when positioned on a chain node in a memtable trie). If the current node does not have children this
+         * is exactly the same as advance(), otherwise it may take multiple steps down (but will not necessarily, even
+         * if they exist).
+         *
+         * Note that if any positions are skipped, their content must be null.
+         *
+         * This is an optional optimization; the default implementation falls back to calling advance.
+         *
+         * It is an error to call this after the trie has already been exhausted (i.e. when depth() == -1);
+         * for performance reasons we won't always check this.
+         *
+         * @param receiver object that will receive all transitions taken except the last;
+         *                 on ascend, or if only one step down was taken, it will not receive any
+         * @return the new depth, -1 if the trie is exhausted
+         */
+        default int advanceMultiple(TransitionsReceiver receiver)
+        {
+            return advance();
+        }
+
+        /**
+         * Advance all the way to the next node with non-null content.
+         *
+         * It is an error to call this after the trie has already been exhausted (i.e. when depth() == -1);
+         * for performance reasons we won't always check this.
+         *
+         * @param receiver object that will receive all taken transitions
+         * @return the content, null if the trie is exhausted
+         */
+        default T advanceToContent(ResettingTransitionsReceiver receiver)
+        {
+            int prevDepth = depth();
+            while (true)
+            {
+                int currDepth = advanceMultiple(receiver);
+                if (currDepth <= 0)
+                    return null;
+                if (receiver != null)
+                {
+                    if (currDepth <= prevDepth)
+                        receiver.resetPathLength(currDepth - 1);
+                    receiver.addPathByte(incomingTransition());
+                }
+                T content = content();
+                if (content != null)
+                    return content;
+                prevDepth = currDepth;
+            }
+        }
+
+        /**
+         * Ignore the current node's children and advance to the next child of the closest node on the parent chain that
+         * has any.
+         *
+         * It is an error to call this after the trie has already been exhausted (i.e. when depth() == -1);
+         * for performance reasons we won't always check this.
+         *
+         * @return the new depth, always <= previous depth; -1 if the trie is exhausted
+         */
+        int skipChildren();
+    }
+
+    protected abstract Cursor<T> cursor();
+
+    /**
+     * Used by {@link Cursor#advanceMultiple} to feed the transitions taken.
+     */
+    protected interface TransitionsReceiver
+    {
+        /** Add a single byte to the path. */
+        void addPathByte(int nextByte);
+        /** Add the count bytes from position pos in the given buffer. */
+        void addPathBytes(DirectBuffer buffer, int pos, int count);
+    }
+
+    /**
+     * Used by {@link Cursor#advanceToContent} to track the transitions and backtracking taken.
+     */
+    protected interface ResettingTransitionsReceiver extends TransitionsReceiver
+    {
+        /** Delete all bytes beyond the given length. */
+        void resetPathLength(int newLength);
+    }
+
+    /**
+     * A push interface for walking over the trie. Builds upon TransitionsReceiver to be given the bytes of the
+     * path, and adds methods called on encountering content and completion.
+     * See {@link TrieDumper} for an example of how this can be used, and {@link TrieEntriesWalker} as a base class
+     * for other common usages.
+     */
+    protected interface Walker<T, R> extends ResettingTransitionsReceiver
+    {
+        /** Called when content is found. */
+        void content(T content);
+
+        /** Called at the completion of the walk. */
+        R complete();
+    }
+
+    // Version of the byte comparable conversion to use for all operations
+    protected static final ByteComparable.Version BYTE_COMPARABLE_VERSION = ByteComparable.Version.OSS42;
+
+    /**
+     * Adapter interface providing the methods a {@link Walker} to a {@link Consumer}, so that the latter can be used
+     * with {@link #process}.
+     *
+     * This enables calls like
+     *     trie.forEachEntry(x -> System.out.println(x));
+     * to be mapped directly to a single call to {@link #process} without extra allocations.
+     */
+    public interface ValueConsumer<T> extends Consumer<T>, Walker<T, Void>
+    {
+        @Override
+        default void content(T content)
+        {
+            accept(content);
+        }
+
+        @Override
+        default Void complete()
+        {
+            return null;
+        }
+
+        @Override
+        default void resetPathLength(int newDepth)
+        {
+            // not tracking path
+        }
+
+        @Override
+        default void addPathByte(int nextByte)
+        {
+            // not tracking path
+        }
+
+        @Override
+        default void addPathBytes(DirectBuffer buffer, int pos, int count)
+        {
+            // not tracking path
+        }
+    }
+
+    /**
+     * Call the given consumer on all content values in the trie in order.
+     */
+    public void forEachValue(ValueConsumer<T> consumer)
+    {
+        process(consumer);
+    }
+
+    /**
+     * Call the given consumer on all (path, content) pairs with non-null content in the trie in order.
+     */
+    public void forEachEntry(BiConsumer<ByteComparable, T> consumer)
+    {
+        process(new TrieEntriesWalker.WithConsumer<T>(consumer));
+        // Note: we can't do the ValueConsumer trick here, because the implementation requires state and cannot be
+        // implemented with default methods alone.
+    }
+
+    /**
+     * Process the trie using the given Walker.
+     */
+    public <R> R process(Walker<T, R> walker)
+    {
+        return process(walker, cursor());
+    }
+
+    static <T, R> R process(Walker<T, R> walker, Cursor<T> cursor)
+    {
+        assert cursor.depth() == 0 : "The provided cursor has already been advanced.";
+        T content = cursor.content();   // handle content on the root node
+        if (content == null)
+            content = cursor.advanceToContent(walker);
+
+        while (content != null)
+        {
+            walker.content(content);
+            content = cursor.advanceToContent(walker);
+        }
+        return walker.complete();
+    }
+
+    /**
+     * Constuct a textual representation of the trie.
+     */
+    public String dump()
+    {
+        return dump(Object::toString);
+    }
+
+    /**
+     * Constuct a textual representation of the trie using the given content-to-string mapper.
+     */
+    public String dump(Function<T, String> contentToString)
+    {
+        return process(new TrieDumper<>(contentToString));
+    }
+
+    /**
+     * Returns a singleton trie mapping the given byte path to content.
+     */
+    public static <T> Trie<T> singleton(ByteComparable b, T v)
+    {
+        return new SingletonTrie<>(b, v);
+    }
+
+    /**
+     * Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries.
+     * The view is live, i.e. any write to the source will be reflected in the subtrie.
+     *
+     * This method will not check its arguments for correctness. The resulting trie may be empty or throw an exception
+     * if the right bound is smaller than the left.
+     *
+     * @param left the left bound for the returned subtrie. If {@code null}, the resulting subtrie is not left-bounded.
+     * @param includeLeft whether {@code left} is an inclusive bound of not.
+     * @param right the right bound for the returned subtrie. If {@code null}, the resulting subtrie is not right-bounded.
+     * @param includeRight whether {@code right} is an inclusive bound of not.
+     * @return a view of the subtrie containing all the keys of this trie falling between {@code left} (inclusively if
+     * {@code includeLeft}) and {@code right} (inclusively if {@code includeRight}).
+     */
+    public Trie<T> subtrie(ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight)
+    {
+        if (left == null && right == null)
+            return this;
+        return new SlicedTrie<>(this, left, includeLeft, right, includeRight);
+    }
+
+    /**
+     * Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries,
+     * left-inclusive and right-exclusive.
+     * The view is live, i.e. any write to the source will be reflected in the subtrie.
+     *
+     * This method will not check its arguments for correctness. The resulting trie may be empty or throw an exception
+     * if the right bound is smaller than the left.
+     *
+     * Equivalent to calling subtrie(left, true, right, false).
+     *
+     * @param left the left bound for the returned subtrie. If {@code null}, the resulting subtrie is not left-bounded.
+     * @param right the right bound for the returned subtrie. If {@code null}, the resulting subtrie is not right-bounded.
+     * @return a view of the subtrie containing all the keys of this trie falling between {@code left} (inclusively if
+     * {@code includeLeft}) and {@code right} (inclusively if {@code includeRight}).
+     */
+    public Trie<T> subtrie(ByteComparable left, ByteComparable right)
+    {
+        return subtrie(left, true, right, false);
+    }
+
+    /**
+     * Returns the ordered entry set of this trie's content as an iterable.
+     */
+    public Iterable<Map.Entry<ByteComparable, T>> entrySet()
+    {
+        return this::entryIterator;
+    }
+
+    /**
+     * Returns the ordered entry set of this trie's content in an iterator.
+     */
+    public Iterator<Map.Entry<ByteComparable, T>> entryIterator()
+    {
+        return new TrieEntriesIterator.AsEntries<>(this);
+    }
+
+    /**
+     * Returns the ordered set of values of this trie as an iterable.
+     */
+    public Iterable<T> values()
+    {
+        return this::valueIterator;
+    }
+
+    /**
+     * Returns the ordered set of values of this trie in an iterator.
+     */
+    public Iterator<T> valueIterator()
+    {
+        return new TrieValuesIterator<>(this);
+    }
+
+    /**
+     * Returns the values in any order. For some tries this is much faster than the ordered iterable.
+     */
+    public Iterable<T> valuesUnordered()
+    {
+        return values();
+    }
+
+    /**
+     * Resolver of content of merged nodes, used for two-source merges (i.e. mergeWith).
+     */
+    public interface MergeResolver<T>
+    {
+        // Note: No guarantees about argument order.
+        // E.g. during t1.mergeWith(t2, resolver), resolver may be called with t1 or t2's items as first argument.
+        T resolve(T b1, T b2);
+    }
+
+    /**
+     * Constructs a view of the merge of this trie with the given one. The view is live, i.e. any write to any of the
+     * sources will be reflected in the merged view.
+     *
+     * If there is content for a given key in both sources, the resolver will be called to obtain the combination.
+     * (The resolver will not be called if there's content from only one source.)
+     */
+    public Trie<T> mergeWith(Trie<T> other, MergeResolver<T> resolver)
+    {
+        return new MergeTrie<>(resolver, this, other);
+    }
+
+    /**
+     * Resolver of content of merged nodes.
+     *
+     * The resolver's methods are only called if more than one of the merged nodes contain content, and the
+     * order in which the arguments are given is not defined. Only present non-null values will be included in the
+     * collection passed to the resolving methods.
+     *
+     * Can also be used as a two-source resolver.
+     */
+    public interface CollectionMergeResolver<T> extends MergeResolver<T>
+    {
+        T resolve(Collection<T> contents);
+
+        @Override
+        default T resolve(T c1, T c2)
+        {
+            return resolve(ImmutableList.of(c1, c2));
+        }
+    }
+
+    private static final CollectionMergeResolver<Object> THROWING_RESOLVER = new CollectionMergeResolver<Object>()
+    {
+        @Override
+        public Object resolve(Collection<Object> contents)
+        {
+            throw error();
+        }
+
+        private AssertionError error()
+        {
+            throw new AssertionError("Entries must be distinct.");
+        }
+    };
+
+    /**
+     * Returns a resolver that throws whenever more than one of the merged nodes contains content.
+     * Can be used to merge tries that are known to have distinct content paths.
+     */
+    @SuppressWarnings("unchecked")
+    public static <T> CollectionMergeResolver<T> throwingResolver()
+    {
+        return (CollectionMergeResolver<T>) THROWING_RESOLVER;
+    }
+
+    /**
+     * Constructs a view of the merge of multiple tries. The view is live, i.e. any write to any of the
+     * sources will be reflected in the merged view.
+     *
+     * If there is content for a given key in more than one sources, the resolver will be called to obtain the
+     * combination. (The resolver will not be called if there's content from only one source.)
+     */
+    public static <T> Trie<T> merge(Collection<? extends Trie<T>> sources, CollectionMergeResolver<T> resolver)
+    {
+        switch (sources.size())
+        {
+        case 0:
+            return empty();
+        case 1:
+            return sources.iterator().next();
+        case 2:
+        {
+            Iterator<? extends Trie<T>> it = sources.iterator();
+            Trie<T> t1 = it.next();
+            Trie<T> t2 = it.next();
+            return t1.mergeWith(t2, resolver);
+        }
+        default:
+            return new CollectionMergeTrie<>(sources, resolver);
+        }
+    }
+
+    /**
+     * Constructs a view of the merge of multiple tries, where each source must have distinct keys. The view is live,
+     * i.e. any write to any of the sources will be reflected in the merged view.
+     *
+     * If there is content for a given key in more than one sources, the merge will throw an assertion error.
+     */
+    public static <T> Trie<T> mergeDistinct(Collection<? extends Trie<T>> sources)
+    {
+        switch (sources.size())
+        {
+        case 0:
+            return empty();
+        case 1:
+            return sources.iterator().next();
+        case 2:
+        {
+            Iterator<? extends Trie<T>> it = sources.iterator();
+            Trie<T> t1 = it.next();
+            Trie<T> t2 = it.next();
+            return new MergeTrie.Distinct<>(t1, t2);
+        }
+        default:
+            return new CollectionMergeTrie.Distinct<>(sources);
+        }
+    }
+
+    private static final Trie<Object> EMPTY = new Trie<Object>()
+    {
+        protected Cursor<Object> cursor()
+        {
+            return new Cursor<Object>()
+            {
+                int depth = 0;
+
+                public int advance()
+                {
+                    return depth = -1;
+                }
+
+                public int skipChildren()
+                {
+                    return depth = -1;
+                }
+
+                public int depth()
+                {
+                    return depth;
+                }
+
+                public Object content()
+                {
+                    return null;
+                }
+
+                public int incomingTransition()
+                {
+                    return -1;
+                }
+            };
+        }
+    };
+
+    @SuppressWarnings("unchecked")
+    public static <T> Trie<T> empty()
+    {
+        return (Trie<T>) EMPTY;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/Trie.md b/src/java/org/apache/cassandra/db/tries/Trie.md
new file mode 100644
index 0000000000..4a258c68f9
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/Trie.md
@@ -0,0 +1,252 @@
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+ 
+     http://www.apache.org/licenses/LICENSE-2.0
+ 
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+# `Trie` interface
+
+Tries in Cassandra are used to represent key-value mappings in an efficient way, currently only for the partition map
+in a memtable. The design we use is focussed on performing the equivalent of a read query, being able to most efficiently:
+- combine multiple sources while maintaining order,
+- restrict the combination to a range of keys,
+- efficiently walk the result and extract all covered key-value combinations.
+
+For this the `Trie` interface provides the following public methods:
+- Consuming the content using `forEachValue` or `forEachEntry`.
+- Conversion to iterable and iterator using `values/valuesIterator` and `entrySet/entryIterator`.
+- Getting a view of the subtrie between a given pair of bounds using `subtrie`.
+- Merging two or multiple tries together using `mergeWith` and the static `merge`.
+- Constructing `singleton` tries representing a single key-to-value mapping.
+
+The internal representation of a trie is given by a `Cursor`, which provides a method of walking the nodes of the trie 
+in order, and to which various transformations like merges and intersections can be easily and efficiently applied.
+The sections below detail the motivation behind this design as well as the implementations of the basic operations.
+
+## Walking a trie
+
+Walking a `Trie` is achieved using a cursor. Before we describe it in detail, let's give a quick example of what a
+classic trie walk looks like and how it can be optimized. Suppose we want to walk the following trie:
+
+![graph](MemtableTrie.md.w1.svg)
+
+(Note: the node labels are `MemtableTrie` node IDs which can be ignored here, with the exception of `contentArray[x]` 
+ones which specify that the relevant node has some associated content.)
+
+The classic walk descends (<span style="color:lightblue">light blue</span>) on every character and backtracks 
+(<span style="color:pink">pink</span>) to the parent, resulting in the following walk:
+
+![graph](MemtableTrie.md.w2.svg)
+
+One can see from this graph that many of the backtracking steps are only taken so that they can immediately be followed
+by another backtracking step. We often also know in advance that a node does not need to be examined further on the way
+back: if it only has one child (which is always the case for all nodes in a `Chain`), or if we are descending into 
+its last child (which is easy to check for `Sparse` nodes). This simplifies the walk to:
+
+![graph](MemtableTrie.md.w3.svg)
+
+In addition to making the walk simpler, shortening the backtracking paths means a smaller walk state representation,
+which is quite helpful in keeping the garbage collection cost down. In this example, the backtracking state of the walk
+at the "tractor" node is only `[("tr", child 2)]`, changes to `[("tr", child 3)]` on descent to "tre", and becomes empty
+(as "tr" has no further children and can be removed) on descent to "tri".  
+
+One further optimization of the walk is to jump directly to the next child without stopping at a branching parent (note:
+the black arrows represent the trie structure):
+
+![graph](MemtableTrie.md.wc1.svg)
+
+This graph is what a cursor walk over this trie is. Looking closely at the graph, one can see that it stops exactly once
+on each node, and that the nodes are visited in lexicographic order. There is no longer a need for a separate backtrack
+or ascend operation, because all arrows _advance_ in the representation of the trie. However, to be able to understand
+where the next transition character sits in the path, every transition now also comes with information about the
+descend-depth it ends in.
+
+To see how this can be used to map the state to a path, we can imagine an array being filled at its `depth-1`th
+position on every transition, and the current path being the `depth`-long sequence. This, e.g. the array would hold
+[t, r, a, c, t, o, r] at the "tractor" node and change to [t, r, e, c, t, o, r] for the next advance, where the new 
+current path is the first 3 characters of the array.
+
+Cursors are stateful objects that store the backtracking state of the walk. That is, a list of all parent nodes in the
+path to reach the current node that have further children, together with information which child backtracking should go
+to. Cursors do not store paths as these are often not needed &mdash; for example, in a walk over a merged trie it makes 
+better sense for the consumer to construct the path instead of having them duplicated in every source cursor. Multiple 
+cursors can be constructed and operated in parallel over a trie.
+
+Cursor walks of this type are very efficient but still carry enough information to make it very easy and efficient to
+merge and intersect tries. If we are walking a single trie (or a single-source branch in a union trie), we can
+improve the efficiency even further by taking multiple steps down in `Chain` nodes, provided we have a suitable
+mechanism of passing additional transition characters:
+
+![graph](MemtableTrie.md.wc2.svg)
+
+This is supported by `Cursor.advanceMultiple`.
+
+### Why cursors instead of nodes?
+
+The most straightforward representation of a trie is done by giving users every `Node` visited as an object.
+Then the consumer can query its transitions, get its children, decide to walk over them in any order it sees
+fit, and retain those that it actually needs. This is a very natural and cheap represention if the nodes are actually 
+the objects in memory that represent the trie.
+
+The latter is not the case for us: we store tries in integer blobs or files on disk and we present transformed views of
+tries. Thus every such `Node` object to give to consumers must be constructed. Also, we only do depth-first walks and it 
+does not make that much sense to provide the full flexibility of that kind of interface.
+
+When doing only depth-first walks, a cursor needs far fewer objects to represent its state than a node representation.
+Consider the following for an approach presenting nodes:
+- In a process that requires single-step descends (i.e. in a merge or intersection) the iteration state must create an
+  object for every intermediate node even when they are known to require no backtracking because they have only one
+  child. 
+- Childless final states require a node.
+- A transformation such as a merge must present the result as a transformed node, but it also requires a node for each
+  input. If the transformed node is retained, so must be the sources.
+
+Cursors can represent the first two in their internal state without additional backtracking state, and require only one
+transformed cursor to be constructed for the entire walk. Additionally, cursors' representation of backtracking state 
+may be closely tied to the specific trie implementation, which also gives further improvement opportunities (e.g. the 
+`Split` node treatment in `MemtableTrie`).
+
+### Why not visitors?
+
+A visitor or a push alternative is one where the trie drives the iteration and the caller provides a visitor or a 
+consumer. This can work well if the trie to walk is single-source, but requires some form of stop/restart or pull
+mechanism to implement ordered merges.
+
+Push-style walks are still a useful way to consume the final transformed/merged content, thus `Trie` provides 
+the `Walker` interface and `process` method. The implementations of `forEachEntry` and `dump` are straightforward
+applications of this.
+
+### The `Cursor` interface
+
+The cursor represents the state of a walk over the nodes of trie. It provides three main features:
+- the current `depth` or descend-depth in the trie;
+- the `incomingTransition`, i.e. the byte that was used to reach the current point;
+- the `content` associated with the current node,
+
+and provides methods for advancing to the next position.  This is enough information to extract all paths, and
+also to easily compare cursors over different tries that are advanced together. Advancing is always done in
+order; if one imagines the set of nodes in the trie with their associated paths, a cursor may only advance from a
+node with a lexicographically smaller path to one with bigger. The `advance` operation moves to the immediate
+next, it is also possible to skip over some items e.g. all children of the current node (`skipChildren`).
+
+Moving to the immediate next position in the lexicographic order is accomplished by:
+- if the current node has children, moving to its first child;
+- otherwise, ascend the parent chain and return the next child of the closest parent that still has any.
+
+As long as the trie is not exhausted, advancing always takes one step down, from the current node, or from a node
+on the parent chain. By comparing the new depth (which `advance` also returns) with the one before the advance,
+one can tell if the former was the case (if `newDepth == oldDepth + 1`) and how many steps up we had to take
+(`oldDepth + 1 - newDepth`). When following a path down, the cursor will stop on all prefixes.
+
+In addition to the single-step `advance` method, the cursor also provides an `advanceMultiple` method for descending
+multiple steps down when this is known to be efficient. If it is not feasible to descend (e.g. because there are no
+children, or because getting to the child of the first child requires a page fetch from disk), `advanceMultiple` will
+act just like `advance`.
+
+For convenience, the interface also provides an `advanceToContent` method for walking to the next node with non-null
+content. This is implemented via `advanceMultiple`.
+
+When the cursor is created it is placed on the root node with `depth() = 0`, `incomingTransition() = -1`. Since
+tries can have mappings for empty, `content()` can possibly be non-null in the starting position. It is not allowed
+for a cursor to start in exhausted state (i.e. with `depth() = -1`).
+
+### Using cursors in parallel
+
+One important feature of cursors is the fact that we can easily walk them in parallel. More precisely, when we use a 
+procedure where we only advance the smaller, or both if they are equal, we can compare the cursors' state by:
+- the reverse order of their current depth (i.e. higher depth first),
+- if depths are equal, the order of their current incoming transition (lexicographically smaller first).
+
+We can prove this by induction, where for two cursors `a` and `b` we maintain that:
+1. for any depth `i < mindepth - 1`, `path(a)[i] == path(b)[i]`
+2. if `a.depth < b.depth`, then `path(a)[mindepth - 1] > path(b)[mindepth - 1]`
+3. if `a.depth > b.depth`, then `path(a)[mindepth - 1] < path(b)[mindepth - 1]`
+
+where `mindepth = min(a.depth, b.depth)` and `path(cursor)` is the path corresponding to the node the cursor is
+positioned at. Note that `path(cursor)[cursor.depth - 1] == cursor.incomingTransition`.
+
+These conditions ensure that `path(a) < path(b)` if and only if `a.depth > b.depth` or `a.depth == b.depth` and 
+`a.incomingTransition < b.incomingTransition`. Indeed, if the second part is true then 1 and 3 enforce the first,
+and if the second part is not true, i.e. `a.depth <= b.depth` and (`a.depth != b.depth` or 
+`a.incomingTransition >= b.incomingTransition`), which entails `a.depth < b.depth` or `a.depth == b.depth` and
+`a.incomingTransition >= b.incomingTransition`, then by 2 and 1 we can conclude that `path(a) >= path(b)`, i.e. the
+first part is not true either.
+
+The conditions are trivially true for the initial state where both cursors are positioned at the root with depth 0.
+Also, when we advance a cursor, it is always the case that the path of the previous state and the path of the new state 
+must be the same in all positions before its new depth minus one'th. Thus, if the two cursors are equal before
+advancing, i.e. they are positioned on exactly the same path, the state after advancing must satisfy condition 1 above
+because the earliest byte in either path that can have changed is the one at position `min(a.depth, b.depth) - 1`.
+Moreover, if the depths are different, the cursor with the lower one will have advanced its character in position
+`depth - 1` while the other cursor's character at that position will have remained the same, thus conditions 2 and 3 are
+also satisfied.
+
+If `path(a)` was smaller before advancing we have that `a.depth >= b.depth`. The parallel walk will then only advance
+`a`. If the new depth of `a` is higher than `b`'s, nothing changes in conditions 1-3 (the bytes before `b.depth` do 
+not change at all in either cursor). If the new depth of `a` is the same as `b`'s, condition 1 is still satisfied
+because these bytes cannot have changed, and the premises in 2 and 3 are false. If the new depth of `a` is lower
+than `b`'s, however, `a` must have advanced the byte at index `depth - 1`, and because (due to 1) it was previously
+equal to `b`'s at this index, it must now be higher, proving 2. Condition 1 is still true because these bytes cannot
+have changed, and 3 is true because it has a false premise.
+
+The same argument holds when `b` is the smaller cursor to be advanced.
+
+## Merging two tries
+
+Two tries can be merged using `Trie.mergeWith`, which is implemented using the class `MergeTrie`. The implementation
+is a straightforward application of the parallel walking scheme above, where the merged cursor presents the depth
+and incoming transition of the currently smaller cursor, and advances by advancing the smaller cursor, or both if they
+are equal.
+
+If the cursors are not equal, we can also apply `advanceMultiple`, because it can only be different from `advance`
+if it descends. When a cursor is known to be smaller it is guaranteed to remain smaller when it descends as its
+new depth will be larger than before and thus larger than the other cursor's. This cannot be done to advance both
+cursors when they are equal, because that can violate the conditions. (Simple example: one descends through "a" and 
+the other through "bb" &mdash; condition 2. is violated, the latter will have higher depth but will not be smaller.)
+
+## Merging an arbitrary number of tries
+
+Merging is extended to an arbitrary number of sources in `CollectionMergeTrie`, used through the static `Trie.merge`.
+The process is a generalization of the above, implemented using the min-heap solution from `MergeIterator` applied
+to cursors.
+
+In this solution an extra head element is maintained _before_ the min-heap to optimize for single-source branches
+where we prefer to advance using just one comparison (head to heap top) instead of two (heap top to its two 
+descendants) at the expense of possibly adding one additional comparison in the general case.
+
+As above, when we know that the head element is not equal to the heap top (i.e. it's necessarily smaller) we can
+use its `advanceMultiple` safely.
+
+## Slicing tries
+
+Slicing, implemented in `SlicedTrie` and used via `Trie.subtrie`, can also be seen as a variation of the parallel 
+walk. In this case we walk the source as well as singletons of the two bounds.
+
+More precisely, while the source cursor is smaller than the left bound, we don't produce any output. That is, we
+keep advancing in a loop, but to avoid walking subtries unnecessarily, we use `skipChildren` instead of `advance`.
+As we saw above, a smaller cursor that descends remains smaller, thus there is no point to do so when we are
+ahead of the left bound. When the source matches a node from the left bound, we descend both and pass the
+state to the consumer. As soon as the source becomes known greater than the left bound, we can stop processing 
+the latter and pass any state we see to the consumer. 
+
+Throughout this we also process the right bound cursor and we stop the iteration (by returning `depth = -1`) 
+as soon as the source becomes larger than the right bound.
+
+`SlicedTrie` does not use singleton tries and cursors over them but opts to implement them directly, using an
+implicit representation using a pair of `depth` and `incomingTransition` for each bound.
+
+In slices we can also use `advanceMultiple` when we are certain to be strictly inside the slice, i.e. beyond the
+left bound and before a prefix of the right bound. As above, descending to any depth in this case is safe as the
+result will remain smaller than the right bound.
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/db/tries/TrieDumper.java b/src/java/org/apache/cassandra/db/tries/TrieDumper.java
new file mode 100644
index 0000000000..9dfb2c190d
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/TrieDumper.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import java.util.function.Function;
+
+import org.agrona.DirectBuffer;
+
+/**
+ * Simple utility class for dumping the structure of a trie to string.
+ */
+class TrieDumper<T> implements Trie.Walker<T, String>
+{
+    private final StringBuilder b;
+    private final Function<T, String> contentToString;
+    int needsIndent = -1;
+    int currentLength = 0;
+
+    public TrieDumper(Function<T, String> contentToString)
+    {
+        this.contentToString = contentToString;
+        this.b = new StringBuilder();
+    }
+
+    private void endLineAndSetIndent(int newIndent)
+    {
+        needsIndent = newIndent;
+    }
+
+    @Override
+    public void resetPathLength(int newLength)
+    {
+        currentLength = newLength;
+        endLineAndSetIndent(newLength);
+    }
+
+    private void maybeIndent()
+    {
+        if (needsIndent >= 0)
+        {
+            b.append('\n');
+            for (int i = 0; i < needsIndent; ++i)
+                b.append("  ");
+            needsIndent = -1;
+        }
+    }
+
+    @Override
+    public void addPathByte(int nextByte)
+    {
+        maybeIndent();
+        ++currentLength;
+        b.append(String.format("%02x", nextByte));
+    }
+
+    @Override
+    public void addPathBytes(DirectBuffer buffer, int pos, int count)
+    {
+        maybeIndent();
+        for (int i = 0; i < count; ++i)
+            b.append(String.format("%02x", buffer.getByte(pos + i) & 0xFF));
+        currentLength += count;
+    }
+
+    @Override
+    public void content(T content)
+    {
+        b.append(" -> ");
+        b.append(contentToString.apply(content));
+        endLineAndSetIndent(currentLength);
+    }
+
+    @Override
+    public String complete()
+    {
+        return b.toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java
new file mode 100644
index 0000000000..7ab3e7de46
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import java.util.AbstractMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Convertor of trie entries to iterator where each entry is passed through {@link #mapContent} (to be implemented by
+ * descendants).
+ */
+public abstract class TrieEntriesIterator<T, V> extends TriePathReconstructor implements Iterator<V>
+{
+    private final Trie.Cursor<T> cursor;
+    T next;
+    boolean gotNext;
+
+    protected TrieEntriesIterator(Trie<T> trie)
+    {
+        cursor = trie.cursor();
+        assert cursor.depth() == 0;
+        next = cursor.content();
+        gotNext = next != null;
+    }
+
+    public boolean hasNext()
+    {
+        if (!gotNext)
+        {
+            next = cursor.advanceToContent(this);
+            gotNext = true;
+        }
+
+        return next != null;
+    }
+
+    public V next()
+    {
+        gotNext = false;
+        T v = next;
+        next = null;
+        return mapContent(v, keyBytes, keyPos);
+    }
+
+    protected abstract V mapContent(T content, byte[] bytes, int byteLength);
+
+    /**
+     * Iterator representing the content of the trie a sequence of (path, content) pairs.
+     */
+    static class AsEntries<T> extends TrieEntriesIterator<T, Map.Entry<ByteComparable, T>>
+    {
+        public AsEntries(Trie<T> trie)
+        {
+            super(trie);
+        }
+
+        @Override
+        protected Map.Entry<ByteComparable, T> mapContent(T content, byte[] bytes, int byteLength)
+        {
+            return toEntry(content, bytes, byteLength);
+        }
+    }
+
+    static <T> java.util.Map.Entry<ByteComparable, T> toEntry(T content, byte[] bytes, int byteLength)
+    {
+        return new AbstractMap.SimpleImmutableEntry<>(toByteComparable(bytes, byteLength), content);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java b/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java
new file mode 100644
index 0000000000..ca06015733
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/TrieEntriesWalker.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import java.util.function.BiConsumer;
+
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Walker of trie entries, used with Trie.process to walk all content in order and provide the path through which values
+ * are reached.
+ */
+public abstract class TrieEntriesWalker<T, V> extends TriePathReconstructor implements Trie.Walker<T, V>
+{
+    @Override
+    public void content(T content)
+    {
+        content(content, keyBytes, keyPos);
+    }
+
+    protected abstract void content(T content, byte[] bytes, int byteLength);
+
+    /**
+     * Iterator representing the content of the trie a sequence of (path, content) pairs.
+     */
+    static class WithConsumer<T> extends TrieEntriesWalker<T, Void>
+    {
+        private final BiConsumer<ByteComparable, T> consumer;
+
+        public WithConsumer(BiConsumer<ByteComparable, T> consumer)
+        {
+            this.consumer = consumer;
+        }
+
+        @Override
+        protected void content(T content, byte[] bytes, int byteLength)
+        {
+            consumer.accept(toByteComparable(bytes, byteLength), content);
+        }
+
+        @Override
+        public Void complete()
+        {
+            return null;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/TriePathReconstructor.java b/src/java/org/apache/cassandra/db/tries/TriePathReconstructor.java
new file mode 100644
index 0000000000..4a9883fa00
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/TriePathReconstructor.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.tries;
+
+import java.util.Arrays;
+
+import org.agrona.DirectBuffer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+public class TriePathReconstructor implements Trie.ResettingTransitionsReceiver
+{
+    protected byte[] keyBytes = new byte[32];
+    protected int keyPos = 0;
+
+    public void addPathByte(int nextByte)
+    {
+        if (keyPos >= keyBytes.length)
+            keyBytes = Arrays.copyOf(keyBytes, keyPos * 2);
+        keyBytes[keyPos++] = (byte) nextByte;
+    }
+
+    public void addPathBytes(DirectBuffer buffer, int pos, int count)
+    {
+        int newPos = keyPos + count;
+        if (newPos > keyBytes.length)
+            keyBytes = Arrays.copyOf(keyBytes, Math.max(newPos + 16, keyBytes.length * 2));
+        buffer.getBytes(pos, keyBytes, keyPos, count);
+        keyPos = newPos;
+    }
+
+    public void resetPathLength(int newLength)
+    {
+        keyPos = newLength;
+    }
+
+    static ByteComparable toByteComparable(byte[] bytes, int byteLength)
+    {
+        return ByteComparable.fixedLength(Arrays.copyOf(bytes, byteLength));
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/TrieToDot.java b/src/java/org/apache/cassandra/db/tries/TrieToDot.java
new file mode 100644
index 0000000000..26059d11c5
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/TrieToDot.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import java.util.function.Function;
+
+import org.agrona.DirectBuffer;
+
+/**
+ * Simple utility class for dumping the structure of a trie to string.
+ */
+class TrieToDot<T> extends TriePathReconstructor implements Trie.Walker<T, String>
+{
+    private final StringBuilder b;
+    private final Function<T, String> contentToString;
+    private final Function<Integer, String> transitionToString;
+    private final boolean useMultiByte;
+    private int prevPos;
+    private int currNodeTextPos;
+
+    public TrieToDot(Function<T, String> contentToString,
+                     Function<Integer, String> transitionToString,
+                     boolean useMultiByte)
+    {
+        this.contentToString = contentToString;
+        this.transitionToString = transitionToString;
+        this.useMultiByte = useMultiByte;
+        this.b = new StringBuilder();
+        b.append("digraph G {\n" +
+                 "  splines=curved");
+        addNodeDefinition(nodeString(0));
+    }
+
+    @Override
+    public void resetPathLength(int newLength)
+    {
+        super.resetPathLength(newLength);
+        prevPos = newLength;
+    }
+
+    private void newLineAndIndent()
+    {
+        b.append('\n');
+        for (int i = 0; i < prevPos + 1; ++i)
+            b.append("  ");
+    }
+
+    @Override
+    public void addPathByte(int nextByte)
+    {
+        newLineAndIndent();
+        super.addPathByte(nextByte);
+        b.append(nodeString(prevPos));
+        b.append(" -> ");
+        String newNode = nodeString(keyPos);
+        b.append(newNode);
+        b.append(" [label=\"");
+        for (int i = prevPos; i < keyPos - 1; ++i)
+            b.append(transitionToString.apply(keyBytes[i] & 0xFF));
+        b.append(transitionToString.apply(nextByte));
+        b.append("\"]");
+        addNodeDefinition(newNode);
+    }
+
+    private void addNodeDefinition(String newNode)
+    {
+        prevPos = keyPos;
+        newLineAndIndent();
+        currNodeTextPos = b.length();
+        b.append(String.format("%s [shape=circle label=\"\"]", newNode));
+    }
+
+    private String nodeString(int keyPos)
+    {
+        StringBuilder b = new StringBuilder();
+        b.append("Node_");
+        for (int i = 0; i < keyPos; ++i)
+            b.append(transitionToString.apply(keyBytes[i] & 0xFF));
+        return b.toString();
+    }
+
+    @Override
+    public void addPathBytes(DirectBuffer buffer, int pos, int count)
+    {
+        if (useMultiByte)
+        {
+            super.addPathBytes(buffer, pos, count);
+        }
+        else
+        {
+            for (int i = 0; i < count; ++i)
+                addPathByte(buffer.getByte(pos + i) & 0xFF);
+        }
+    }
+
+    @Override
+    public void content(T content)
+    {
+        b.replace(currNodeTextPos, b.length(), String.format("%s [shape=doublecircle label=\"%s\"]", nodeString(keyPos), contentToString.apply(content)));
+    }
+
+    @Override
+    public String complete()
+    {
+        b.append("\n}\n");
+        return b.toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java b/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java
new file mode 100644
index 0000000000..29d3642b2e
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.tries;
+
+import java.util.Iterator;
+
+/**
+ * Ordered iterator of trie content.
+ */
+class TrieValuesIterator<T> implements Iterator<T>
+{
+    private final Trie.Cursor<T> cursor;
+    T next;
+    boolean gotNext;
+
+    protected TrieValuesIterator(Trie<T> trie)
+    {
+        cursor = trie.cursor();
+        assert cursor.depth() == 0;
+        next = cursor.content();
+        gotNext = next != null;
+    }
+
+    public boolean hasNext()
+    {
+        if (!gotNext)
+        {
+            next = cursor.advanceToContent(null);
+            gotNext = true;
+        }
+
+        return next != null;
+    }
+
+    public T next()
+    {
+        gotNext = false;
+        T v = next;
+        next = null;
+        return v;
+    }
+}
diff --git a/test/microbench/org/apache/cassandra/test/microbench/tries/ComparisonReadBench.java b/test/microbench/org/apache/cassandra/test/microbench/tries/ComparisonReadBench.java
new file mode 100644
index 0000000000..7a1675078a
--- /dev/null
+++ b/test/microbench/org/apache/cassandra/test/microbench/tries/ComparisonReadBench.java
@@ -0,0 +1,517 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.test.microbench.tries;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Random;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.tries.MemtableTrie;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
+import org.github.jamm.MemoryMeter;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+@Warmup(iterations = 2, time = 1)
+@Measurement(iterations = 3, time = 1)
+@Fork(value = 1,jvmArgsAppend = { "-Xmx4G", "-Xms4G", "-Djmh.executor=CUSTOM", "-Djmh.executor.class=org.apache.cassandra.test.microbench.FastThreadExecutor"})
+@Threads(1) // no concurrent writes
+@State(Scope.Benchmark)
+public class ComparisonReadBench
+{
+    // Note: To see a printout of the usage for each object, add .enableDebug() here (most useful with smaller number of
+    // partitions).
+    static MemoryMeter meter = new MemoryMeter().ignoreKnownSingletons()
+                                                .withGuessing(MemoryMeter.Guess.FALLBACK_UNSAFE);
+
+    @Param({"ON_HEAP"})
+    BufferType bufferType = BufferType.OFF_HEAP;
+
+    @Param({"1000", "100000", "10000000"})
+    int count = 1000;
+
+    @Param({"TREE_MAP", "CSLM", "TRIE"})
+    MapOption map = MapOption.TRIE;
+
+    @Param({"LONG"})
+    TypeOption type = TypeOption.LONG;
+
+    final static MemtableTrie.UpsertTransformer<Byte, Byte> resolver = (x, y) -> y;
+
+    Access<?> access;
+
+    @Setup(Level.Trial)
+    public void setup() throws Throwable
+    {
+        switch (map)
+        {
+            case TREE_MAP:
+                access = new NavigableMapAccess(new TreeMap(type.type.comparator()), type.type);
+                break;
+            case CSLM:
+                access = new NavigableMapAccess(new ConcurrentSkipListMap(type.type.comparator()), type.type);
+                break;
+            case TRIE:
+                access = new TrieAccess(type.type);
+                break;
+        }
+        Random rand = new Random(1);
+
+        System.out.format("Putting %,d\n", count);
+        long time = System.currentTimeMillis();
+        for (long current = 0; current < count; ++current)
+        {
+            long l = rand.nextLong();
+            access.put(l, Byte.valueOf((byte) (l >> 56)));
+        }
+        time = System.currentTimeMillis() - time;
+        System.out.format("Took %.3f seconds\n", time * 0.001);
+        access.printSize();
+    }
+
+    interface Type<T>
+    {
+        T fromLong(long l);
+        T fromByteComparable(ByteComparable bc);
+        ByteComparable longToByteComparable(long l);
+        Comparator<T> comparator();
+    }
+
+    public enum TypeOption
+    {
+        LONG(new LongType()),
+        BIGINT(new BigIntType()),
+        DECIMAL(new BigDecimalType()),
+        STRING_BASE16(new StringType(16)),
+        STRING_BASE10(new StringType(10)),
+        STRING_BASE4(new StringType(4)),
+        ARRAY(new ArrayType(1)),
+        ARRAY_REP3(new ArrayType(3)),
+        ARRAY_REP7(new ArrayType(7));
+
+        final Type<?> type;
+
+        TypeOption(Type<?> type)
+        {
+            this.type = type;
+        }
+    }
+
+    static class LongType implements Type<Long>
+    {
+        public Long fromLong(long l)
+        {
+            return l;
+        }
+
+        public Long fromByteComparable(ByteComparable bc)
+        {
+            return ByteSourceInverse.getSignedLong(bc.asComparableBytes(ByteComparable.Version.OSS42));
+        }
+
+        public ByteComparable longToByteComparable(long l)
+        {
+            return ByteComparable.of(l);
+        }
+
+        public Comparator<Long> comparator()
+        {
+            return Comparator.naturalOrder();
+        }
+    }
+
+    static class BigIntType implements Type<BigInteger>
+    {
+        public BigInteger fromLong(long l)
+        {
+            return BigInteger.valueOf(l);
+        }
+
+        public BigInteger fromByteComparable(ByteComparable bc)
+        {
+            return IntegerType.instance.compose(IntegerType.instance.fromComparableBytes(ByteSource.peekable(bc.asComparableBytes(ByteComparable.Version.OSS42)),
+                                                                                         ByteComparable.Version.OSS42));
+        }
+
+        public ByteComparable longToByteComparable(long l)
+        {
+            return v -> IntegerType.instance.asComparableBytes(IntegerType.instance.decompose(fromLong(l)), v);
+        }
+
+        public Comparator<BigInteger> comparator()
+        {
+            return Comparator.naturalOrder();
+        }
+    }
+
+    static class BigDecimalType implements Type<BigDecimal>
+    {
+        public BigDecimal fromLong(long l)
+        {
+            return BigDecimal.valueOf(l);
+        }
+
+        public BigDecimal fromByteComparable(ByteComparable bc)
+        {
+            return DecimalType.instance.compose(DecimalType.instance.fromComparableBytes(ByteSource.peekable(bc.asComparableBytes(ByteComparable.Version.OSS42)),
+                                                                                         ByteComparable.Version.OSS42));
+        }
+
+        public ByteComparable longToByteComparable(long l)
+        {
+            return v -> DecimalType.instance.asComparableBytes(DecimalType.instance.decompose(fromLong(l)), v);
+        }
+
+        public Comparator<BigDecimal> comparator()
+        {
+            return Comparator.naturalOrder();
+        }
+    }
+
+    static class StringType implements Type<String>
+    {
+        final int base;
+
+        StringType(int base)
+        {
+            this.base = base;
+        }
+
+        public String fromLong(long l)
+        {
+            return Long.toString(l, base);
+        }
+
+        public String fromByteComparable(ByteComparable bc)
+        {
+            return new String(ByteSourceInverse.readBytes(bc.asComparableBytes(ByteComparable.Version.OSS42)), StandardCharsets.UTF_8);
+        }
+
+        public ByteComparable longToByteComparable(long l)
+        {
+            return ByteComparable.fixedLength(fromLong(l).getBytes(StandardCharsets.UTF_8));
+        }
+
+        public Comparator<String> comparator()
+        {
+            return Comparator.naturalOrder();
+        }
+    }
+
+    static class ArrayType implements Type<byte[]>
+    {
+        final int reps;
+
+        ArrayType(int reps)
+        {
+            this.reps = reps;
+        }
+
+        public byte[] fromLong(long l)
+        {
+            byte[] value = new byte[8 * reps];
+            for (int i = 0; i < 8; ++i)
+            {
+                for (int j = 0; j < reps; ++j)
+                    value[i * reps + j] = (byte)(l >> (56 - i * 8));
+            }
+            return value;
+        }
+
+        public byte[] fromByteComparable(ByteComparable bc)
+        {
+            return ByteSourceInverse.readBytes(bc.asComparableBytes(ByteComparable.Version.OSS42));
+        }
+
+        public ByteComparable longToByteComparable(long l)
+        {
+            return ByteComparable.fixedLength(fromLong(l));
+        }
+
+        public Comparator<byte[]> comparator()
+        {
+            return ByteArrayUtil::compareUnsigned;
+        }
+    }
+
+    interface Access<T>
+    {
+        void put(long v, byte b);
+        byte get(long v);
+        Iterable<Byte> values();
+        Iterable<Byte> valuesSlice(long left, boolean includeLeft, long right, boolean includeRight);
+        Iterable<Map.Entry<T, Byte>> entrySet();
+        void consumeValues(Consumer<Byte> consumer);
+        void consumeEntries(BiConsumer<T, Byte> consumer);
+        void printSize();
+    }
+
+    public enum MapOption
+    {
+        TREE_MAP,
+        CSLM,
+        TRIE
+    }
+
+    class TrieAccess<T> implements Access<T>
+    {
+        final MemtableTrie<Byte> trie;
+        final Type<T> type;
+
+        TrieAccess(Type<T> type)
+        {
+            this.type = type;
+            trie = new MemtableTrie<>(bufferType);
+        }
+
+        public void put(long v, byte b)
+        {
+            try
+            {
+                trie.putRecursive(type.longToByteComparable(v), b, resolver);
+            }
+            catch (MemtableTrie.SpaceExhaustedException e)
+            {
+                throw Throwables.propagate(e);
+            }
+        }
+
+        public byte get(long v)
+        {
+            return trie.get(type.longToByteComparable(v));
+        }
+
+        public Iterable<Byte> values()
+        {
+            return trie.values();
+        }
+
+        public Iterable<Byte> valuesSlice(long left, boolean includeLeft, long right, boolean includeRight)
+        {
+            return trie.subtrie(type.longToByteComparable(left), includeLeft, type.longToByteComparable(right), includeRight)
+                       .values();
+        }
+
+        public Iterable<Map.Entry<T, Byte>> entrySet()
+        {
+            return Iterables.transform(trie.entrySet(),
+                    en -> new AbstractMap.SimpleEntry<>(type.fromByteComparable(en.getKey()),
+                                                        en.getValue()));
+        }
+
+        public void consumeValues(Consumer<Byte> consumer)
+        {
+            trie.forEachValue(consumer::accept);
+        }
+
+        public void consumeEntries(BiConsumer<T, Byte> consumer)
+        {
+            trie.forEachEntry((key, value) -> consumer.accept(type.fromByteComparable(key), value));
+        }
+
+        public void printSize()
+        {
+            long deepsize = meter.measureDeep(trie);
+            System.out.format("Trie size on heap %,d off heap %,d deep size %,d\n",
+                              trie.sizeOnHeap(), trie.sizeOffHeap(), deepsize);
+            System.out.format("per entry on heap %.2f off heap %.2f deep size %.2f\n",
+                              trie.sizeOnHeap() * 1.0 / count, trie.sizeOffHeap() * 1.0 / count, deepsize * 1.0 / count);
+        }
+    }
+
+    class NavigableMapAccess<T> implements Access<T>
+    {
... 2478 lines suppressed ...


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