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:27 UTC

[cassandra] 05/05: Renames MemtableTrie to InMemoryTrie

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 9074ee7ef8e041e1b15116373be0df80b985e3d9
Author: Branimir Lambov <br...@datastax.com>
AuthorDate: Thu Oct 20 12:32:07 2022 +0300

    Renames MemtableTrie to InMemoryTrie
    
    patch by Branimir Lambov; reviewed by Andres de la Peña and Caleb Rackliffe for CASSANDRA-17240
---
 .../apache/cassandra/db/memtable/TrieMemtable.java | 14 +++----
 ...MemtableReadTrie.java => InMemoryReadTrie.java} | 13 +++---
 .../tries/{MemtableTrie.java => InMemoryTrie.java} | 23 ++++++-----
 .../db/tries/{MemtableTrie.md => InMemoryTrie.md}  | 46 +++++++++++-----------
 ...mtableTrie.md.a1.svg => InMemoryTrie.md.a1.svg} |  0
 ...mtableTrie.md.g1.svg => InMemoryTrie.md.g1.svg} |  0
 ...mtableTrie.md.g2.svg => InMemoryTrie.md.g2.svg} |  0
 ...mtableTrie.md.g3.svg => InMemoryTrie.md.g3.svg} |  0
 ...mtableTrie.md.g4.svg => InMemoryTrie.md.g4.svg} |  0
 ...mtableTrie.md.m1.svg => InMemoryTrie.md.m1.svg} |  0
 ...mtableTrie.md.m2.svg => InMemoryTrie.md.m2.svg} |  0
 ...mtableTrie.md.m3.svg => InMemoryTrie.md.m3.svg} |  0
 ...mtableTrie.md.p1.svg => InMemoryTrie.md.p1.svg} |  0
 ...mtableTrie.md.w1.svg => InMemoryTrie.md.w1.svg} |  0
 ...mtableTrie.md.w2.svg => InMemoryTrie.md.w2.svg} |  0
 ...mtableTrie.md.w3.svg => InMemoryTrie.md.w3.svg} |  0
 ...ableTrie.md.wc1.svg => InMemoryTrie.md.wc1.svg} |  0
 ...ableTrie.md.wc2.svg => InMemoryTrie.md.wc2.svg} |  0
 src/java/org/apache/cassandra/db/tries/Trie.md     | 14 +++----
 .../test/microbench/tries/ComparisonReadBench.java | 10 ++---
 ...ieReadBench.java => InMemoryTrieReadBench.java} | 10 ++---
 ...UnionBench.java => InMemoryTrieUnionBench.java} | 16 ++++----
 ...WriteBench.java => InMemoryTrieWriteBench.java} | 22 +++++------
 .../db/tries/CollectionMergeTrieTest.java          | 22 +++++------
 ...ieApplyTest.java => InMemoryTrieApplyTest.java} |  2 +-
 ...leTriePutTest.java => InMemoryTriePutTest.java} | 22 +++++------
 ...TrieTestBase.java => InMemoryTrieTestBase.java} | 34 ++++++++--------
 ...adedTest.java => InMemoryTrieThreadedTest.java} |  8 ++--
 .../apache/cassandra/db/tries/MergeTrieTest.java   | 18 ++++-----
 .../apache/cassandra/db/tries/SlicedTrieTest.java  | 22 +++++------
 .../apache/cassandra/db/tries/TrieToDotTest.java   |  4 +-
 .../cassandra/metrics/TrieMemtableMetricsTest.java |  2 +-
 32 files changed, 150 insertions(+), 152 deletions(-)

diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java
index 4c59bbba6f..ae3d2e8ec3 100644
--- a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java
+++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java
@@ -55,7 +55,7 @@ import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.rows.EncodingStats;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
-import org.apache.cassandra.db.tries.MemtableTrie;
+import org.apache.cassandra.db.tries.InMemoryTrie;
 import org.apache.cassandra.db.tries.Trie;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Bounds;
@@ -130,7 +130,7 @@ public class TrieMemtable extends AbstractShardedMemtable
 
     /**
      * A merged view of the memtable map. Used for partition range queries and flush.
-     * For efficiency we serve single partition requests off the shard which offers more direct MemtableTrie methods.
+     * For efficiency we serve single partition requests off the shard which offers more direct InMemoryTrie methods.
      */
     private final Trie<BTreePartitionData> mergedTrie;
 
@@ -214,9 +214,9 @@ public class TrieMemtable extends AbstractShardedMemtable
 
             return colUpdateTimeDelta;
         }
-        catch (MemtableTrie.SpaceExhaustedException e)
+        catch (InMemoryTrie.SpaceExhaustedException e)
         {
-            // This should never happen as {@link MemtableTrie#reachedAllocatedSizeThreshold} should become
+            // This should never happen as {@link InMemoryTrie#reachedAllocatedSizeThreshold} should become
             // true and trigger a memtable switch long before this limit is reached.
             throw new IllegalStateException(e);
         }
@@ -443,7 +443,7 @@ public class TrieMemtable extends AbstractShardedMemtable
         // unsafely, meaning that the memtable will not be discarded as long as the data is used, or whether the data
         // should be copied on heap for off-heap allocators.
         @VisibleForTesting
-        final MemtableTrie<BTreePartitionData> data;
+        final InMemoryTrie<BTreePartitionData> data;
 
         private final ColumnsCollector columnsCollector;
 
@@ -458,14 +458,14 @@ public class TrieMemtable extends AbstractShardedMemtable
         @VisibleForTesting
         MemtableShard(TableMetadataRef metadata, MemtableAllocator allocator, TrieMemtableMetricsView metrics)
         {
-            this.data = new MemtableTrie<>(BUFFER_TYPE);
+            this.data = new InMemoryTrie<>(BUFFER_TYPE);
             this.columnsCollector = new AbstractMemtable.ColumnsCollector(metadata.get().regularAndStaticColumns());
             this.statsCollector = new AbstractMemtable.StatsCollector();
             this.allocator = allocator;
             this.metrics = metrics;
         }
 
-        public long put(DecoratedKey key, PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) throws MemtableTrie.SpaceExhaustedException
+        public long put(DecoratedKey key, PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) throws InMemoryTrie.SpaceExhaustedException
         {
             BTreePartitionUpdater updater = new BTreePartitionUpdater(allocator, allocator.cloner(opGroup), opGroup, indexer);
             boolean locked = writeLock.tryLock();
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableReadTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java
similarity index 98%
rename from src/java/org/apache/cassandra/db/tries/MemtableReadTrie.java
rename to src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java
index 5b7919c932..88f5987a33 100644
--- a/src/java/org/apache/cassandra/db/tries/MemtableReadTrie.java
+++ b/src/java/org/apache/cassandra/db/tries/InMemoryReadTrie.java
@@ -26,12 +26,11 @@ 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.
+ * 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.
+ * This class provides the read-only functionality, expanded in {@link InMemoryTrie} to writes.
  */
-public class MemtableReadTrie<T> extends Trie<T>
+public class InMemoryReadTrie<T> extends Trie<T>
 {
     /*
     TRIE FORMAT AND NODE TYPES
@@ -122,7 +121,7 @@ public class MemtableReadTrie<T> extends Trie<T>
      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.
+     For further descriptions and examples of the mechanics of the trie, see InMemoryTrie.md.
      */
 
     static final int BLOCK_SIZE = 32;
@@ -219,7 +218,7 @@ public class MemtableReadTrie<T> extends Trie<T>
     final UnsafeBuffer[] buffers;
     final AtomicReferenceArray<T>[] contentArrays;
 
-    MemtableReadTrie(UnsafeBuffer[] buffers, AtomicReferenceArray<T>[] contentArrays, int root)
+    InMemoryReadTrie(UnsafeBuffer[] buffers, AtomicReferenceArray<T>[] contentArrays, int root)
     {
         this.buffers = buffers;
         this.contentArrays = contentArrays;
@@ -526,7 +525,7 @@ public class MemtableReadTrie<T> extends Trie<T>
     /*
      * Cursor implementation.
      *
-     * MemtableTrie cursors maintain their backtracking state in CursorBacktrackingState where they store
+     * InMemoryTrie 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:
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.java b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java
similarity index 98%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.java
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.java
index f0dbe098f6..19f28c339b 100644
--- a/src/java/org/apache/cassandra/db/tries/MemtableTrie.java
+++ b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.java
@@ -35,8 +35,7 @@ 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.
+ * 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
@@ -45,9 +44,9 @@ import org.github.jamm.MemoryLayoutSpecification;
  *
  * 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>
+public class InMemoryTrie<T> extends InMemoryReadTrie<T>
 {
-    // See the trie format description in MemtableReadTrie.
+    // See the trie format description in InMemoryReadTrie.
 
     /**
      * Trie size limit. This is not enforced, but users must check from time to time that it is not exceeded (using
@@ -79,13 +78,13 @@ public class MemtableTrie<T> extends MemtableReadTrie<T>
 
     static
     {
-        MemtableTrie<Object> empty = new MemtableTrie<>(BufferType.ON_HEAP);
+        InMemoryTrie<Object> empty = new InMemoryTrie<>(BufferType.ON_HEAP);
         EMPTY_SIZE_ON_HEAP = ObjectSizes.measureDeep(empty);
-        empty = new MemtableTrie<>(BufferType.OFF_HEAP);
+        empty = new InMemoryTrie<>(BufferType.OFF_HEAP);
         EMPTY_SIZE_OFF_HEAP = ObjectSizes.measureDeep(empty);
     }
 
-    public MemtableTrie(BufferType bufferType)
+    public InMemoryTrie(BufferType bufferType)
     {
         super(new UnsafeBuffer[31 - BUF_START_SHIFT],  // last one is 1G for a total of ~2G bytes
               new AtomicReferenceArray[29 - CONTENTS_START_SHIFT],  // takes at least 4 bytes to write pointer to one content -> 4 times smaller than buffers
@@ -139,7 +138,7 @@ public class MemtableTrie<T> extends MemtableReadTrie<T>
 
     private int allocateBlock() throws SpaceExhaustedException
     {
-        // Note: If this method is modified, please run MemtableTrieTest.testOver1GSize to verify it acts correctly
+        // Note: If this method is modified, please run InMemoryTrieTest.testOver1GSize to verify it acts correctly
         // close to the 2G limit.
         int v = allocatedPos;
         if (inChunkPointer(v) == 0)
@@ -727,7 +726,7 @@ public class MemtableTrie<T> extends MemtableReadTrie<T>
             if (isNull(updatedPostContentNode))
                 setUpdatedPostContentNode(expandOrCreateChainNode(transition, child));
             else
-                setUpdatedPostContentNode(MemtableTrie.this.attachChild(updatedPostContentNode,
+                setUpdatedPostContentNode(InMemoryTrie.this.attachChild(updatedPostContentNode,
                                                                         transition,
                                                                         child));
         }
@@ -793,11 +792,11 @@ public class MemtableTrie<T> extends MemtableReadTrie<T>
 
     /**
      * 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
+     * upserted into a {@link InMemoryTrie}. 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}.
+     * @param <T> The content type for this {@link InMemoryTrie}.
+     * @param <U> The type of the new content being applied to this {@link InMemoryTrie}.
      */
     public interface UpsertTransformer<T, U>
     {
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md
similarity index 97%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md
index 901e29ce16..010003a679 100644
--- a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md
+++ b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md
@@ -16,9 +16,9 @@
  limitations under the License.
 -->
 
-# MemtableTrie Design
+# InMemoryTrie Design
 
-The `MemtableTrie` is one of the main components of the trie infrastructure, a mutable in-memory trie built for fast
+The `InMemoryTrie` 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:
@@ -37,7 +37,7 @@ as Java objects in a content array). The structure resides in one `UnsafeBuffer`
 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
+Like all tries, `InMemoryTrie` 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.
 
@@ -74,7 +74,7 @@ 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`,
+`InMemoryTrie`'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()`.
@@ -133,7 +133,7 @@ nodes (an example will be given below). In any case, the byte pointed directly b
 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)
+![graph](InMemoryTrie.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`.
@@ -172,7 +172,7 @@ To explain this better, we will give an example of the evolution of a sparse nod
 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)
+![graph](InMemoryTrie.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
@@ -374,7 +374,7 @@ To find a child in this structure, we follow the transitions along the bits of t
 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)
+![graph](InMemoryTrie.md.g3.svg)
 
 #### Content `Prefix`
 
@@ -428,25 +428,25 @@ offset|content|example
 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)
+![graph](InMemoryTrie.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
+`InMemoryTrie` 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).
+interface implemented by `InMemoryTrie` (see `Trie.md` for a description of cursors).
 
-![graph](MemtableTrie.md.wc1.svg)
+![graph](InMemoryTrie.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`
+### Cursors over `InMemoryTrie`
 
-`MemtableTrie` implements cursors using arrays of integers to store the backtracking state (as the simplest
+`InMemoryTrie` 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 
@@ -465,19 +465,19 @@ This substructure is a little more efficient than storing only one entry for the
 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:
+`InMemoryTrie` cursors also implement `advanceMultiple`, which jumps over intermediate nodes in `Chain` blocks:
 
-![graph](MemtableTrie.md.wc2.svg)
+![graph](InMemoryTrie.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
+Mutation of `InMemoryTrie` 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 
+The main method for mutating a `InMemoryTrie` is `apply`, which merges the structure of another `Trie` in. 
+`InMemoryTrie` 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`
@@ -487,7 +487,7 @@ insertion process walks the trie to find corresponding existing nodes for the on
 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)
+![graph](InMemoryTrie.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
@@ -501,7 +501,7 @@ writing the new character at the address just before the child pointer, and retu
 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)
+![graph](InMemoryTrie.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
@@ -524,7 +524,7 @@ 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)
+![graph](InMemoryTrie.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
@@ -706,7 +706,7 @@ After all modifications coming as the result of application of child branches ha
 
 For example (adding a trie containing "traverse, truck" to the "tractor, tree, trie" one):
 
-![graph](MemtableTrie.md.a1.svg)
+![graph](InMemoryTrie.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)
@@ -745,7 +745,7 @@ 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)
+![graph](InMemoryTrie.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
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.a1.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.a1.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.a1.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.a1.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g1.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.g1.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g1.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.g1.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g2.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.g2.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g2.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.g2.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g3.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.g3.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g3.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.g3.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g4.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.g4.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g4.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.g4.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m1.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.m1.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m1.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.m1.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m2.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.m2.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m2.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.m2.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m3.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.m3.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m3.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.m3.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.p1.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.p1.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.p1.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.p1.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w1.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.w1.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w1.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.w1.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w2.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.w2.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w2.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.w2.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w3.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.w3.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w3.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.w3.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc1.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.wc1.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc1.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.wc1.svg
diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc2.svg b/src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.wc2.svg
similarity index 100%
rename from src/java/org/apache/cassandra/db/tries/MemtableTrie.md.wc2.svg
rename to src/java/org/apache/cassandra/db/tries/InMemoryTrie.md.wc2.svg
diff --git a/src/java/org/apache/cassandra/db/tries/Trie.md b/src/java/org/apache/cassandra/db/tries/Trie.md
index 4a258c68f9..4265871e7b 100644
--- a/src/java/org/apache/cassandra/db/tries/Trie.md
+++ b/src/java/org/apache/cassandra/db/tries/Trie.md
@@ -40,22 +40,22 @@ The sections below detail the motivation behind this design as well as the imple
 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)
+![graph](InMemoryTrie.md.w1.svg)
 
-(Note: the node labels are `MemtableTrie` node IDs which can be ignored here, with the exception of `contentArray[x]` 
+(Note: the node labels are `InMemoryTrie` 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)
+![graph](InMemoryTrie.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)
+![graph](InMemoryTrie.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
@@ -65,7 +65,7 @@ at the "tractor" node is only `[("tr", child 2)]`, changes to `[("tr", child 3)]
 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)
+![graph](InMemoryTrie.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
@@ -89,7 +89,7 @@ merge and intersect tries. If we are walking a single trie (or a single-source b
 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)
+![graph](InMemoryTrie.md.wc2.svg)
 
 This is supported by `Cursor.advanceMultiple`.
 
@@ -116,7 +116,7 @@ Consider the following for an approach presenting nodes:
 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`).
+`Split` node treatment in `InMemoryTrie`).
 
 ### Why not visitors?
 
diff --git a/test/microbench/org/apache/cassandra/test/microbench/tries/ComparisonReadBench.java b/test/microbench/org/apache/cassandra/test/microbench/tries/ComparisonReadBench.java
index 7a1675078a..1709601b53 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/tries/ComparisonReadBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/tries/ComparisonReadBench.java
@@ -36,7 +36,7 @@ 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.db.tries.InMemoryTrie;
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.utils.ByteArrayUtil;
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
@@ -83,7 +83,7 @@ public class ComparisonReadBench
     @Param({"LONG"})
     TypeOption type = TypeOption.LONG;
 
-    final static MemtableTrie.UpsertTransformer<Byte, Byte> resolver = (x, y) -> y;
+    final static InMemoryTrie.UpsertTransformer<Byte, Byte> resolver = (x, y) -> y;
 
     Access<?> access;
 
@@ -302,13 +302,13 @@ public class ComparisonReadBench
 
     class TrieAccess<T> implements Access<T>
     {
-        final MemtableTrie<Byte> trie;
+        final InMemoryTrie<Byte> trie;
         final Type<T> type;
 
         TrieAccess(Type<T> type)
         {
             this.type = type;
-            trie = new MemtableTrie<>(bufferType);
+            trie = new InMemoryTrie<>(bufferType);
         }
 
         public void put(long v, byte b)
@@ -317,7 +317,7 @@ public class ComparisonReadBench
             {
                 trie.putRecursive(type.longToByteComparable(v), b, resolver);
             }
-            catch (MemtableTrie.SpaceExhaustedException e)
+            catch (InMemoryTrie.SpaceExhaustedException e)
             {
                 throw Throwables.propagate(e);
             }
diff --git a/test/microbench/org/apache/cassandra/test/microbench/tries/MemtableTrieReadBench.java b/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieReadBench.java
similarity index 95%
rename from test/microbench/org/apache/cassandra/test/microbench/tries/MemtableTrieReadBench.java
rename to test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieReadBench.java
index 0d8dff5a86..cff2e4a3eb 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/tries/MemtableTrieReadBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieReadBench.java
@@ -22,7 +22,7 @@ import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import java.util.function.BiConsumer;
 
-import org.apache.cassandra.db.tries.MemtableTrie;
+import org.apache.cassandra.db.tries.InMemoryTrie;
 import org.apache.cassandra.db.tries.Trie;
 import org.apache.cassandra.db.tries.TrieEntriesWalker;
 import org.apache.cassandra.io.compress.BufferType;
@@ -36,7 +36,7 @@ import org.openjdk.jmh.annotations.*;
 @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 MemtableTrieReadBench
+public class InMemoryTrieReadBench
 {
     @Param({"ON_HEAP", "OFF_HEAP"})
     BufferType bufferType = BufferType.OFF_HEAP;
@@ -44,14 +44,14 @@ public class MemtableTrieReadBench
     @Param({"1000", "100000", "10000000"})
     int count = 1000;
 
-    final static MemtableTrie.UpsertTransformer<Byte, Byte> resolver = (x, y) -> y;
+    final static InMemoryTrie.UpsertTransformer<Byte, Byte> resolver = (x, y) -> y;
 
-    MemtableTrie<Byte> trie;
+    InMemoryTrie<Byte> trie;
 
     @Setup(Level.Trial)
     public void setup() throws Throwable
     {
-        trie = new MemtableTrie<>(bufferType);
+        trie = new InMemoryTrie<>(bufferType);
         Random rand = new Random(1);
 
         System.out.format("Putting %,d\n", count);
diff --git a/test/microbench/org/apache/cassandra/test/microbench/tries/MemtableTrieUnionBench.java b/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieUnionBench.java
similarity index 90%
rename from test/microbench/org/apache/cassandra/test/microbench/tries/MemtableTrieUnionBench.java
rename to test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieUnionBench.java
index 3d40b0bfa6..e32e20f7e6 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/tries/MemtableTrieUnionBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieUnionBench.java
@@ -25,7 +25,7 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.db.tries.MemtableTrie;
+import org.apache.cassandra.db.tries.InMemoryTrie;
 import org.apache.cassandra.db.tries.Trie;
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
@@ -38,7 +38,7 @@ import org.openjdk.jmh.annotations.*;
 @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 MemtableTrieUnionBench
+public class InMemoryTrieUnionBench
 {
     @Param({"ON_HEAP", "OFF_HEAP"})
     BufferType bufferType = BufferType.OFF_HEAP;
@@ -52,26 +52,26 @@ public class MemtableTrieUnionBench
     @Param({"false", "true"})
     boolean sequential = true;
 
-    final static MemtableTrie.UpsertTransformer<Byte, Byte> resolver = (x, y) -> y;
+    final static InMemoryTrie.UpsertTransformer<Byte, Byte> resolver = (x, y) -> y;
 
     Trie<Byte> trie;
 
     @Setup(Level.Trial)
     public void setup() throws Throwable
     {
-        List<MemtableTrie<Byte>> tries = new ArrayList<>(sources);
+        List<InMemoryTrie<Byte>> tries = new ArrayList<>(sources);
         System.out.format("Putting %,d among %d tries\n", count, sources);
         Random rand = new Random(1);
         if (sequential)
         {
             long sz = 65536 / sources;
             for (int i = 0; i < sources; ++i)
-                tries.add(new MemtableTrie<>(bufferType));
+                tries.add(new InMemoryTrie<>(bufferType));
 
             for (long current = 0; current < count; ++current)
             {
                 long l = rand.nextLong();
-                MemtableTrie<Byte> tt = tries.get(Math.min((int) (((l >> 48) + 32768) / sz), sources - 1));
+                InMemoryTrie<Byte> tt = tries.get(Math.min((int) (((l >> 48) + 32768) / sz), sources - 1));
                 tt.putRecursive(ByteComparable.of(l), (byte) (l >> 56), resolver);
             }
 
@@ -81,7 +81,7 @@ public class MemtableTrieUnionBench
             long current = 0;
             for (int i = 0; i < sources; ++i)
             {
-                MemtableTrie<Byte> trie = new MemtableTrie(bufferType);
+                InMemoryTrie<Byte> trie = new InMemoryTrie(bufferType);
                 int currMax = this.count * (i + 1) / sources;
 
                 for (; current < currMax; ++current)
@@ -93,7 +93,7 @@ public class MemtableTrieUnionBench
             }
         }
 
-        for (MemtableTrie<Byte> trie : tries)
+        for (InMemoryTrie<Byte> trie : tries)
         {
             System.out.format("Trie size on heap %,d off heap %,d\n",
                               trie.sizeOnHeap(), trie.sizeOffHeap());
diff --git a/test/microbench/org/apache/cassandra/test/microbench/tries/MemtableTrieWriteBench.java b/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieWriteBench.java
similarity index 85%
rename from test/microbench/org/apache/cassandra/test/microbench/tries/MemtableTrieWriteBench.java
rename to test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieWriteBench.java
index e473a220a9..f2be11dd8a 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/tries/MemtableTrieWriteBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/tries/InMemoryTrieWriteBench.java
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.db.tries.MemtableTrie;
+import org.apache.cassandra.db.tries.InMemoryTrie;
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 import org.openjdk.jmh.annotations.*;
@@ -34,7 +34,7 @@ import org.openjdk.jmh.infra.Blackhole;
 @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 MemtableTrieWriteBench
+public class InMemoryTrieWriteBench
 {
     @Param({"ON_HEAP", "OFF_HEAP"})
     BufferType bufferType = BufferType.OFF_HEAP;
@@ -45,16 +45,16 @@ public class MemtableTrieWriteBench
     @Param({"8"})
     int keyLength = 8;
 
-    final static MemtableTrie.UpsertTransformer<Byte, Byte> resolver = (x, y) -> y;
+    final static InMemoryTrie.UpsertTransformer<Byte, Byte> resolver = (x, y) -> y;
 
     // Set this to true to print the trie sizes after insertions for sanity checking.
     // This might affect the timings, do not commit with this set to true.
     final static boolean PRINT_SIZES = false;
 
     @Benchmark
-    public void putSequential(Blackhole bh) throws MemtableTrie.SpaceExhaustedException
+    public void putSequential(Blackhole bh) throws InMemoryTrie.SpaceExhaustedException
     {
-        MemtableTrie<Byte> trie = new MemtableTrie(bufferType);
+        InMemoryTrie<Byte> trie = new InMemoryTrie(bufferType);
         ByteBuffer buf = ByteBuffer.allocate(keyLength);
 
         for (long current = 0; current < count; ++current)
@@ -69,9 +69,9 @@ public class MemtableTrieWriteBench
     }
 
     @Benchmark
-    public void putRandom(Blackhole bh) throws MemtableTrie.SpaceExhaustedException
+    public void putRandom(Blackhole bh) throws InMemoryTrie.SpaceExhaustedException
     {
-        MemtableTrie<Byte> trie = new MemtableTrie(bufferType);
+        InMemoryTrie<Byte> trie = new InMemoryTrie(bufferType);
         Random rand = new Random(1);
         byte[] buf = new byte[keyLength];
 
@@ -86,9 +86,9 @@ public class MemtableTrieWriteBench
     }
 
     @Benchmark
-    public void applySequential(Blackhole bh) throws MemtableTrie.SpaceExhaustedException
+    public void applySequential(Blackhole bh) throws InMemoryTrie.SpaceExhaustedException
     {
-        MemtableTrie<Byte> trie = new MemtableTrie(bufferType);
+        InMemoryTrie<Byte> trie = new InMemoryTrie(bufferType);
         ByteBuffer buf = ByteBuffer.allocate(keyLength);
 
         for (long current = 0; current < count; ++current)
@@ -103,9 +103,9 @@ public class MemtableTrieWriteBench
     }
 
     @Benchmark
-    public void applyRandom(Blackhole bh) throws MemtableTrie.SpaceExhaustedException
+    public void applyRandom(Blackhole bh) throws InMemoryTrie.SpaceExhaustedException
     {
-        MemtableTrie<Byte> trie = new MemtableTrie(bufferType);
+        InMemoryTrie<Byte> trie = new InMemoryTrie(bufferType);
         Random rand = new Random(1);
         byte[] buf = new byte[keyLength];
 
diff --git a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java
index 4d2a37f13e..94903a74d7 100644
--- a/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java
+++ b/test/unit/org/apache/cassandra/db/tries/CollectionMergeTrieTest.java
@@ -32,7 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 
-import static org.apache.cassandra.db.tries.MemtableTrieTestBase.*;
+import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.*;
 import static org.apache.cassandra.db.tries.MergeTrieTest.removeDuplicates;
 
 public class CollectionMergeTrieTest
@@ -48,8 +48,8 @@ public class CollectionMergeTrieTest
         SortedMap<ByteComparable, ByteBuffer> content1 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
         SortedMap<ByteComparable, ByteBuffer> content2 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
 
-        MemtableTrie<ByteBuffer> trie1 = makeMemtableTrie(src1, content1, true);
-        MemtableTrie<ByteBuffer> trie2 = makeMemtableTrie(src2, content2, true);
+        InMemoryTrie<ByteBuffer> trie1 = makeInMemoryTrie(src1, content1, true);
+        InMemoryTrie<ByteBuffer> trie2 = makeInMemoryTrie(src2, content2, true);
 
         content1.putAll(content2);
         // construct directly, trie.merge() will defer to mergeWith on two sources
@@ -66,11 +66,11 @@ public class CollectionMergeTrieTest
         SortedMap<ByteComparable, ByteBuffer> content1 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
         SortedMap<ByteComparable, ByteBuffer> content2 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
 
-        MemtableTrie<ByteBuffer> trie1 = makeMemtableTrie(src1, content1, true);
-        MemtableTrie<ByteBuffer> trie2 = makeMemtableTrie(src2, content2, true);
+        InMemoryTrie<ByteBuffer> trie1 = makeInMemoryTrie(src1, content1, true);
+        InMemoryTrie<ByteBuffer> trie2 = makeInMemoryTrie(src2, content2, true);
 
-        addToMemtableTrie(generateKeys(new Random(5), COUNT), content1, trie1, true);
-        addToMemtableTrie(generateKeys(new Random(5), COUNT), content2, trie2, true);
+        addToInMemoryTrie(generateKeys(new Random(5), COUNT), content1, trie1, true);
+        addToInMemoryTrie(generateKeys(new Random(5), COUNT), content2, trie2, true);
 
         content1.putAll(content2);
         Trie<ByteBuffer> union = new CollectionMergeTrie<>(ImmutableList.of(trie1, trie2), x -> x.iterator().next());
@@ -83,12 +83,12 @@ public class CollectionMergeTrieTest
     {
         ByteComparable[] src1 = generateKeys(rand, COUNT);
         SortedMap<ByteComparable, ByteBuffer> content1 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
-        MemtableTrie<ByteBuffer> trie1 = makeMemtableTrie(src1, content1, true);
+        InMemoryTrie<ByteBuffer> trie1 = makeInMemoryTrie(src1, content1, true);
 
         ByteComparable[] src2 = generateKeys(rand, COUNT);
         src2 = removeDuplicates(src2, content1);
         SortedMap<ByteComparable, ByteBuffer> content2 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
-        MemtableTrie<ByteBuffer> trie2 = makeMemtableTrie(src2, content2, true);
+        InMemoryTrie<ByteBuffer> trie2 = makeInMemoryTrie(src2, content2, true);
 
         content1.putAll(content2);
         Trie<ByteBuffer> union = new CollectionMergeTrie.Distinct<>(ImmutableList.of(trie1, trie2));
@@ -149,7 +149,7 @@ public class CollectionMergeTrieTest
         for (int i = 0; i < mergeCount; ++i)
         {
             ByteComparable[] src = removeDuplicates(generateKeys(rand, count), content);
-            Trie<ByteBuffer> trie = makeMemtableTrie(src, content, true);
+            Trie<ByteBuffer> trie = makeInMemoryTrie(src, content, true);
             tries.add(trie);
         }
 
@@ -175,7 +175,7 @@ public class CollectionMergeTrieTest
                     src[j] = keys[randomButNot(rand, mergeCount, i)][rand.nextInt(count)];
             }
 
-            Trie<ByteBuffer> trie = makeMemtableTrie(keys[i], content, true);
+            Trie<ByteBuffer> trie = makeInMemoryTrie(keys[i], content, true);
             tries.add(trie);
         }
 
diff --git a/test/unit/org/apache/cassandra/db/tries/MemtableTrieApplyTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieApplyTest.java
similarity index 93%
rename from test/unit/org/apache/cassandra/db/tries/MemtableTrieApplyTest.java
rename to test/unit/org/apache/cassandra/db/tries/InMemoryTrieApplyTest.java
index 760717293d..4b0b7c42c6 100644
--- a/test/unit/org/apache/cassandra/db/tries/MemtableTrieApplyTest.java
+++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieApplyTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.cassandra.db.tries;
 
-public class MemtableTrieApplyTest extends MemtableTrieTestBase
+public class InMemoryTrieApplyTest extends InMemoryTrieTestBase
 {
     @Override
     boolean usePut()
diff --git a/test/unit/org/apache/cassandra/db/tries/MemtableTriePutTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTriePutTest.java
similarity index 85%
rename from test/unit/org/apache/cassandra/db/tries/MemtableTriePutTest.java
rename to test/unit/org/apache/cassandra/db/tries/InMemoryTriePutTest.java
index dfa18a55ed..51b23d83c7 100644
--- a/test/unit/org/apache/cassandra/db/tries/MemtableTriePutTest.java
+++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTriePutTest.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 
 import static org.junit.Assert.fail;
 
-public class MemtableTriePutTest extends MemtableTrieTestBase
+public class InMemoryTriePutTest extends InMemoryTrieTestBase
 {
     @Override
     boolean usePut()
@@ -39,9 +39,9 @@ public class MemtableTriePutTest extends MemtableTrieTestBase
     }
 
     @Test
-    public void testLongKey_StackOverflow() throws MemtableTrie.SpaceExhaustedException
+    public void testLongKey_StackOverflow() throws InMemoryTrie.SpaceExhaustedException
     {
-        MemtableTrie<String> trie = new MemtableTrie<>(BufferType.ON_HEAP);
+        InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.ON_HEAP);
         Random rand = new Random(1);
         byte[] key = new byte[40960];
         rand.nextBytes(key);
@@ -62,12 +62,12 @@ public class MemtableTriePutTest extends MemtableTrieTestBase
 
     // This tests that trie space allocation works correctly close to the 2G limit. It is normally disabled because
     // the test machines don't provide enough heap memory (test requires ~8G heap to finish). Run it manually when
-    // MemtableTrie.allocateBlock is modified.
+    // InMemoryTrie.allocateBlock is modified.
     @Ignore
     @Test
-    public void testOver1GSize() throws MemtableTrie.SpaceExhaustedException
+    public void testOver1GSize() throws InMemoryTrie.SpaceExhaustedException
     {
-        MemtableTrie<String> trie = new MemtableTrie<>(BufferType.ON_HEAP);
+        InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.ON_HEAP);
         trie.advanceAllocatedPos(0x20000000);
         String t1 = "test1";
         String t2 = "testing2";
@@ -77,7 +77,7 @@ public class MemtableTriePutTest extends MemtableTrieTestBase
         Assert.assertNull(trie.get(ByteComparable.of(t2)));
         Assert.assertFalse(trie.reachedAllocatedSizeThreshold());
 
-        trie.advanceAllocatedPos(MemtableTrie.ALLOCATED_SIZE_THRESHOLD + 0x1000);
+        trie.advanceAllocatedPos(InMemoryTrie.ALLOCATED_SIZE_THRESHOLD + 0x1000);
         trie.putRecursive(ByteComparable.of(t2), t2, (x, y) -> y);
         Assert.assertEquals(t1, trie.get(ByteComparable.of(t1)));
         Assert.assertEquals(t2, trie.get(ByteComparable.of(t2)));
@@ -93,9 +93,9 @@ public class MemtableTriePutTest extends MemtableTrieTestBase
         try
         {
             trie.putRecursive(ByteComparable.of(t3), t3, (x, y) -> y);  // should put it over the edge
-            fail("MemtableTrie.SpaceExhaustedError was expected");
+            fail("InMemoryTrie.SpaceExhaustedError was expected");
         }
-        catch (MemtableTrie.SpaceExhaustedException e)
+        catch (InMemoryTrie.SpaceExhaustedException e)
         {
             // expected
         }
@@ -108,9 +108,9 @@ public class MemtableTriePutTest extends MemtableTrieTestBase
         try
         {
             trie.advanceAllocatedPos(Integer.MAX_VALUE);
-            fail("MemtableTrie.SpaceExhaustedError was expected");
+            fail("InMemoryTrie.SpaceExhaustedError was expected");
         }
-        catch (MemtableTrie.SpaceExhaustedException e)
+        catch (InMemoryTrie.SpaceExhaustedException e)
         {
             // expected
         }
diff --git a/test/unit/org/apache/cassandra/db/tries/MemtableTrieTestBase.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java
similarity index 95%
rename from test/unit/org/apache/cassandra/db/tries/MemtableTrieTestBase.java
rename to test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java
index 251227e0ab..d1c1711423 100644
--- a/test/unit/org/apache/cassandra/db/tries/MemtableTrieTestBase.java
+++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieTestBase.java
@@ -37,7 +37,7 @@ import org.apache.cassandra.utils.ObjectSizes;
 
 import static org.junit.Assert.assertEquals;
 
-public abstract class MemtableTrieTestBase
+public abstract class InMemoryTrieTestBase
 {
     // Set this to true (in combination with smaller count) to dump the tries while debugging a problem.
     // Do not commit the code with VERBOSE = true.
@@ -50,7 +50,7 @@ public abstract class MemtableTrieTestBase
 
     Random rand = new Random();
 
-    static final ByteComparable.Version VERSION = MemtableTrie.BYTE_COMPARABLE_VERSION;
+    static final ByteComparable.Version VERSION = InMemoryTrie.BYTE_COMPARABLE_VERSION;
 
     abstract boolean usePut();
 
@@ -58,7 +58,7 @@ public abstract class MemtableTrieTestBase
     public void testSingle()
     {
         ByteComparable e = ByteComparable.of("test");
-        MemtableTrie<String> trie = new MemtableTrie<>(BufferType.OFF_HEAP);
+        InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.OFF_HEAP);
         putSimpleResolve(trie, e, "test", (x, y) -> y);
         System.out.println("Trie " + trie.dump());
         assertEquals("test", trie.get(e));
@@ -86,7 +86,7 @@ public abstract class MemtableTrieTestBase
         "40bdd47ec043641f2b403131323400",
         "40bd00bf5ae8cf9d1d403133323800",
         };
-        MemtableTrie<String> trie = new MemtableTrie<>(BufferType.OFF_HEAP);
+        InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.OFF_HEAP);
         for (String test : tests)
         {
             ByteComparable e = ByteComparable.fixedLength(ByteBufferUtil.hexToBytes(test));
@@ -116,7 +116,7 @@ public abstract class MemtableTrieTestBase
     {
         String[] tests = new String[] {"testing", "tests", "trials", "trial", "testing", "trial", "trial"};
         String[] values = new String[] {"testing", "tests", "trials", "trial", "t2", "x2", "y2"};
-        MemtableTrie<String> trie = new MemtableTrie<>(BufferType.OFF_HEAP);
+        InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.OFF_HEAP);
         for (int i = 0; i < tests.length; ++i)
         {
             String test = tests[i];
@@ -298,7 +298,7 @@ public abstract class MemtableTrieTestBase
     {
         ByteComparable[] src = generateKeys(rand, COUNT);
         SortedMap<ByteComparable, ByteBuffer> content = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
-        MemtableTrie<ByteBuffer> trie = makeMemtableTrie(src, content, usePut());
+        InMemoryTrie<ByteBuffer> trie = makeInMemoryTrie(src, content, usePut());
         int keysize = Arrays.stream(src)
                             .mapToInt(src1 -> ByteComparable.length(src1, VERSION))
                             .sum();
@@ -395,7 +395,7 @@ public abstract class MemtableTrieTestBase
     private void testEntries(String[] tests, Function<String, ByteComparable> mapping)
 
     {
-        MemtableTrie<String> trie = new MemtableTrie<>(BufferType.OFF_HEAP);
+        InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.OFF_HEAP);
         for (String test : tests)
         {
             ByteComparable e = mapping.apply(test);
@@ -408,19 +408,19 @@ public abstract class MemtableTrieTestBase
             assertEquals(test, trie.get(mapping.apply(test)));
     }
 
-    static MemtableTrie<ByteBuffer> makeMemtableTrie(ByteComparable[] src,
+    static InMemoryTrie<ByteBuffer> makeInMemoryTrie(ByteComparable[] src,
                                                      Map<ByteComparable, ByteBuffer> content,
                                                      boolean usePut)
 
     {
-        MemtableTrie<ByteBuffer> trie = new MemtableTrie<>(BufferType.OFF_HEAP);
-        addToMemtableTrie(src, content, trie, usePut);
+        InMemoryTrie<ByteBuffer> trie = new InMemoryTrie<>(BufferType.OFF_HEAP);
+        addToInMemoryTrie(src, content, trie, usePut);
         return trie;
     }
 
-    static void addToMemtableTrie(ByteComparable[] src,
+    static void addToInMemoryTrie(ByteComparable[] src,
                                   Map<ByteComparable, ByteBuffer> content,
-                                  MemtableTrie<ByteBuffer> trie,
+                                  InMemoryTrie<ByteBuffer> trie,
                                   boolean usePut)
 
     {
@@ -439,7 +439,7 @@ public abstract class MemtableTrieTestBase
         }
     }
 
-    static void checkGet(MemtableTrie<ByteBuffer> trie, Map<ByteComparable, ByteBuffer> items)
+    static void checkGet(InMemoryTrie<ByteBuffer> trie, Map<ByteComparable, ByteBuffer> items)
     {
         for (Map.Entry<ByteComparable, ByteBuffer> en : items.entrySet())
         {
@@ -536,7 +536,7 @@ public abstract class MemtableTrieTestBase
         }
         if (!failedAt.isEmpty())
         {
-            String message = "Failed at " + Lists.transform(failedAt, MemtableTrieTestBase::asString);
+            String message = "Failed at " + Lists.transform(failedAt, InMemoryTrieTestBase::asString);
             System.err.println(message);
             System.err.println(b);
             Assert.fail(message);
@@ -601,7 +601,7 @@ public abstract class MemtableTrieTestBase
         return bc != null ? bc.byteComparableAsString(VERSION) : "null";
     }
 
-    <T, M> void putSimpleResolve(MemtableTrie<T> trie,
+    <T, M> void putSimpleResolve(InMemoryTrie<T> trie,
                                  ByteComparable key,
                                  T value,
                                  Trie.MergeResolver<T> resolver)
@@ -609,7 +609,7 @@ public abstract class MemtableTrieTestBase
         putSimpleResolve(trie, key, value, resolver, usePut());
     }
 
-    static <T, M> void putSimpleResolve(MemtableTrie<T> trie,
+    static <T, M> void putSimpleResolve(InMemoryTrie<T> trie,
                                         ByteComparable key,
                                         T value,
                                         Trie.MergeResolver<T> resolver,
@@ -622,7 +622,7 @@ public abstract class MemtableTrieTestBase
                               (existing, update) -> existing != null ? resolver.resolve(existing, update) : update,
                               usePut);
         }
-        catch (MemtableTrie.SpaceExhaustedException e)
+        catch (InMemoryTrie.SpaceExhaustedException e)
         {
             // Should not happen, test stays well below size limit.
             throw new AssertionError(e);
diff --git a/test/unit/org/apache/cassandra/db/tries/MemtableTrieThreadedTest.java b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java
similarity index 96%
rename from test/unit/org/apache/cassandra/db/tries/MemtableTrieThreadedTest.java
rename to test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java
index 77441c1f56..c8b6ed57f0 100644
--- a/test/unit/org/apache/cassandra/db/tries/MemtableTrieThreadedTest.java
+++ b/test/unit/org/apache/cassandra/db/tries/InMemoryTrieThreadedTest.java
@@ -33,10 +33,10 @@ import org.junit.Test;
 import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 
-import static org.apache.cassandra.db.tries.MemtableTrieTestBase.VERSION;
-import static org.apache.cassandra.db.tries.MemtableTrieTestBase.generateKeys;
+import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.VERSION;
+import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.generateKeys;
 
-public class MemtableTrieThreadedTest
+public class InMemoryTrieThreadedTest
 {
     private static final int COUNT = 300000;
     private static final int OTHERS = COUNT / 10;
@@ -54,7 +54,7 @@ public class MemtableTrieThreadedTest
     public void testThreaded() throws InterruptedException
     {
         ByteComparable[] src = generateKeys(rand, COUNT + OTHERS);
-        MemtableTrie<String> trie = new MemtableTrie<>(BufferType.ON_HEAP);
+        InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.ON_HEAP);
         ConcurrentLinkedQueue<Throwable> errors = new ConcurrentLinkedQueue<>();
         List<Thread> threads = new ArrayList<>();
         AtomicBoolean writeCompleted = new AtomicBoolean(false);
diff --git a/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java b/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java
index 10f28167ab..cc08401102 100644
--- a/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java
+++ b/test/unit/org/apache/cassandra/db/tries/MergeTrieTest.java
@@ -28,7 +28,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 
-import static org.apache.cassandra.db.tries.MemtableTrieTestBase.*;
+import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.*;
 
 public class MergeTrieTest
 {
@@ -43,8 +43,8 @@ public class MergeTrieTest
         SortedMap<ByteComparable, ByteBuffer> content1 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
         SortedMap<ByteComparable, ByteBuffer> content2 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
 
-        MemtableTrie<ByteBuffer> trie1 = makeMemtableTrie(src1, content1, true);
-        MemtableTrie<ByteBuffer> trie2 = makeMemtableTrie(src2, content2, true);
+        InMemoryTrie<ByteBuffer> trie1 = makeInMemoryTrie(src1, content1, true);
+        InMemoryTrie<ByteBuffer> trie2 = makeInMemoryTrie(src2, content2, true);
 
         content1.putAll(content2);
         Trie<ByteBuffer> union = trie1.mergeWith(trie2, (x, y) -> x);
@@ -60,11 +60,11 @@ public class MergeTrieTest
         SortedMap<ByteComparable, ByteBuffer> content1 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
         SortedMap<ByteComparable, ByteBuffer> content2 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
 
-        MemtableTrie<ByteBuffer> trie1 = makeMemtableTrie(src1, content1, true);
-        MemtableTrie<ByteBuffer> trie2 = makeMemtableTrie(src2, content2, true);
+        InMemoryTrie<ByteBuffer> trie1 = makeInMemoryTrie(src1, content1, true);
+        InMemoryTrie<ByteBuffer> trie2 = makeInMemoryTrie(src2, content2, true);
 
-        addToMemtableTrie(generateKeys(new Random(5), COUNT), content1, trie1, true);
-        addToMemtableTrie(generateKeys(new Random(5), COUNT), content2, trie2, true);
+        addToInMemoryTrie(generateKeys(new Random(5), COUNT), content1, trie1, true);
+        addToInMemoryTrie(generateKeys(new Random(5), COUNT), content2, trie2, true);
 
         content1.putAll(content2);
         Trie<ByteBuffer> union = trie1.mergeWith(trie2, (x, y) -> y);
@@ -77,12 +77,12 @@ public class MergeTrieTest
     {
         ByteComparable[] src1 = generateKeys(rand, COUNT);
         SortedMap<ByteComparable, ByteBuffer> content1 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
-        MemtableTrie<ByteBuffer> trie1 = makeMemtableTrie(src1, content1, true);
+        InMemoryTrie<ByteBuffer> trie1 = makeInMemoryTrie(src1, content1, true);
 
         ByteComparable[] src2 = generateKeys(rand, COUNT);
         src2 = removeDuplicates(src2, content1);
         SortedMap<ByteComparable, ByteBuffer> content2 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, VERSION));
-        MemtableTrie<ByteBuffer> trie2 = makeMemtableTrie(src2, content2, true);
+        InMemoryTrie<ByteBuffer> trie2 = makeInMemoryTrie(src2, content2, true);
 
         content1.putAll(content2);
         Trie<ByteBuffer> union = new MergeTrie.Distinct<>(trie1, trie2);
diff --git a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java
index 478f9ce78c..3cabc0924d 100644
--- a/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java
+++ b/test/unit/org/apache/cassandra/db/tries/SlicedTrieTest.java
@@ -35,10 +35,10 @@ import org.junit.Test;
 import com.googlecode.concurrenttrees.common.Iterables;
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 
-import static org.apache.cassandra.db.tries.MemtableTrieTestBase.asString;
-import static org.apache.cassandra.db.tries.MemtableTrieTestBase.assertSameContent;
-import static org.apache.cassandra.db.tries.MemtableTrieTestBase.generateKeys;
-import static org.apache.cassandra.db.tries.MemtableTrieTestBase.makeMemtableTrie;
+import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.asString;
+import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.assertSameContent;
+import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.generateKeys;
+import static org.apache.cassandra.db.tries.InMemoryTrieTestBase.makeInMemoryTrie;
 import static java.util.Arrays.asList;
 import static org.junit.Assert.assertEquals;
 
@@ -97,15 +97,15 @@ public class SlicedTrieTest
         ByteComparable[] src1 = generateKeys(rand, count);
         NavigableMap<ByteComparable, ByteBuffer> content1 = new TreeMap<>((bytes1, bytes2) -> ByteComparable.compare(bytes1, bytes2, Trie.BYTE_COMPARABLE_VERSION));
 
-        MemtableTrie<ByteBuffer> trie1 = makeMemtableTrie(src1, content1, true);
+        InMemoryTrie<ByteBuffer> trie1 = makeInMemoryTrie(src1, content1, true);
 
         checkEqualRange(content1, trie1, null, true, null, true);
-        checkEqualRange(content1, trie1, MemtableTrieTestBase.generateKey(rand), true, null, true);
-        checkEqualRange(content1, trie1, null, true, MemtableTrieTestBase.generateKey(rand), true);
+        checkEqualRange(content1, trie1, InMemoryTrieTestBase.generateKey(rand), true, null, true);
+        checkEqualRange(content1, trie1, null, true, InMemoryTrieTestBase.generateKey(rand), true);
         for (int i = 0; i < 4; ++i)
         {
-            ByteComparable l = rand.nextBoolean() ? MemtableTrieTestBase.generateKey(rand) : src1[rand.nextInt(src1.length)];
-            ByteComparable r = rand.nextBoolean() ? MemtableTrieTestBase.generateKey(rand) : src1[rand.nextInt(src1.length)];
+            ByteComparable l = rand.nextBoolean() ? InMemoryTrieTestBase.generateKey(rand) : src1[rand.nextInt(src1.length)];
+            ByteComparable r = rand.nextBoolean() ? InMemoryTrieTestBase.generateKey(rand) : src1[rand.nextInt(src1.length)];
             int cmp = ByteComparable.compare(l, r, Trie.BYTE_COMPARABLE_VERSION);
             if (cmp > 0)
             {
@@ -180,7 +180,7 @@ public class SlicedTrieTest
     {
         Arrays.sort(BOUNDARIES, BYTE_COMPARABLE_COMPARATOR);
         NavigableMap<ByteComparable, ByteBuffer> content1 = new TreeMap<>(BYTE_COMPARABLE_COMPARATOR);
-        MemtableTrie<ByteBuffer> trie1 = makeMemtableTrie(KEYS, content1, true);
+        InMemoryTrie<ByteBuffer> trie1 = makeInMemoryTrie(KEYS, content1, true);
 
         for (int li = -1; li < BOUNDARIES.length; ++li)
         {
@@ -225,7 +225,7 @@ public class SlicedTrieTest
         List<Trie<ByteBuffer>> tries = new ArrayList<>();
         for (int i = 0; i < mergeCount; ++i)
         {
-            tries.add(makeMemtableTrie(Arrays.copyOfRange(KEYS,
+            tries.add(makeInMemoryTrie(Arrays.copyOfRange(KEYS,
                                                            KEYS.length * i / mergeCount,
                                                            KEYS.length * (i + 1) / mergeCount),
                                         content1,
diff --git a/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java b/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java
index 5dcc162ee6..41c66aaaeb 100644
--- a/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java
+++ b/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java
@@ -27,12 +27,12 @@ public class TrieToDotTest
     @Test
     public void testToDotContent() throws Exception
     {
-        MemtableTrie<String> trie = new MemtableTrie<>(BufferType.OFF_HEAP);
+        InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.OFF_HEAP);
         String s = "Trie node types and manipulation mechanisms. The main purpose of this is to allow for handling tries directly as" +
                    " they are on disk without any serialization, and to enable the creation of such files.";
         s = s.toLowerCase();
         for (String word : s.split("[^a-z]+"))
-            trie.putRecursive(MemtableTrieTestBase.comparable(word), word, (x, y) -> y);
+            trie.putRecursive(InMemoryTrieTestBase.comparable(word), word, (x, y) -> y);
 
         System.out.println(trie.process(new TrieToDot(Object::toString,
                                                       x -> Character.toString((char) ((int) x)),
diff --git a/test/unit/org/apache/cassandra/metrics/TrieMemtableMetricsTest.java b/test/unit/org/apache/cassandra/metrics/TrieMemtableMetricsTest.java
index b92d2bdc51..b2018498ec 100644
--- a/test/unit/org/apache/cassandra/metrics/TrieMemtableMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/TrieMemtableMetricsTest.java
@@ -138,7 +138,7 @@ public class TrieMemtableMetricsTest extends SchemaLoader
 
     @Test
     @BMRules(rules = { @BMRule(name = "Delay memtable update",
-    targetClass = "MemtableTrie",
+    targetClass = "InMemoryTrie",
     targetMethod = "putSingleton",
     action = "java.lang.Thread.sleep(10)")})
     public void testContentionMetrics() throws IOException, ExecutionException, InterruptedException


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