Numeric on-disk index write and search

Includes:
  - The disk/v1/kdtree package containing the
kdtree writer and reader
  - The implementation code to tie these into
the existing read and write paths. The main parts
of this are the NumericIndexWriter and the
NumericIndexSegmentSearcher
  - Additional testing for the new code

patch by Mike Adamson; reviewed by Caleb Rackliffe and Andres de la Peña for CASSANDRA-18067

Co-authored-by: Mike Adamson <madamson@datastax.com>
Co-authored-by: Caleb Rackliffe <calebrackliffe@gmail.com>
Co-authored-by: Piotr Kołaczkowski <pkolaczk@gmail.com>
Co-authored-by: Jason Rutherglen <jason.rutherglen@gmail.com>
Co-authored-by: Zhao Yang <zhaoyangsingapore@gmail.com>
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
index 09c5535..5d5c142 100644
--- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
@@ -419,11 +419,23 @@
     // SAI specific properties
 
     /** Controls the maximum number of index query intersections that will take part in a query */
-    SAI_INTERSECTION_CLAUSE_LIMIT("cassandra.sai.intersection.clause.limit", "2"),
+    SAI_INTERSECTION_CLAUSE_LIMIT("cassandra.sai.intersection_clause_limit", "2"),
     /** Latest version to be used for SAI index writing */
     SAI_LATEST_VERSION("cassandra.sai.latest_version", "aa"),
     SAI_MAX_FROZEN_TERM_SIZE("cassandra.sai.max_frozen_term_size_kb", "5"),
     SAI_MAX_STRING_TERM_SIZE("cassandra.sai.max_string_term_size_kb", "1"),
+
+    /** Minimum number of reachable leaves for a given node to be eligible for an auxiliary posting list */
+    SAI_MINIMUM_POSTINGS_LEAVES("cassandra.sai.minimum_postings_leaves", "64"),
+
+    /**
+     * Skip, or the sampling interval, for selecting a balanced tree level that is eligible for an auxiliary posting list.
+     * Sampling starts from 0, but balanced tree root node is at level 1. For skip = 4, eligible levels are 4, 8, 12, etc. (no
+     * level 0, because there is no node at level 0).
+     */
+    SAI_POSTINGS_SKIP("cassandra.sai.postings_skip", "3"),
+
+    SAI_TEST_BALANCED_TREE_DEBUG_ENABLED("cassandra.sai.test.balanced_tree_debug_enabled", "false"),
     SAI_TEST_DISABLE_TIMEOUT("cassandra.sai.test.disable.timeout", "false"),
 
     SCHEMA_PULL_INTERVAL_MS("cassandra.schema_pull_interval_ms", "60000"),
diff --git a/src/java/org/apache/cassandra/db/ClusteringComparator.java b/src/java/org/apache/cassandra/db/ClusteringComparator.java
index d007c5d..2949130 100644
--- a/src/java/org/apache/cassandra/db/ClusteringComparator.java
+++ b/src/java/org/apache/cassandra/db/ClusteringComparator.java
@@ -353,6 +353,8 @@
     {
         ByteComparable.Version version = ByteComparable.Version.OSS50;
         ByteSource.Peekable orderedBytes = ByteSource.peekable(comparable.asComparableBytes(version));
+        if (orderedBytes == null)
+            return null;
 
         // First check for special cases (partition key only, static clustering) that can do without buffers.
         int sep = orderedBytes.next();
diff --git a/src/java/org/apache/cassandra/index/sai/IndexContext.java b/src/java/org/apache/cassandra/index/sai/IndexContext.java
index 032f289..f166073 100644
--- a/src/java/org/apache/cassandra/index/sai/IndexContext.java
+++ b/src/java/org/apache/cassandra/index/sai/IndexContext.java
@@ -113,9 +113,11 @@
 
         this.indexMetadata = indexMetadata;
         this.memtableIndexManager = indexMetadata == null ? null : new MemtableIndexManager(this);
+
         this.indexMetrics = indexMetadata == null ? null : new IndexMetrics(this);
         this.viewManager = new IndexViewManager(this);
-        this.columnQueryMetrics = new ColumnQueryMetrics.TrieIndexMetrics(this);
+        this.columnQueryMetrics = isLiteral() ? new ColumnQueryMetrics.TrieIndexMetrics(this)
+                                              : new ColumnQueryMetrics.BalancedTreeIndexMetrics(this);
 
         // We currently only support the NoOpAnalyzer
         this.indexAnalyzerFactory = AbstractAnalyzer.fromOptions(getValidator(), Collections.emptyMap());
diff --git a/src/java/org/apache/cassandra/index/sai/QueryContext.java b/src/java/org/apache/cassandra/index/sai/QueryContext.java
index 6bb85df..e9db787 100644
--- a/src/java/org/apache/cassandra/index/sai/QueryContext.java
+++ b/src/java/org/apache/cassandra/index/sai/QueryContext.java
@@ -48,10 +48,14 @@
     public long rowsFiltered = 0;
 
     public long trieSegmentsHit = 0;
-
     public long triePostingsSkips = 0;
     public long triePostingsDecodes = 0;
 
+    public long balancedTreePostingListsHit = 0;
+    public long balancedTreeSegmentsHit = 0;
+    public long balancedTreePostingsSkips = 0;
+    public long balancedTreePostingsDecodes = 0;
+
     public boolean queryTimedOut = false;
 
     public QueryContext(ReadCommand readCommand, long executionQuotaMs)
diff --git a/src/java/org/apache/cassandra/index/sai/README.md b/src/java/org/apache/cassandra/index/sai/README.md
index 45cdfee..6be0fa9 100644
--- a/src/java/org/apache/cassandra/index/sai/README.md
+++ b/src/java/org/apache/cassandra/index/sai/README.md
@@ -25,8 +25,8 @@
 architectural character (and even some actual code), but makes significant improvements in a number of areas:
 
 - The on-disk/SSTable index formats for both string and numeric data have been completely replaced. Strings are indexed
-  on disk using our proprietary on-disk byte-ordered trie data structure, while numeric types are indexed using Lucene's
-  balanced kd-tree.
+  on disk using our proprietary on-disk byte-ordered trie data structure, while numeric types are indexed using a 
+  balanced tree.
 - While indexes continue to be managed at the column level from the user's perspective, the storage design at the column
   index level is row-based, with related offset and token information stored only once at the SSTable level. This
   drastically reduces our on-disk footprint when several columns are indexed on the same table.
diff --git a/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponent.java b/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponent.java
index 4d802cc..b1b2b4a 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponent.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponent.java
@@ -39,6 +39,12 @@
     META("Meta"),
 
     /**
+     * Balanced tree written by {@code BlockBalancedTreeWriter} indexes mappings of term to one or more segment row IDs
+     * (segment row ID = SSTable row ID - segment row ID offset).
+     */
+    BALANCED_TREE("BalancedTree"),
+
+    /**
      * Term dictionary written by {@link TrieTermsDictionaryWriter} stores mappings of term and
      * file pointer to posting block on posting file.
      */
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/MemtableIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/MemtableIndexWriter.java
index 6a497fb..da57ac2 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/MemtableIndexWriter.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/MemtableIndexWriter.java
@@ -35,6 +35,8 @@
 import org.apache.cassandra.index.sai.disk.RowMapping;
 import org.apache.cassandra.index.sai.disk.format.IndexComponent;
 import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.BlockBalancedTreeIterator;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.NumericIndexWriter;
 import org.apache.cassandra.index.sai.disk.v1.segment.SegmentMetadata;
 import org.apache.cassandra.index.sai.disk.v1.trie.LiteralIndexWriter;
 import org.apache.cassandra.index.sai.memory.MemtableIndex;
@@ -110,7 +112,7 @@
 
             try (MemtableTermsIterator terms = new MemtableTermsIterator(memtable.getMinTerm(), memtable.getMaxTerm(), iterator))
             {
-                long cellCount = flush(minKey, maxKey, indexContext.getValidator(), terms);
+                long cellCount = flush(minKey, maxKey, indexContext.getValidator(), terms, rowMapping.maxSSTableRowId);
 
                 indexDescriptor.createComponentOnDisk(IndexComponent.COLUMN_COMPLETION_MARKER, indexContext);
 
@@ -136,10 +138,11 @@
     private long flush(DecoratedKey minKey,
                        DecoratedKey maxKey,
                        AbstractType<?> termComparator,
-                       MemtableTermsIterator terms) throws IOException
+                       MemtableTermsIterator terms,
+                       long maxSSTableRowId) throws IOException
     {
-        long numRows = 0;
-        SegmentMetadata.ComponentMetadataMap indexMetas = null;
+        long numRows;
+        SegmentMetadata.ComponentMetadataMap indexMetas;
 
         if (TypeUtil.isLiteral(termComparator))
         {
@@ -149,6 +152,15 @@
                 numRows = writer.getPostingsCount();
             }
         }
+        else
+        {
+            NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+                                                               indexContext,
+                                                               TypeUtil.fixedSizeOf(termComparator),
+                                                               maxSSTableRowId);
+            indexMetas = writer.writeCompleteSegment(BlockBalancedTreeIterator.fromTermsIterator(terms, termComparator));
+            numRows = writer.getValueCount();
+        }
 
         // If no rows were written we need to delete any created column index components
         // so that the index is correctly identified as being empty (only having a completion marker)
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/PerColumnIndexFiles.java b/src/java/org/apache/cassandra/index/sai/disk/v1/PerColumnIndexFiles.java
index 7d129ba..37c0bbd 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/PerColumnIndexFiles.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/PerColumnIndexFiles.java
@@ -25,7 +25,6 @@
 import org.apache.cassandra.index.sai.IndexContext;
 import org.apache.cassandra.index.sai.disk.format.IndexComponent;
 import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
-import org.apache.cassandra.index.sai.utils.TypeUtil;
 import org.apache.cassandra.io.util.FileHandle;
 import org.apache.cassandra.io.util.FileUtils;
 
@@ -45,11 +44,16 @@
     {
         this.indexDescriptor = indexDescriptor;
         this.indexContext = indexContext;
-        if (TypeUtil.isLiteral(indexContext.getValidator()))
+        if (indexContext.isLiteral())
         {
             files.put(IndexComponent.POSTING_LISTS, indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext));
             files.put(IndexComponent.TERMS_DATA, indexDescriptor.createPerIndexFileHandle(IndexComponent.TERMS_DATA, indexContext));
         }
+        else
+        {
+            files.put(IndexComponent.BALANCED_TREE, indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext));
+            files.put(IndexComponent.POSTING_LISTS, indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext));
+        }
     }
 
     public FileHandle termsData()
@@ -62,6 +66,11 @@
         return getFile(IndexComponent.POSTING_LISTS);
     }
 
+    public FileHandle balancedTree()
+    {
+        return getFile(IndexComponent.BALANCED_TREE);
+    }
+
     @SuppressWarnings({"resource", "RedundantSuppression"})
     private FileHandle getFile(IndexComponent indexComponent)
     {
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/RowAwarePrimaryKeyMap.java b/src/java/org/apache/cassandra/index/sai/disk/v1/RowAwarePrimaryKeyMap.java
index 1319006..1f9c853 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/RowAwarePrimaryKeyMap.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/RowAwarePrimaryKeyMap.java
@@ -188,6 +188,9 @@
                                        : clusteringComparator.clusteringFromByteComparable(ByteBufferAccessor.instance,
                                                                                            v -> ByteSourceInverse.nextComponentSource(peekable));
 
+            if (clustering == null)
+                clustering = Clustering.EMPTY;
+
             return primaryKeyFactory.create(partitionKey, clustering);
         }
         catch (IOException e)
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java
index eeb94d1..9727816 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java
@@ -327,7 +327,9 @@
 
     private SegmentBuilder newSegmentBuilder()
     {
-        SegmentBuilder builder = new SegmentBuilder.RAMStringSegmentBuilder(indexContext.getValidator(), limiter);
+        SegmentBuilder builder = TypeUtil.isLiteral(indexContext.getValidator())
+                                 ? new SegmentBuilder.RAMStringSegmentBuilder(indexContext.getValidator(), limiter)
+                                 : new SegmentBuilder.BlockBalancedTreeSegmentBuilder(indexContext.getValidator(), limiter);
 
         long globalBytesUsed = limiter.increment(builder.totalBytesAllocated());
         logger.debug(indexContext.logMessage("Created new segment builder while flushing SSTable {}. Global segment memory usage now at {}."),
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/V1OnDiskFormat.java b/src/java/org/apache/cassandra/index/sai/disk/v1/V1OnDiskFormat.java
index 189d9b6..830712b 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/V1OnDiskFormat.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/V1OnDiskFormat.java
@@ -57,16 +57,21 @@
 
     @VisibleForTesting
     public static final Set<IndexComponent> PER_SSTABLE_COMPONENTS = EnumSet.of(IndexComponent.GROUP_COMPLETION_MARKER,
-                                                                                 IndexComponent.GROUP_META,
-                                                                                 IndexComponent.TOKEN_VALUES,
-                                                                                 IndexComponent.PRIMARY_KEY_TRIE,
-                                                                                 IndexComponent.PRIMARY_KEY_BLOCKS,
-                                                                                 IndexComponent.PRIMARY_KEY_BLOCK_OFFSETS);
+                                                                                IndexComponent.GROUP_META,
+                                                                                IndexComponent.TOKEN_VALUES,
+                                                                                IndexComponent.PRIMARY_KEY_TRIE,
+                                                                                IndexComponent.PRIMARY_KEY_BLOCKS,
+                                                                                IndexComponent.PRIMARY_KEY_BLOCK_OFFSETS);
     @VisibleForTesting
     public static final Set<IndexComponent> LITERAL_COMPONENTS = EnumSet.of(IndexComponent.COLUMN_COMPLETION_MARKER,
-                                                                             IndexComponent.META,
-                                                                             IndexComponent.TERMS_DATA,
-                                                                             IndexComponent.POSTING_LISTS);
+                                                                            IndexComponent.META,
+                                                                            IndexComponent.TERMS_DATA,
+                                                                            IndexComponent.POSTING_LISTS);
+    @VisibleForTesting
+    public static final Set<IndexComponent> NUMERIC_COMPONENTS = EnumSet.of(IndexComponent.COLUMN_COMPLETION_MARKER,
+                                                                            IndexComponent.META,
+                                                                            IndexComponent.BALANCED_TREE,
+                                                                            IndexComponent.POSTING_LISTS);
 
     /**
      * Global limit on heap consumed by all index segment building that occurs outside the context of Memtable flush.
@@ -225,7 +230,7 @@
     @Override
     public Set<IndexComponent> perColumnIndexComponents(IndexContext indexContext)
     {
-        return LITERAL_COMPONENTS;
+        return indexContext.isLiteral() ? LITERAL_COMPONENTS : NUMERIC_COMPONENTS;
     }
 
     @Override
@@ -239,7 +244,8 @@
     @Override
     public int openFilesPerColumnIndex(IndexContext indexContext)
     {
-        // For the V1 format there are always 2 open files per index - index (kdtree or terms) + postings
+        // For the V1 format there are always 2 open files per index - index (balanced tree or terms) + auxiliary postings
+        // for the balanced tree and postings for the literal terms
         return 2;
     }
 
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIterator.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIterator.java
new file mode 100644
index 0000000..a650784
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIterator.java
@@ -0,0 +1,104 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.TermsIterator;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+/**
+ * A helper class for mapping disparate iterators to a single iterator for consumption by the
+ * {@link BlockBalancedTreeWriter}
+ */
+public interface BlockBalancedTreeIterator extends Iterator<Pair<byte[], PostingList>>
+{
+    static BlockBalancedTreeIterator fromTermsIterator(final TermsIterator termsIterator, AbstractType<?> termsComparator)
+    {
+        return new BlockBalancedTreeIterator()
+        {
+            final byte[] scratch = new byte[TypeUtil.fixedSizeOf(termsComparator)];
+
+            @Override
+            public boolean hasNext()
+            {
+                return termsIterator.hasNext();
+            }
+
+            @Override
+            public Pair<byte[], PostingList> next()
+            {
+                ByteComparable term = termsIterator.next();
+                ByteSourceInverse.copyBytes(term.asComparableBytes(ByteComparable.Version.OSS50), scratch);
+                return Pair.create(scratch, termsIterator.postings());
+            }
+        };
+    }
+
+    static BlockBalancedTreeIterator fromTrieIterator(Iterator<Map.Entry<ByteComparable, PackedLongValues.Builder>> iterator, int bytesPerValue)
+    {
+        return new BlockBalancedTreeIterator()
+        {
+            final byte[] scratch = new byte[bytesPerValue];
+
+            @Override
+            public boolean hasNext()
+            {
+                return iterator.hasNext();
+            }
+
+            @Override
+            public Pair<byte[], PostingList> next()
+            {
+                Map.Entry<ByteComparable, PackedLongValues.Builder> entry = iterator.next();
+                ByteSourceInverse.copyBytes(entry.getKey().asComparableBytes(ByteComparable.Version.OSS50), scratch);
+                PackedLongValues postings = entry.getValue().build();
+                PackedLongValues.Iterator postingsIterator = postings.iterator();
+                return Pair.create(scratch, new PostingList() {
+                    @Override
+                    public long nextPosting()
+                    {
+                        if (postingsIterator.hasNext())
+                            return postingsIterator.next();
+                        return END_OF_STREAM;
+                    }
+
+                    @Override
+                    public long size()
+                    {
+                        return postings.size();
+                    }
+
+                    @Override
+                    public long advance(long targetRowID)
+                    {
+                        throw new UnsupportedOperationException();
+                    }
+                });
+            }
+        };
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsIndex.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsIndex.java
new file mode 100644
index 0000000..4b87f46
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsIndex.java
@@ -0,0 +1,83 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.io.IOException;
+
+import com.carrotsearch.hppc.IntLongHashMap;
+import com.carrotsearch.hppc.IntLongMap;
+import org.apache.cassandra.index.sai.disk.io.IndexInputReader;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+
+import static org.apache.cassandra.index.sai.disk.v1.SAICodecUtils.validate;
+
+/**
+ * Mapping between node ID and an offset to its auxiliary posting list (containing every row id from all leaves
+ * reachable from that node. See {@link BlockBalancedTreePostingsWriter}).
+ */
+class BlockBalancedTreePostingsIndex
+{
+    private final int size;
+    public final IntLongMap index = new IntLongHashMap();
+
+    BlockBalancedTreePostingsIndex(FileHandle postingsFileHandle, long filePosition) throws IOException
+    {
+        try (RandomAccessReader reader = postingsFileHandle.createReader();
+             IndexInputReader input = IndexInputReader.create(reader))
+        {
+            validate(input);
+            input.seek(filePosition);
+
+            size = input.readVInt();
+
+            for (int x = 0; x < size; x++)
+            {
+                final int node = input.readVInt();
+                final long filePointer = input.readVLong();
+
+                index.put(node, filePointer);
+            }
+        }
+    }
+
+    /**
+     * Returns <tt>true</tt> if given node ID has an auxiliary posting list.
+     */
+    boolean exists(int nodeID)
+    {
+        return index.containsKey(nodeID);
+    }
+
+    /**
+     * Returns an offset within the balanced tree postings file to the begining of the blocks summary of given node's auxiliary
+     * posting list.
+     *
+     * @throws IllegalArgumentException when given nodeID doesn't have an auxiliary posting list. Check first with
+     * {@link #exists(int)}
+     */
+    long getPostingsFilePointer(int nodeID)
+    {
+        return index.get(nodeID);
+    }
+
+    int size()
+    {
+        return size;
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriter.java
new file mode 100644
index 0000000..6ba6894
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriter.java
@@ -0,0 +1,214 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.agrona.collections.IntArrayList;
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.disk.io.IndexOutputWriter;
+import org.apache.cassandra.index.sai.disk.v1.postings.MergePostingList;
+import org.apache.cassandra.index.sai.disk.v1.postings.PackedLongsPostingList;
+import org.apache.cassandra.index.sai.disk.v1.postings.PostingsWriter;
+import org.apache.cassandra.index.sai.postings.PeekablePostingList;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+/**
+ * Writes leaf postings and auxiliary posting lists for bbtree nodes. If a node has a posting list attached,
+ * it will contain every row id from all leaves reachable from that node.
+ * <p>
+ * Writer is stateful, because it needs to collect data from the balanced tree data structure first to find set of eligible
+ * nodes and leaf nodes reachable from them.
+ * <p>
+ * The leaf blocks are written in value order (in the order we pass them to the {@link BlockBalancedTreeWriter}).
+ * This allows us to skip reading the leaves, instead just order leaf blocks by their offset in the index file,
+ * and correlate them with buffered posting lists.
+ */
+@NotThreadSafe
+public class BlockBalancedTreePostingsWriter implements BlockBalancedTreeWalker.TraversalCallback
+{
+    private static final Logger logger = LoggerFactory.getLogger(BlockBalancedTreePostingsWriter.class);
+
+    private final TreeMap<Long, Integer> leafOffsetToNodeID = new TreeMap<>(Long::compareTo);
+    private final Multimap<Integer, Integer> nodeToChildLeaves = HashMultimap.create();
+
+    /**
+     * Minimum number of reachable leaves for a given node to be eligible for an auxiliary posting list.
+     */
+    private final int minimumPostingsLeaves;
+    /**
+     * Skip, or the sampling interval, for selecting a balanced tree level that is eligible for an auxiliary posting list.
+     * Sampling starts from 0, but the balanced tree root node is at level 1. For skip = 4, eligible levels are 4, 8, 12, etc. (no
+     * level 0, because there is no node at level 0).
+     */
+    private final int postingsSkip;
+
+    int numNonLeafPostings = 0;
+    int numLeafPostings = 0;
+
+    public BlockBalancedTreePostingsWriter()
+    {
+        minimumPostingsLeaves = CassandraRelevantProperties.SAI_MINIMUM_POSTINGS_LEAVES.getInt();
+        postingsSkip = CassandraRelevantProperties.SAI_POSTINGS_SKIP.getInt();
+    }
+
+    /**
+     * Called when a leaf node is hit as we traverse the packed index.
+     *
+     * @param leafNodeID the current leaf node ID in the packed inded
+     * @param leafBlockFP the file pointer to the on-disk leaf block
+     * @param pathToRoot the path to the root leaf above this leaf. Contains all the intermediate leaf node IDs.
+     */
+    @Override
+    public void onLeaf(int leafNodeID, long leafBlockFP, IntArrayList pathToRoot)
+    {
+        checkArgument(!pathToRoot.containsInt(leafNodeID));
+        checkArgument(pathToRoot.isEmpty() || leafNodeID > pathToRoot.get(pathToRoot.size() - 1));
+
+        leafOffsetToNodeID.put(leafBlockFP, leafNodeID);
+        for (int i = 0; i < pathToRoot.size(); i++)
+        {
+            int level = i + 1;
+            if (isLevelEligibleForPostingList(level))
+            {
+                int nodeID = pathToRoot.get(i);
+                nodeToChildLeaves.put(nodeID, leafNodeID);
+            }
+        }
+    }
+
+    /**
+     * Writes merged posting lists for eligible internal nodes and leaf postings for each leaf in the tree.
+     * The merged postings list for an internal node contains all postings from the postings lists of leaf nodes
+     * in the subtree rooted at that node.
+     * <p>
+     * After writing out the postings, it writes a map of node ID -> postings file pointer for all
+     * nodes with an attached postings list. It then returns the file pointer to this map.
+     */
+    public long finish(IndexOutputWriter out, List<PackedLongValues> leafPostings, IndexContext indexContext) throws IOException
+    {
+        checkState(leafPostings.size() == leafOffsetToNodeID.size(),
+                   "Expected equal number of postings lists (%s) and leaf offsets (%s).",
+                   leafPostings.size(), leafOffsetToNodeID.size());
+
+        try (PostingsWriter postingsWriter = new PostingsWriter(out))
+        {
+            Iterator<PackedLongValues> postingsIterator = leafPostings.iterator();
+            Map<Integer, PackedLongValues> leafToPostings = new HashMap<>();
+            leafOffsetToNodeID.forEach((fp, nodeID) -> leafToPostings.put(nodeID, postingsIterator.next()));
+
+            long postingsRamBytesUsed = leafPostings.stream()
+                                                .mapToLong(PackedLongValues::ramBytesUsed)
+                                                .sum();
+
+            List<Integer> internalNodeIDs = nodeToChildLeaves.keySet()
+                                                             .stream()
+                                                             .filter(i -> nodeToChildLeaves.get(i).size() >= minimumPostingsLeaves)
+                                                             .collect(Collectors.toList());
+
+            Collection<Integer> leafNodeIDs = leafOffsetToNodeID.values();
+
+            logger.debug(indexContext.logMessage("Writing posting lists for {} internal and {} leaf balanced tree nodes. Leaf postings memory usage: {}."),
+                         internalNodeIDs.size(), leafNodeIDs.size(), FBUtilities.prettyPrintMemory(postingsRamBytesUsed));
+
+            long startFP = out.getFilePointer();
+            Stopwatch flushTime = Stopwatch.createStarted();
+            TreeMap<Integer, Long> nodeIDToPostingsFilePointer = new TreeMap<>();
+            PriorityQueue<PeekablePostingList> postingLists = new PriorityQueue<>(minimumPostingsLeaves, Comparator.comparingLong(PeekablePostingList::peek));
+            for (int nodeID : Iterables.concat(internalNodeIDs, leafNodeIDs))
+            {
+                Collection<Integer> leaves = nodeToChildLeaves.get(nodeID);
+
+                if (leaves.isEmpty())
+                {
+                    leaves = Collections.singletonList(nodeID);
+                    numLeafPostings++;
+                }
+                else
+                {
+                    numNonLeafPostings++;
+                }
+
+                for (Integer leaf : leaves)
+                    postingLists.add(PeekablePostingList.makePeekable(new PackedLongsPostingList(leafToPostings.get(leaf))));
+
+                try (PostingList mergedPostingList = MergePostingList.merge(postingLists))
+                {
+                    long postingFilePosition = postingsWriter.write(mergedPostingList);
+                    // During compaction, we could end up with an empty postings due to deletions.
+                    // The writer will return a fp of -1 if no postings were written.
+                    if (postingFilePosition >= 0)
+                        nodeIDToPostingsFilePointer.put(nodeID, postingFilePosition);
+                }
+                postingLists.clear();
+            }
+            flushTime.stop();
+            logger.debug(indexContext.logMessage("Flushed {} of posting lists for balanced tree nodes in {} ms."),
+                         FBUtilities.prettyPrintMemory(out.getFilePointer() - startFP),
+                         flushTime.elapsed(TimeUnit.MILLISECONDS));
+
+            long indexFilePointer = out.getFilePointer();
+            writeMap(nodeIDToPostingsFilePointer, out);
+            postingsWriter.complete();
+            return indexFilePointer;
+        }
+    }
+
+    private boolean isLevelEligibleForPostingList(int level)
+    {
+        return level > 1 && level % postingsSkip == 0;
+    }
+
+    private void writeMap(Map<Integer, Long> map, IndexOutput out) throws IOException
+    {
+        out.writeVInt(map.size());
+
+        for (Map.Entry<Integer, Long> e : map.entrySet())
+        {
+            out.writeVInt(e.getKey());
+            out.writeVLong(e.getValue());
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueries.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueries.java
new file mode 100644
index 0000000..483fa5a
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueries.java
@@ -0,0 +1,161 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.lucene.index.PointValues.Relation;
+
+public class BlockBalancedTreeQueries
+{
+    private static final BlockBalancedTreeReader.IntersectVisitor MATCH_ALL = new BlockBalancedTreeReader.IntersectVisitor()
+    {
+        @Override
+        public boolean contains(byte[] packedValue)
+        {
+            return true;
+        }
+
+        @Override
+        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+        {
+            return Relation.CELL_INSIDE_QUERY;
+        }
+    };
+
+    public static BlockBalancedTreeReader.IntersectVisitor balancedTreeQueryFrom(Expression expression, int bytesPerValue)
+    {
+        if (expression.lower == null && expression.upper == null)
+        {
+            return MATCH_ALL;
+        }
+
+        Bound lower = null ;
+        if (expression.lower != null)
+        {
+            final byte[] lowerBound = toComparableBytes(bytesPerValue, expression.lower.value.encoded, expression.validator);
+            lower = new Bound(lowerBound, !expression.lower.inclusive);
+        }
+
+        Bound upper = null;
+        if (expression.upper != null)
+        {
+            final byte[] upperBound = toComparableBytes(bytesPerValue, expression.upper.value.encoded, expression.validator);
+            upper = new Bound(upperBound, !expression.upper.inclusive);
+        }
+
+        return new RangeQueryVisitor(lower, upper);
+    }
+
+    private static byte[] toComparableBytes(int bytesPerDim, ByteBuffer value, AbstractType<?> type)
+    {
+        byte[] buffer = new byte[TypeUtil.fixedSizeOf(type)];
+        assert buffer.length == bytesPerDim;
+        TypeUtil.toComparableBytes(value, type, buffer);
+        return buffer;
+    }
+
+    private static class Bound
+    {
+        private final byte[] bound;
+        private final boolean exclusive;
+
+        Bound(byte[] bound, boolean exclusive)
+        {
+            this.bound = bound;
+            this.exclusive = exclusive;
+        }
+
+        boolean smallerThan(byte[] packedValue)
+        {
+            int cmp = compareTo(packedValue);
+            return cmp < 0 || (cmp == 0 && exclusive);
+        }
+
+        boolean greaterThan(byte[] packedValue)
+        {
+            int cmp = compareTo(packedValue);
+            return cmp > 0 || (cmp == 0 && exclusive);
+        }
+
+        private int compareTo(byte[] packedValue)
+        {
+            return ByteArrayUtil.compareUnsigned(bound, 0, packedValue, 0, bound.length);
+        }
+    }
+
+    private static class RangeQueryVisitor implements BlockBalancedTreeReader.IntersectVisitor
+    {
+        private final Bound lower;
+        private final Bound upper;
+
+        private RangeQueryVisitor(Bound lower, Bound upper)
+        {
+            this.lower = lower;
+            this.upper = upper;
+        }
+
+        @Override
+        public boolean contains(byte[] packedValue)
+        {
+            if (lower != null)
+            {
+                if (lower.greaterThan(packedValue))
+                {
+                    // value is too low, in this dimension
+                    return false;
+                }
+            }
+
+            if (upper != null)
+            {
+                return !upper.smallerThan(packedValue);
+            }
+
+            return true;
+        }
+
+        @Override
+        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+        {
+            boolean crosses = false;
+
+            if (lower != null)
+            {
+                if (lower.greaterThan(maxPackedValue))
+                    return Relation.CELL_OUTSIDE_QUERY;
+
+                crosses = lower.greaterThan(minPackedValue);
+            }
+
+            if (upper != null)
+            {
+                if (upper.smallerThan(minPackedValue))
+                    return Relation.CELL_OUTSIDE_QUERY;
+
+                crosses |= upper.smallerThan(maxPackedValue);
+            }
+
+            return crosses ? Relation.CELL_CROSSES_QUERY : Relation.CELL_INSIDE_QUERY;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBuffer.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBuffer.java
new file mode 100644
index 0000000..dfea17e
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBuffer.java
@@ -0,0 +1,104 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.util.concurrent.atomic.LongAdder;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.db.memtable.TrieMemtable;
+import org.apache.cassandra.db.tries.InMemoryTrie;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+/**
+ * On-heap buffer for values that provides a sorted view of itself as a {@link BlockBalancedTreeIterator}.
+ */
+@NotThreadSafe
+public class BlockBalancedTreeRamBuffer
+{
+    private final InMemoryTrie<PackedLongValues.Builder> trie;
+    private final PostingsAccumulator postingsAccumulator;
+    private final int bytesPerValue;
+    private int numRows;
+
+    public BlockBalancedTreeRamBuffer(int bytesPerValue)
+    {
+        trie = new InMemoryTrie<>(TrieMemtable.BUFFER_TYPE);
+        postingsAccumulator = new PostingsAccumulator();
+        this.bytesPerValue = bytesPerValue;
+    }
+
+    public int numRows()
+    {
+        return numRows;
+    }
+
+    public long memoryUsed()
+    {
+        return trie.sizeOnHeap() + postingsAccumulator.heapAllocations();
+    }
+
+    public long add(int segmentRowId, byte[] value)
+    {
+        final long initialSizeOnHeap = trie.sizeOnHeap();
+        final long reducerHeapSize = postingsAccumulator.heapAllocations();
+
+        try
+        {
+            trie.putRecursive(v -> ByteSource.fixedLength(value), segmentRowId, postingsAccumulator);
+        }
+        catch (InMemoryTrie.SpaceExhaustedException e)
+        {
+            throw Throwables.unchecked(e);
+        }
+
+        numRows++;
+        return (trie.sizeOnHeap() - initialSizeOnHeap) + (postingsAccumulator.heapAllocations() - reducerHeapSize);
+    }
+
+    public BlockBalancedTreeIterator iterator()
+    {
+        return BlockBalancedTreeIterator.fromTrieIterator(trie.entrySet().iterator(), bytesPerValue);
+    }
+
+    private static class PostingsAccumulator implements InMemoryTrie.UpsertTransformer<PackedLongValues.Builder, Integer>
+    {
+        private final LongAdder heapAllocations = new LongAdder();
+
+        @Override
+        public PackedLongValues.Builder apply(PackedLongValues.Builder existing, Integer rowID)
+        {
+            if (existing == null)
+            {
+                existing = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
+                heapAllocations.add(existing.ramBytesUsed());
+            }
+            long ramBefore = existing.ramBytesUsed();
+            existing.add(rowID);
+            heapAllocations.add(existing.ramBytesUsed() - ramBefore);
+            return existing;
+        }
+
+        long heapAllocations()
+        {
+            return heapAllocations.longValue();
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReader.java
new file mode 100644
index 0000000..59271e3
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReader.java
@@ -0,0 +1,427 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Comparator;
+import java.util.PriorityQueue;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.exceptions.QueryCancelledException;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.disk.io.IndexFileUtils;
+import org.apache.cassandra.index.sai.disk.io.SeekingRandomAccessInput;
+import org.apache.cassandra.index.sai.disk.v1.postings.FilteringPostingList;
+import org.apache.cassandra.index.sai.disk.v1.postings.MergePostingList;
+import org.apache.cassandra.index.sai.disk.v1.postings.PostingsReader;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.postings.PeekablePostingList;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.packed.DirectReader;
+import org.apache.lucene.util.packed.DirectWriter;
+
+/**
+ * Handles intersection of a point or point range with a block balanced tree previously written with
+ * {@link BlockBalancedTreeWriter}.
+ */
+public class BlockBalancedTreeReader extends BlockBalancedTreeWalker implements Closeable
+{
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+    private static final Comparator<PeekablePostingList> COMPARATOR = Comparator.comparingLong(PeekablePostingList::peek);
+
+    private final IndexContext indexContext;
+    private final FileHandle postingsFile;
+    private final BlockBalancedTreePostingsIndex postingsIndex;
+    private final int leafOrderMapBitsRequired;
+    /**
+     * Performs a blocking read.
+     */
+    public BlockBalancedTreeReader(IndexContext indexContext,
+                                   FileHandle treeIndexFile,
+                                   long treeIndexRoot,
+                                   FileHandle postingsFile,
+                                   long treePostingsRoot) throws IOException
+    {
+        super(treeIndexFile, treeIndexRoot);
+        this.indexContext = indexContext;
+        this.postingsFile = postingsFile;
+        this.postingsIndex = new BlockBalancedTreePostingsIndex(postingsFile, treePostingsRoot);
+        leafOrderMapBitsRequired = DirectWriter.unsignedBitsRequired(state.maxPointsInLeafNode - 1);
+    }
+
+    public int getBytesPerValue()
+    {
+        return state.bytesPerValue;
+    }
+
+    public long getPointCount()
+    {
+        return state.valueCount;
+    }
+
+    @Override
+    public void close()
+    {
+        super.close();
+        FileUtils.closeQuietly(postingsFile);
+    }
+
+    @SuppressWarnings({"resource", "RedundantSuppression"})
+    public PostingList intersect(IntersectVisitor visitor, QueryEventListener.BalancedTreeEventListener listener, QueryContext context)
+    {
+        Relation relation = visitor.compare(state.minPackedValue, state.maxPackedValue);
+
+        if (relation == Relation.CELL_OUTSIDE_QUERY)
+        {
+            listener.onIntersectionEarlyExit();
+            return null;
+        }
+
+        listener.onSegmentHit();
+        IndexInput treeInput = IndexFileUtils.instance.openInput(treeIndexFile);
+        IndexInput postingsInput = IndexFileUtils.instance.openInput(postingsFile);
+        IndexInput postingsSummaryInput = IndexFileUtils.instance.openInput(postingsFile);
+        state.reset();
+
+        Intersection intersection = relation == Relation.CELL_INSIDE_QUERY
+                                    ? new Intersection(treeInput, postingsInput, postingsSummaryInput, listener, context)
+                                    : new FilteringIntersection(treeInput, postingsInput, postingsSummaryInput, visitor, listener, context);
+
+        return intersection.execute();
+    }
+
+    /**
+     * Synchronous intersection of a point or point range with a block balanced tree previously written
+     * with {@link BlockBalancedTreeWriter}.
+     */
+    private class Intersection
+    {
+        private final Stopwatch queryExecutionTimer = Stopwatch.createStarted();
+        final QueryContext context;
+
+        final IndexInput treeInput;
+        final IndexInput postingsInput;
+        final IndexInput postingsSummaryInput;
+        final QueryEventListener.BalancedTreeEventListener listener;
+        final PriorityQueue<PeekablePostingList> postingLists;
+
+        Intersection(IndexInput treeInput, IndexInput postingsInput, IndexInput postingsSummaryInput,
+                     QueryEventListener.BalancedTreeEventListener listener, QueryContext context)
+        {
+            this.treeInput = treeInput;
+            this.postingsInput = postingsInput;
+            this.postingsSummaryInput = postingsSummaryInput;
+            this.listener = listener;
+            this.context = context;
+            postingLists = new PriorityQueue<>(state.numLeaves, COMPARATOR);
+        }
+
+        public PostingList execute()
+        {
+            try
+            {
+                executeInternal();
+
+                FileUtils.closeQuietly(treeInput);
+
+                return mergePostings();
+            }
+            catch (Throwable t)
+            {
+                if (!(t instanceof QueryCancelledException))
+                    logger.error(indexContext.logMessage("Balanced tree intersection failed on {}"), treeIndexFile.path(), t);
+
+                closeOnException();
+                throw Throwables.cleaned(t);
+            }
+        }
+
+        protected void executeInternal() throws IOException
+        {
+            collectPostingLists();
+        }
+
+        protected void closeOnException()
+        {
+            FileUtils.closeQuietly(treeInput);
+            FileUtils.closeQuietly(postingsInput);
+            FileUtils.closeQuietly(postingsSummaryInput);
+        }
+
+        protected PostingList mergePostings()
+        {
+            final long elapsedMicros = queryExecutionTimer.stop().elapsed(TimeUnit.MICROSECONDS);
+
+            listener.onIntersectionComplete(elapsedMicros, TimeUnit.MICROSECONDS);
+            listener.postingListsHit(postingLists.size());
+
+            if (postingLists.isEmpty())
+            {
+                FileUtils.closeQuietly(postingsInput);
+                FileUtils.closeQuietly(postingsSummaryInput);
+                return null;
+            }
+            else
+            {
+                if (logger.isTraceEnabled())
+                    logger.trace(indexContext.logMessage("[{}] Intersection completed in {} microseconds. {} leaf and internal posting lists hit."),
+                                 treeIndexFile.path(), elapsedMicros, postingLists.size());
+                return MergePostingList.merge(postingLists, () -> FileUtils.close(postingsInput, postingsSummaryInput));
+            }
+        }
+
+        private void collectPostingLists() throws IOException
+        {
+            context.checkpoint();
+
+            // This will return true if the node is a child leaf that has postings or if there is postings for the
+            // entire subtree under a leaf
+            if (postingsIndex.exists(state.nodeID))
+            {
+                postingLists.add(initPostingReader(postingsIndex.getPostingsFilePointer(state.nodeID)));
+                return;
+            }
+
+            if (state.atLeafNode())
+                throw new CorruptIndexException(indexContext.logMessage(String.format("Leaf node %s does not have balanced tree postings.", state.nodeID)), "");
+
+            // Recurse on left subtree:
+            state.pushLeft();
+            collectPostingLists();
+            state.pop();
+
+            // Recurse on right subtree:
+            state.pushRight();
+            collectPostingLists();
+            state.pop();
+        }
+
+        @SuppressWarnings({"resource", "RedundantSuppression"})
+        private PeekablePostingList initPostingReader(long offset) throws IOException
+        {
+            final PostingsReader.BlocksSummary summary = new PostingsReader.BlocksSummary(postingsSummaryInput, offset);
+            return PeekablePostingList.makePeekable(new PostingsReader(postingsInput, summary, listener.postingListEventListener()));
+        }
+    }
+
+    private class FilteringIntersection extends Intersection
+    {
+        private final IntersectVisitor visitor;
+        private final byte[] packedValue;
+        private final short[] origIndex;
+
+        FilteringIntersection(IndexInput treeInput, IndexInput postingsInput, IndexInput postingsSummaryInput,
+                              IntersectVisitor visitor, QueryEventListener.BalancedTreeEventListener listener, QueryContext context)
+        {
+            super(treeInput, postingsInput, postingsSummaryInput, listener, context);
+            this.visitor = visitor;
+            this.packedValue = new byte[state.bytesPerValue];
+            this.origIndex = new short[state.maxPointsInLeafNode];
+        }
+
+        @Override
+        public void executeInternal() throws IOException
+        {
+            collectPostingLists(state.minPackedValue, state.maxPackedValue);
+        }
+
+        private void collectPostingLists(byte[] minPackedValue, byte[] maxPackedValue) throws IOException
+        {
+            context.checkpoint();
+
+            final Relation r = visitor.compare(minPackedValue, maxPackedValue);
+
+            // This value range is fully outside the query shape: stop recursing
+            if (r == Relation.CELL_OUTSIDE_QUERY)
+                return;
+
+            if (r == Relation.CELL_INSIDE_QUERY)
+            {
+                // This value range is fully inside the query shape: recursively add all points from this node without filtering
+                super.collectPostingLists();
+                return;
+            }
+
+            if (state.atLeafNode())
+            {
+                if (state.nodeExists())
+                    filterLeaf();
+                return;
+            }
+
+            visitNode(minPackedValue, maxPackedValue);
+        }
+
+        private void filterLeaf() throws IOException
+        {
+            treeInput.seek(state.getLeafBlockFP());
+
+            int count = treeInput.readVInt();
+            int orderMapLength = treeInput.readVInt();
+            long orderMapPointer = treeInput.getFilePointer();
+
+            SeekingRandomAccessInput randomAccessInput = new SeekingRandomAccessInput(treeInput);
+            LongValues leafOrderMapReader = DirectReader.getInstance(randomAccessInput, leafOrderMapBitsRequired, orderMapPointer);
+            for (int index = 0; index < count; index++)
+            {
+                origIndex[index] = (short) Math.toIntExact(leafOrderMapReader.get(index));
+            }
+
+            // seek beyond the ordermap
+            treeInput.seek(orderMapPointer + orderMapLength);
+
+            FixedBitSet fixedBitSet = buildPostingsFilter(treeInput, count, visitor, origIndex);
+
+            if (postingsIndex.exists(state.nodeID) && fixedBitSet.cardinality() > 0)
+            {
+                long pointer = postingsIndex.getPostingsFilePointer(state.nodeID);
+                postingLists.add(initFilteringPostingReader(pointer, fixedBitSet));
+            }
+        }
+
+        void visitNode(byte[] minPackedValue, byte[] maxPackedValue) throws IOException
+        {
+            assert !state.atLeafNode() : "Cannot recurse down tree because nodeID " + state.nodeID + " is a leaf node";
+
+            byte[] splitValue = state.getSplitValue();
+
+            if (BlockBalancedTreeWriter.DEBUG)
+            {
+                // make sure cellMin <= splitValue <= cellMax:
+                assert ByteArrayUtil.compareUnsigned(minPackedValue, 0, splitValue, 0, state.bytesPerValue) <= 0 :"bytesPerValue=" + state.bytesPerValue;
+                assert ByteArrayUtil.compareUnsigned(maxPackedValue, 0, splitValue, 0, state.bytesPerValue) >= 0 : "bytesPerValue=" + state.bytesPerValue;
+            }
+
+            // Recurse on left subtree:
+            state.pushLeft();
+            collectPostingLists(minPackedValue, splitValue);
+            state.pop();
+
+            // Recurse on right subtree:
+            state.pushRight();
+            collectPostingLists(splitValue, maxPackedValue);
+            state.pop();
+        }
+
+        @SuppressWarnings({"resource", "RedundantSuppression"})
+        private PeekablePostingList initFilteringPostingReader(long offset, FixedBitSet filter) throws IOException
+        {
+            final PostingsReader.BlocksSummary summary = new PostingsReader.BlocksSummary(postingsSummaryInput, offset);
+            PostingsReader postingsReader = new PostingsReader(postingsInput, summary, listener.postingListEventListener());
+            return PeekablePostingList.makePeekable(new FilteringPostingList(filter, postingsReader));
+        }
+
+        private FixedBitSet buildPostingsFilter(IndexInput in, int count, IntersectVisitor visitor, short[] origIndex) throws IOException
+        {
+            int commonPrefixLength = readCommonPrefixLength(in);
+            return commonPrefixLength == state.bytesPerValue ? buildPostingsFilterForSingleValueLeaf(count, visitor, origIndex)
+                                                             : buildPostingsFilterForMultiValueLeaf(commonPrefixLength, in, count, visitor, origIndex);
+        }
+
+        private FixedBitSet buildPostingsFilterForMultiValueLeaf(int commonPrefixLength,
+                                                                 IndexInput in,
+                                                                 int count,
+                                                                 IntersectVisitor visitor,
+                                                                 short[] origIndex) throws IOException
+        {
+            // the byte at `compressedByteOffset` is compressed using run-length compression,
+            // other suffix bytes are stored verbatim
+            int compressedByteOffset = commonPrefixLength;
+            commonPrefixLength++;
+            int i;
+
+            FixedBitSet fixedBitSet = new FixedBitSet(state.maxPointsInLeafNode);
+
+            for (i = 0; i < count; )
+            {
+                packedValue[compressedByteOffset] = in.readByte();
+                final int runLen = Byte.toUnsignedInt(in.readByte());
+                for (int j = 0; j < runLen; ++j)
+                {
+                    in.readBytes(packedValue, commonPrefixLength, state.bytesPerValue - commonPrefixLength);
+                    final int rowIDIndex = origIndex[i + j];
+                    if (visitor.contains(packedValue))
+                        fixedBitSet.set(rowIDIndex);
+                }
+                i += runLen;
+            }
+            if (i != count)
+                throw new CorruptIndexException(String.format("Expected %d sub-blocks but read %d.", count, i), in);
+
+            return fixedBitSet;
+        }
+
+        private FixedBitSet buildPostingsFilterForSingleValueLeaf(int count, IntersectVisitor visitor, final short[] origIndex)
+        {
+            FixedBitSet fixedBitSet = new FixedBitSet(state.maxPointsInLeafNode);
+
+            // All the values in the leaf are the same, so we only
+            // need to visit once then set the bits for the relevant indexes
+            if (visitor.contains(packedValue))
+            {
+                for (int i = 0; i < count; ++i)
+                    fixedBitSet.set(origIndex[i]);
+            }
+            return fixedBitSet;
+        }
+
+        private int readCommonPrefixLength(IndexInput in) throws IOException
+        {
+            int prefixLength = in.readVInt();
+            if (prefixLength > 0)
+                in.readBytes(packedValue, 0, prefixLength);
+            return prefixLength;
+        }
+    }
+
+    /**
+     * We recurse the balanced tree, using a provided instance of this to guide the recursion.
+     */
+    public interface IntersectVisitor
+    {
+        /**
+         * Called for all values in a leaf cell that crosses the query.  The consumer should scrutinize the packedValue
+         * to decide whether to accept it. Values are visited in increasing order, and in the case of ties,
+         * in increasing order by segment row ID.
+         */
+        boolean contains(byte[] packedValue);
+
+        /**
+         * Called for non-leaf cells to test how the cell relates to the query, to
+         * determine how to further recurse down the tree.
+         */
+        Relation compare(byte[] minPackedValue, byte[] maxPackedValue);
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWalker.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWalker.java
new file mode 100644
index 0000000..ebfcd4c
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWalker.java
@@ -0,0 +1,318 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.agrona.collections.IntArrayList;
+import org.apache.cassandra.index.sai.disk.io.IndexInputReader;
+import org.apache.cassandra.index.sai.disk.v1.SAICodecUtils;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Base reader for a block balanced tree previously written with {@link BlockBalancedTreeWriter}.
+ * <p>
+ * Holds the index tree on heap and enables its traversal via {@link #traverse(TraversalCallback)}.
+ */
+public class BlockBalancedTreeWalker implements Closeable
+{
+    final FileHandle treeIndexFile;
+    final TraversalState state;
+
+    BlockBalancedTreeWalker(FileHandle treeIndexFile, long treeIndexRoot)
+    {
+        this.treeIndexFile = treeIndexFile;
+
+        try (RandomAccessReader reader = treeIndexFile.createReader();
+             IndexInput indexInput = IndexInputReader.create(reader))
+        {
+            SAICodecUtils.validate(indexInput);
+            indexInput.seek(treeIndexRoot);
+
+            state = new TraversalState(indexInput);
+        }
+        catch (Throwable t)
+        {
+            FileUtils.closeQuietly(treeIndexFile);
+            throw Throwables.unchecked(t);
+        }
+    }
+
+    public long memoryUsage()
+    {
+        return state.memoryUsage;
+    }
+
+    @Override
+    public void close()
+    {
+        FileUtils.closeQuietly(treeIndexFile);
+    }
+
+    void traverse(TraversalCallback callback)
+    {
+        state.reset();
+        traverse(callback, new IntArrayList());
+    }
+
+    private void traverse(TraversalCallback callback, IntArrayList pathToRoot)
+    {
+        if (state.atLeafNode())
+        {
+            // In the unbalanced case it's possible the left most node only has one child:
+            if (state.nodeExists())
+            {
+                callback.onLeaf(state.nodeID, state.getLeafBlockFP(), pathToRoot);
+            }
+        }
+        else
+        {
+            IntArrayList currentPath = new IntArrayList();
+            currentPath.addAll(pathToRoot);
+            currentPath.add(state.nodeID);
+
+            state.pushLeft();
+            traverse(callback, currentPath);
+            state.pop();
+
+            state.pushRight();
+            traverse(callback, currentPath);
+            state.pop();
+        }
+    }
+
+    interface TraversalCallback
+    {
+        void onLeaf(int leafNodeID, long leafBlockFP, IntArrayList pathToRoot);
+    }
+
+    /**
+     * This maintains the state for a traversal of the packed index. It is loaded once and can be resused
+     * by calling the reset method.
+     * <p>
+     * The packed index is a packed representation of a balanced tree and takes the form of a packed array of
+     * file pointer / split value pairs. Both the file pointers and split values are prefix compressed by tree level
+     * requiring us to maintain a stack of values for each level in the tree. The stack size is always the tree depth.
+     * <p>
+     * The tree is traversed by recursively following the left and then right subtrees under the current node. For the
+     * following tree (split values in square brackets):
+     * <pre>
+     *        1[16]
+     *       / \
+     *      /   \
+     *     2[8]  3[24]
+     *    / \   / \
+     *   4   5 6   7
+     * </pre>
+     * The traversal will be 1 -> 2 -> 4 -> 5 -> 3 -> 6 -> 7 with nodes 4, 5, 6 & 7 being leaf nodes.
+     * <p>
+     * Assuming the full range of values in the tree is 0 -> 32, the non-leaf nodes will represent the following
+     * values:
+     * <pre>
+     *         1[0-32]
+     *        /      \
+     *    2[0-16]   3[16-32]
+     * </pre>
+     */
+    final static class TraversalState
+    {
+        final int bytesPerValue;
+        final int numLeaves;
+        final int treeDepth;
+        final byte[] minPackedValue;
+        final byte[] maxPackedValue;
+        final long valueCount;
+        final int maxPointsInLeafNode;
+        final long memoryUsage;
+
+        // used to read the packed index byte[]
+        final ByteArrayDataInput dataInput;
+        // holds the minimum (left most) leaf block file pointer for each level we've recursed to:
+        final long[] leafBlockFPStack;
+        // holds the address, in the packed byte[] index, of the left-node of each level:
+        final int[] leftNodePositions;
+        // holds the address, in the packed byte[] index, of the right-node of each level:
+        final int[] rightNodePositions;
+        // holds the packed per-level split values; the run method uses this to save the cell min/max as it recurses:
+        final byte[][] splitValuesStack;
+
+        int nodeID;
+        int level;
+        @VisibleForTesting
+        int maxLevel;
+
+        TraversalState(DataInput dataInput) throws IOException
+        {
+            maxPointsInLeafNode = dataInput.readVInt();
+            bytesPerValue = dataInput.readVInt();
+
+            // Read index:
+            numLeaves = dataInput.readVInt();
+            assert numLeaves > 0;
+            treeDepth = dataInput.readVInt();
+            minPackedValue = new byte[bytesPerValue];
+            maxPackedValue = new byte[bytesPerValue];
+
+            dataInput.readBytes(minPackedValue, 0, bytesPerValue);
+            dataInput.readBytes(maxPackedValue, 0, bytesPerValue);
+
+            if (ByteArrayUtil.compareUnsigned(minPackedValue, 0, maxPackedValue, 0, bytesPerValue) > 0)
+            {
+                String message = String.format("Min packed value %s is > max packed value %s.",
+                                               new BytesRef(minPackedValue), new BytesRef(maxPackedValue));
+                throw new CorruptIndexException(message, dataInput);
+            }
+
+            valueCount = dataInput.readVLong();
+
+            int numBytes = dataInput.readVInt();
+            byte[] packedIndex = new byte[numBytes];
+            dataInput.readBytes(packedIndex, 0, numBytes);
+
+            nodeID = 1;
+            level = 0;
+            leafBlockFPStack = new long[treeDepth];
+            leftNodePositions = new int[treeDepth];
+            rightNodePositions = new int[treeDepth];
+            splitValuesStack = new byte[treeDepth][];
+
+            memoryUsage = ObjectSizes.sizeOfArray(packedIndex) +
+                          ObjectSizes.sizeOfArray(minPackedValue) +
+                          ObjectSizes.sizeOfArray(maxPackedValue) +
+                          ObjectSizes.sizeOfArray(leafBlockFPStack) +
+                          ObjectSizes.sizeOfArray(leftNodePositions) +
+                          ObjectSizes.sizeOfArray(rightNodePositions) +
+                          ObjectSizes.sizeOfArray(splitValuesStack) * bytesPerValue;
+
+            this.dataInput = new ByteArrayDataInput(packedIndex);
+            readNodeData(false);
+        }
+
+        public void reset()
+        {
+            nodeID = 1;
+            level = 0;
+            dataInput.setPosition(0);
+        }
+
+        public void pushLeft()
+        {
+            int nodePosition = leftNodePositions[level];
+            nodeID *= 2;
+            level++;
+            maxLevel = Math.max(maxLevel, level);
+            dataInput.setPosition(nodePosition);
+            readNodeData(true);
+        }
+
+        public void pushRight()
+        {
+            int nodePosition = rightNodePositions[level];
+            nodeID = nodeID * 2 + 1;
+            level++;
+            maxLevel = Math.max(maxLevel, level);
+            dataInput.setPosition(nodePosition);
+            readNodeData(false);
+        }
+
+        public void pop()
+        {
+            nodeID /= 2;
+            level--;
+        }
+
+        public boolean atLeafNode()
+        {
+            return nodeID >= numLeaves;
+        }
+
+        public boolean nodeExists()
+        {
+            return nodeID - numLeaves < numLeaves;
+        }
+
+        public long getLeafBlockFP()
+        {
+            return leafBlockFPStack[level];
+        }
+
+        public byte[] getSplitValue()
+        {
+            assert !atLeafNode();
+            return splitValuesStack[level];
+        }
+
+        private void readNodeData(boolean isLeft)
+        {
+            leafBlockFPStack[level] = level == 0 ? 0 : leafBlockFPStack[level - 1];
+
+            // read leaf block FP delta
+            if (!isLeft)
+                leafBlockFPStack[level] += dataInput.readVLong();
+
+            if (!atLeafNode())
+            {
+                // read prefix, firstDiffByteDelta encoded as int:
+                int code = dataInput.readVInt();
+                int prefix = code % (1 + bytesPerValue);
+                int suffix = bytesPerValue - prefix;
+
+                pushSplitValueStack();
+                if (suffix > 0)
+                {
+                    int firstDiffByteDelta = code / (1 + bytesPerValue);
+                    // If we are pushing to the left subtree then the delta will be negative
+                    if (isLeft)
+                        firstDiffByteDelta = -firstDiffByteDelta;
+                    int oldByte = splitValuesStack[level][prefix] & 0xFF;
+                    splitValuesStack[level][prefix] = (byte) (oldByte + firstDiffByteDelta);
+                    dataInput.readBytes(splitValuesStack[level], prefix + 1, suffix - 1);
+                }
+
+                int leftNumBytes = nodeID * 2 < numLeaves ? dataInput.readVInt() : 0;
+
+                leftNodePositions[level] = dataInput.getPosition();
+                rightNodePositions[level] = leftNodePositions[level] + leftNumBytes;
+            }
+        }
+
+        private void pushSplitValueStack()
+        {
+            if (splitValuesStack[level] == null)
+                splitValuesStack[level] = new byte[bytesPerValue];
+            if (level == 0)
+                Arrays.fill(splitValuesStack[level], (byte) 0);
+            else
+                System.arraycopy(splitValuesStack[level - 1], 0, splitValuesStack[level], 0, bytesPerValue);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWriter.java
new file mode 100644
index 0000000..f74a69c
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWriter.java
@@ -0,0 +1,762 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import com.google.common.base.MoreObjects;
+
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.index.sai.disk.ResettableByteBuffersIndexOutput;
+import org.apache.cassandra.index.sai.disk.v1.SAICodecUtils;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntroSorter;
+import org.apache.lucene.util.Sorter;
+import org.apache.lucene.util.bkd.BKDWriter;
+
+import static org.apache.cassandra.index.sai.postings.PostingList.END_OF_STREAM;
+
+/**
+ * This is a specialisation of the Lucene {@link BKDWriter} that only writes a single dimension
+ * balanced tree.
+ * <p>
+ * Recursively builds a block balanced tree to assign all incoming points to smaller
+ * and smaller rectangles (cells) until the number of points in a given
+ * rectangle is &lt;= <code>maxPointsInLeafNode</code>.  The tree is
+ * fully balanced, which means the leaf nodes will have between 50% and 100% of
+ * the requested <code>maxPointsInLeafNode</code>.  Values that fall exactly
+ * on a cell boundary may be in either cell.
+ * <p>
+ * Visual representation of the disk format:
+ * <pre>
+ *
+ * +========+=======================================+==================+========+
+ * | HEADER | LEAF BLOCK LIST                       | BALANCED TREE    | FOOTER |
+ * +========+================+=====+================+==================+========+
+ *          | LEAF BLOCK (0) | ... | LEAF BLOCK (N) | VALUES PER LEAF  |
+ *          +----------------+-----+----------------+------------------|
+ *          | ORDER INDEX    |                      | BYTES PER VALUE  |
+ *          +----------------+                      +------------------+
+ *          | PREFIX         |                      | NUMBER OF LEAVES |
+ *          +----------------+                      +------------------+
+ *          | VALUES         |                      | MINIMUM VALUE    |
+ *          +----------------+                      +------------------+
+ *                                                  | MAXIMUM VALUE    |
+ *                                                  +------------------+
+ *                                                  | TOTAL VALUES     |
+ *                                                  +------------------+
+ *                                                  | INDEX TREE       |
+ *                                                  +--------+---------+
+ *                                                  | LENGTH | BYTES   |
+ *                                                  +--------+---------+
+ *  </pre>
+ *
+ * <p>
+ * <b>NOTE</b>: This can write at most Integer.MAX_VALUE * <code>maxPointsInLeafNode</code> total points.
+ * <p>
+ * @see BKDWriter
+ */
+@NotThreadSafe
+public class BlockBalancedTreeWriter
+{
+    // Enable to check that values are added to the tree in correct order and within bounds
+    public static final boolean DEBUG = CassandraRelevantProperties.SAI_TEST_BALANCED_TREE_DEBUG_ENABLED.getBoolean();
+
+    // Default maximum number of point in each leaf block
+    public static final int DEFAULT_MAX_POINTS_IN_LEAF_NODE = 1024;
+
+    private final int bytesPerValue;
+    private final int maxPointsInLeafNode;
+    private final byte[] minPackedValue;
+    private final byte[] maxPackedValue;
+    private long valueCount;
+
+    public BlockBalancedTreeWriter(int bytesPerValue, int maxPointsInLeafNode)
+    {
+        if (maxPointsInLeafNode <= 0)
+            throw new IllegalArgumentException("maxPointsInLeafNode must be > 0; got " + maxPointsInLeafNode);
+        if (maxPointsInLeafNode > ArrayUtil.MAX_ARRAY_LENGTH)
+            throw new IllegalArgumentException("maxPointsInLeafNode must be <= ArrayUtil.MAX_ARRAY_LENGTH (= " +
+                                               ArrayUtil.MAX_ARRAY_LENGTH + "); got " + maxPointsInLeafNode);
+
+        this.maxPointsInLeafNode = maxPointsInLeafNode;
+        this.bytesPerValue = bytesPerValue;
+
+        minPackedValue = new byte[bytesPerValue];
+        maxPackedValue = new byte[bytesPerValue];
+    }
+
+    public long getValueCount()
+    {
+        return valueCount;
+    }
+
+    public int getBytesPerValue()
+    {
+        return bytesPerValue;
+    }
+
+    public int getMaxPointsInLeafNode()
+    {
+        return maxPointsInLeafNode;
+    }
+
+    /**
+     * Write the sorted values from a {@link BlockBalancedTreeIterator}.
+     * <p>
+     * @param treeOutput The {@link IndexOutput} to write the balanced tree to
+     * @param values The {@link BlockBalancedTreeIterator} containing the values and rowIDs to be written
+     * @param callback The {@link Callback} used to record the leaf postings for each leaf
+     *
+     * @return The file pointer to the beginning of the balanced tree
+     */
+    public long write(IndexOutput treeOutput, BlockBalancedTreeIterator values, final Callback callback) throws IOException
+    {
+        SAICodecUtils.writeHeader(treeOutput);
+
+        LeafWriter leafWriter = new LeafWriter(treeOutput, callback);
+
+        while (values.hasNext())
+        {
+            Pair<byte[], PostingList> pair = values.next();
+            long segmentRowId;
+            while ((segmentRowId = pair.right.nextPosting()) != END_OF_STREAM)
+                leafWriter.add(pair.left, segmentRowId);
+        }
+
+        valueCount = leafWriter.finish();
+
+        long treeFilePointer = valueCount == 0 ? -1 : treeOutput.getFilePointer();
+
+        // There is only any point in writing the balanced tree if any values were added
+        if (treeFilePointer >= 0)
+            writeBalancedTree(treeOutput, maxPointsInLeafNode, leafWriter.leafBlockStartValues, leafWriter.leafBlockFilePointers);
+
+        SAICodecUtils.writeFooter(treeOutput);
+
+        return treeFilePointer;
+    }
+
+    private void writeBalancedTree(IndexOutput out, int countPerLeaf, List<byte[]> leafBlockStartValues, List<Long> leafBlockFilePointer) throws IOException
+    {
+        int numInnerNodes = leafBlockStartValues.size();
+        byte[] splitValues = new byte[(1 + numInnerNodes) * bytesPerValue];
+        int treeDepth = recurseBalanceTree(1, 0, numInnerNodes, 1, splitValues, leafBlockStartValues);
+        long[] leafBlockFPs = leafBlockFilePointer.stream().mapToLong(l -> l).toArray();
+        byte[] packedIndex = packIndex(leafBlockFPs, splitValues);
+
+        out.writeVInt(countPerLeaf);
+        out.writeVInt(bytesPerValue);
+
+        out.writeVInt(leafBlockFPs.length);
+        out.writeVInt(Math.min(treeDepth, leafBlockFPs.length));
+
+        out.writeBytes(minPackedValue, 0, bytesPerValue);
+        out.writeBytes(maxPackedValue, 0, bytesPerValue);
+
+        out.writeVLong(valueCount);
+
+        out.writeVInt(packedIndex.length);
+        out.writeBytes(packedIndex, 0, packedIndex.length);
+    }
+
+    /**
+     * This can, potentially, be removed in the future by CASSANDRA-18597
+     */
+    private int recurseBalanceTree(int nodeID, int offset, int count, int treeDepth, byte[] splitValues, List<byte[]> leafBlockStartValues)
+    {
+        if (count == 1)
+        {
+            treeDepth++;
+            // Leaf index node
+            System.arraycopy(leafBlockStartValues.get(offset), 0, splitValues, nodeID * bytesPerValue, bytesPerValue);
+        }
+        else if (count > 1)
+        {
+            treeDepth++;
+            // Internal index node: binary partition of count
+            int countAtLevel = 1;
+            int totalCount = 0;
+            while (true)
+            {
+                int countLeft = count - totalCount;
+                if (countLeft <= countAtLevel)
+                {
+                    // This is the last level, possibly partially filled:
+                    int lastLeftCount = Math.min(countAtLevel / 2, countLeft);
+                    assert lastLeftCount >= 0;
+                    int leftHalf = (totalCount - 1) / 2 + lastLeftCount;
+
+                    int rootOffset = offset + leftHalf;
+
+                    System.arraycopy(leafBlockStartValues.get(rootOffset), 0, splitValues, nodeID * bytesPerValue, bytesPerValue);
+
+                    // TODO: we could optimize/specialize, when we know it's simply fully balanced binary tree
+                    // under here, to save this while loop on each recursion
+
+                    // Recurse left
+                    int leftTreeDepth = recurseBalanceTree(2 * nodeID, offset, leftHalf, treeDepth, splitValues, leafBlockStartValues);
+
+                    // Recurse right
+                    int rightTreeDepth = recurseBalanceTree(2 * nodeID + 1, rootOffset + 1, count - leftHalf - 1, treeDepth, splitValues, leafBlockStartValues);
+                    return Math.max(leftTreeDepth, rightTreeDepth);
+                }
+                totalCount += countAtLevel;
+                countAtLevel *= 2;
+            }
+        }
+        else
+        {
+            assert count == 0;
+        }
+        return treeDepth;
+    }
+
+    // Packs the two arrays, representing a balanced binary tree, into a compact byte[] structure.
+    private byte[] packIndex(long[] leafBlockFPs, byte[] splitValues) throws IOException
+    {
+        int numLeaves = leafBlockFPs.length;
+
+        // Possibly rotate the leaf block FPs, if the index is not a fully balanced binary tree (only happens
+        // if it was created by TreeWriter).  In this case the leaf nodes may straddle the two bottom
+        // levels of the binary tree:
+        if (numLeaves > 1)
+        {
+            int levelCount = 2;
+            while (true)
+            {
+                if (numLeaves >= levelCount && numLeaves <= 2 * levelCount)
+                {
+                    int lastLevel = 2 * (numLeaves - levelCount);
+                    assert lastLevel >= 0;
+                    if (lastLevel != 0)
+                    {
+                        // Last level is partially filled, so we must rotate the leaf FPs to match.  We do this here, after loading
+                        // at read-time, so that we can still delta code them on disk at write:
+                        long[] newLeafBlockFPs = new long[numLeaves];
+                        System.arraycopy(leafBlockFPs, lastLevel, newLeafBlockFPs, 0, leafBlockFPs.length - lastLevel);
+                        System.arraycopy(leafBlockFPs, 0, newLeafBlockFPs, leafBlockFPs.length - lastLevel, lastLevel);
+                        leafBlockFPs = newLeafBlockFPs;
+                    }
+                    break;
+                }
+
+                levelCount *= 2;
+            }
+        }
+
+        // Reused while packing the index
+        try (ResettableByteBuffersIndexOutput writeBuffer = new ResettableByteBuffersIndexOutput("PackedIndex"))
+        {
+            // This is the "file" we append the byte[] to:
+            List<byte[]> blocks = new ArrayList<>();
+            byte[] lastSplitValue = new byte[bytesPerValue];
+            int totalSize = recursePackIndex(writeBuffer, leafBlockFPs, splitValues, 0, blocks, 1, lastSplitValue, false);
+            // Compact the byte[] blocks into single byte index:
+            byte[] index = new byte[totalSize];
+            int upto = 0;
+            for (byte[] block : blocks)
+            {
+                System.arraycopy(block, 0, index, upto, block.length);
+                upto += block.length;
+            }
+            assert upto == totalSize;
+
+            return index;
+        }
+    }
+
+    /**
+     * lastSplitValue is the split value previously seen; we use this to prefix-code the split byte[] on each
+     * inner node
+     */
+    private int recursePackIndex(ResettableByteBuffersIndexOutput writeBuffer, long[] leafBlockFPs, byte[] splitValues,
+                                 long minBlockFP, List<byte[]> blocks, int nodeID, byte[] lastSplitValue, boolean isLeft) throws IOException
+    {
+        if (nodeID >= leafBlockFPs.length)
+        {
+            int leafID = nodeID - leafBlockFPs.length;
+
+            // In the unbalanced case it's possible the left most node only has one child:
+            if (leafID < leafBlockFPs.length)
+            {
+                long delta = leafBlockFPs[leafID] - minBlockFP;
+                if (isLeft)
+                {
+                    assert delta == 0;
+                    return 0;
+                }
+                else
+                {
+                    assert nodeID == 1 || delta > 0 : "nodeID=" + nodeID;
+                    writeBuffer.writeVLong(delta);
+                    return appendBlock(writeBuffer, blocks);
+                }
+            }
+            else
+            {
+                throw new IllegalStateException("Unbalanced tree");
+            }
+        }
+        else
+        {
+            long leftBlockFP;
+            if (!isLeft)
+            {
+                leftBlockFP = getLeftMostLeafBlockFP(leafBlockFPs, nodeID);
+                long delta = leftBlockFP - minBlockFP;
+                assert nodeID == 1 || delta > 0;
+                writeBuffer.writeVLong(delta);
+            }
+            else
+            {
+                // The left tree's left most leaf block FP is always the minimal FP:
+                leftBlockFP = minBlockFP;
+            }
+
+            int address = nodeID * bytesPerValue;
+
+            // find common prefix with last split value in this dim:
+            int prefix = 0;
+            for (; prefix < bytesPerValue; prefix++)
+            {
+                if (splitValues[address + prefix] != lastSplitValue[prefix])
+                {
+                    break;
+                }
+            }
+
+            int firstDiffByteDelta;
+            if (prefix < bytesPerValue)
+            {
+                firstDiffByteDelta = (splitValues[address + prefix] & 0xFF) - (lastSplitValue[prefix] & 0xFF);
+                // If this is left then we need to negate the delta
+                if (isLeft)
+                    firstDiffByteDelta = -firstDiffByteDelta;
+                assert firstDiffByteDelta > 0;
+            }
+            else
+            {
+                firstDiffByteDelta = 0;
+            }
+
+            // pack the prefix and delta first diff byte into a single vInt:
+            int code = (firstDiffByteDelta * (1 + bytesPerValue) + prefix);
+
+            writeBuffer.writeVInt(code);
+
+            // write the split value, prefix coded vs. our parent's split value:
+            int suffix = bytesPerValue - prefix;
+            byte[] savSplitValue = new byte[suffix];
+            if (suffix > 1)
+            {
+                writeBuffer.writeBytes(splitValues, address + prefix + 1, suffix - 1);
+            }
+
+            byte[] cmp = lastSplitValue.clone();
+
+            System.arraycopy(lastSplitValue, prefix, savSplitValue, 0, suffix);
+
+            // copy our split value into lastSplitValue for our children to prefix-code against
+            System.arraycopy(splitValues, address + prefix, lastSplitValue, prefix, suffix);
+
+            int numBytes = appendBlock(writeBuffer, blocks);
+
+            // placeholder for left-tree numBytes; we need this so that at search time if we only need to recurse into
+            // the right subtree we can quickly seek to its starting point
+            int idxSav = blocks.size();
+            blocks.add(null);
+
+            int leftNumBytes = recursePackIndex(writeBuffer, leafBlockFPs, splitValues, leftBlockFP, blocks, 2 * nodeID, lastSplitValue, true);
+
+            if (nodeID * 2 < leafBlockFPs.length)
+            {
+                writeBuffer.writeVInt(leftNumBytes);
+            }
+            else
+            {
+                assert leftNumBytes == 0 : "leftNumBytes=" + leftNumBytes;
+            }
+            int numBytes2 = Math.toIntExact(writeBuffer.getFilePointer());
+            byte[] bytes2 = writeBuffer.toArrayCopy();
+            writeBuffer.reset();
+            // replace our placeholder:
+            blocks.set(idxSav, bytes2);
+
+            int rightNumBytes = recursePackIndex(writeBuffer, leafBlockFPs, splitValues, leftBlockFP, blocks, 2 * nodeID + 1, lastSplitValue, false);
+
+            // restore lastSplitValue to what caller originally passed us:
+            System.arraycopy(savSplitValue, 0, lastSplitValue, prefix, suffix);
+
+            assert Arrays.equals(lastSplitValue, cmp);
+
+            return numBytes + numBytes2 + leftNumBytes + rightNumBytes;
+        }
+    }
+
+    /** Appends the current contents of writeBuffer as another block on the growing in-memory file */
+    private int appendBlock(ResettableByteBuffersIndexOutput writeBuffer, List<byte[]> blocks)
+    {
+        int pos = Math.toIntExact(writeBuffer.getFilePointer());
+        byte[] bytes = writeBuffer.toArrayCopy();
+        writeBuffer.reset();
+        blocks.add(bytes);
+        return pos;
+    }
+
+    private long getLeftMostLeafBlockFP(long[] leafBlockFPs, int nodeID)
+    {
+        // TODO: can we do this cheaper, e.g. a closed form solution instead of while loop?  Or
+        // change the recursion while packing the index to return this left-most leaf block FP
+        // from each recursion instead?
+        //
+        // Still, the overall cost here is minor: this method's cost is O(log(N)), and while writing
+        // we call it O(N) times (N = number of leaf blocks)
+        while (nodeID < leafBlockFPs.length)
+        {
+            nodeID *= 2;
+        }
+        int leafID = nodeID - leafBlockFPs.length;
+        long result = leafBlockFPs[leafID];
+        if (result < 0)
+        {
+            throw new AssertionError(result + " for leaf " + leafID);
+        }
+        return result;
+    }
+
+    interface Callback
+    {
+        void writeLeafPostings(RowIDAndIndex[] leafPostings, int offset, int count);
+    }
+
+    static class RowIDAndIndex
+    {
+        public int valueOrderIndex;
+        public long rowID;
+
+        @Override
+        public String toString()
+        {
+            return MoreObjects.toStringHelper(this)
+                              .add("valueOrderIndex", valueOrderIndex)
+                              .add("rowID", rowID)
+                              .toString();
+        }
+    }
+
+    /**
+     * Responsible for writing the leaf blocks at the beginning of the balanced tree index.
+     */
+    private class LeafWriter
+    {
+        private final IndexOutput treeOutput;
+        private final List<Long> leafBlockFilePointers = new ArrayList<>();
+        private final List<byte[]> leafBlockStartValues = new ArrayList<>();
+        private final byte[] leafValues = new byte[maxPointsInLeafNode * bytesPerValue];
+        private final long[] leafRowIDs = new long[maxPointsInLeafNode];
+        private final RowIDAndIndex[] rowIDAndIndexes = new RowIDAndIndex[maxPointsInLeafNode];
+        private final int[] orderIndex = new int[maxPointsInLeafNode];
+        private final Callback callback;
+        private final ByteBuffersDataOutput leafOrderIndexOutput = new ByteBuffersDataOutput(2 * 1024);
+        private final ByteBuffersDataOutput leafBlockOutput = new ByteBuffersDataOutput(32 * 1024);
+        private final byte[] lastPackedValue = new byte[bytesPerValue];
+
+        private long valueCount;
+        private int leafValueCount;
+        private long lastRowID;
+
+        LeafWriter(IndexOutput treeOutput, Callback callback)
+        {
+            assert callback != null : "Callback cannot be null in TreeWriter";
+
+            this.treeOutput = treeOutput;
+            this.callback = callback;
+
+            for (int x = 0; x < rowIDAndIndexes.length; x++)
+            {
+                rowIDAndIndexes[x] = new RowIDAndIndex();
+            }
+        }
+
+        /**
+         * Adds a value and row ID to the current leaf block. If the leaf block is full after the addition
+         * the current leaf block is written to disk.
+         */
+        void add(byte[] packedValue, long rowID) throws IOException
+        {
+            if (DEBUG)
+                valueInOrder(valueCount + leafValueCount, lastPackedValue, packedValue, 0, rowID, lastRowID);
+
+            System.arraycopy(packedValue, 0, leafValues, leafValueCount * bytesPerValue, bytesPerValue);
+            leafRowIDs[leafValueCount] = rowID;
+            leafValueCount++;
+
+            if (leafValueCount == maxPointsInLeafNode)
+            {
+                // We write a block once we hit exactly the max count
+                writeLeafBlock();
+                leafValueCount = 0;
+            }
+
+            if (DEBUG)
+                if ((lastRowID = rowID) < 0)
+                    throw new AssertionError("row id must be >= 0; got " + rowID);
+        }
+
+        /**
+         * Write a leaf block if we have unwritten values and return the total number of values added
+         */
+        public long finish() throws IOException
+        {
+            if (leafValueCount > 0)
+                writeLeafBlock();
+
+            return valueCount;
+        }
+
+        private void writeLeafBlock() throws IOException
+        {
+            assert leafValueCount != 0;
+            if (valueCount == 0)
+            {
+                System.arraycopy(leafValues, 0, minPackedValue, 0, bytesPerValue);
+            }
+            System.arraycopy(leafValues, (leafValueCount - 1) * bytesPerValue, maxPackedValue, 0, bytesPerValue);
+
+            valueCount += leafValueCount;
+
+            if (leafBlockFilePointers.size() > 0)
+            {
+                // Save the first (minimum) value in each leaf block except the first, to build the split value index in the end:
+                leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, bytesPerValue));
+            }
+            leafBlockFilePointers.add(treeOutput.getFilePointer());
+            checkMaxLeafNodeCount(leafBlockFilePointers.size());
+
+            // Find the common prefix between the first and last values in the block
+            int commonPrefixLength = bytesPerValue;
+            int offset = (leafValueCount - 1) * bytesPerValue;
+            for (int j = 0; j < bytesPerValue; j++)
+            {
+                if (leafValues[j] != leafValues[offset + j])
+                {
+                    commonPrefixLength = j;
+                    break;
+                }
+            }
+
+            treeOutput.writeVInt(leafValueCount);
+
+            for (int x = 0; x < leafValueCount; x++)
+            {
+                rowIDAndIndexes[x].valueOrderIndex = x;
+                rowIDAndIndexes[x].rowID = leafRowIDs[x];
+            }
+
+            final Sorter sorter = new IntroSorter()
+            {
+                RowIDAndIndex pivot;
+
+                @Override
+                protected void swap(int i, int j)
+                {
+                    RowIDAndIndex o = rowIDAndIndexes[i];
+                    rowIDAndIndexes[i] = rowIDAndIndexes[j];
+                    rowIDAndIndexes[j] = o;
+                }
+
+                @Override
+                protected void setPivot(int i)
+                {
+                    pivot = rowIDAndIndexes[i];
+                }
+
+                @Override
+                protected int comparePivot(int j)
+                {
+                    return Long.compare(pivot.rowID, rowIDAndIndexes[j].rowID);
+                }
+            };
+
+            sorter.sort(0, leafValueCount);
+
+            // write the leaf order index: leaf rowID -> orig index
+            leafOrderIndexOutput.reset();
+
+            // iterate in row ID order to get the row ID index for the given value order index
+            // place into an array to be written as packed ints
+            for (int x = 0; x < leafValueCount; x++)
+                orderIndex[rowIDAndIndexes[x].valueOrderIndex] = x;
+
+            LeafOrderMap.write(orderIndex, leafValueCount, maxPointsInLeafNode - 1, leafOrderIndexOutput);
+
+            treeOutput.writeVInt((int) leafOrderIndexOutput.size());
+            leafOrderIndexOutput.copyTo(treeOutput);
+
+            callback.writeLeafPostings(rowIDAndIndexes, 0, leafValueCount);
+
+            // Write the common prefix for the leaf block
+            writeCommonPrefix(treeOutput, commonPrefixLength, leafValues);
+
+            // Write the run length encoded packed values for the leaf block
+            leafBlockOutput.reset();
+
+            if (DEBUG)
+                valuesInOrderAndBounds(leafValueCount,
+                                       ArrayUtil.copyOfSubArray(leafValues, 0, bytesPerValue),
+                                       ArrayUtil.copyOfSubArray(leafValues, (leafValueCount - 1) * bytesPerValue, leafValueCount * bytesPerValue),
+                                       leafRowIDs);
+
+            writeLeafBlockPackedValues(leafBlockOutput, commonPrefixLength, leafValueCount);
+
+            leafBlockOutput.copyTo(treeOutput);
+        }
+
+        private void checkMaxLeafNodeCount(int numLeaves)
+        {
+            if (bytesPerValue * (long) numLeaves > ArrayUtil.MAX_ARRAY_LENGTH)
+            {
+                throw new IllegalStateException("too many nodes; increase maxPointsInLeafNode (currently " + maxPointsInLeafNode + ") and reindex");
+            }
+        }
+
+        private void writeCommonPrefix(DataOutput treeOutput, int commonPrefixLength, byte[] packedValue) throws IOException
+        {
+            treeOutput.writeVInt(commonPrefixLength);
+            if (commonPrefixLength > 0)
+                treeOutput.writeBytes(packedValue, 0, commonPrefixLength);
+        }
+
+        private void writeLeafBlockPackedValues(DataOutput out, int commonPrefixLength, int count) throws IOException
+        {
+            // If all the values are the same (e.g. the common prefix length == bytes per value) then we don't
+            // need to write anything. Otherwise, we run length compress the values to disk.
+            if (commonPrefixLength != bytesPerValue)
+            {
+                int compressedByteOffset = commonPrefixLength;
+                commonPrefixLength++;
+                for (int i = 0; i < count; )
+                {
+                    // do run-length compression on the byte at compressedByteOffset
+                    int runLen = runLen(i, Math.min(i + 0xff, count), compressedByteOffset);
+                    assert runLen <= 0xff;
+                    byte prefixByte = leafValues[i * bytesPerValue + compressedByteOffset];
+                    out.writeByte(prefixByte);
+                    out.writeByte((byte) runLen);
+                    writeLeafBlockPackedValuesRange(out, commonPrefixLength, i, i + runLen);
+                    i += runLen;
+                    assert i <= count;
+                }
+            }
+        }
+
+        private void writeLeafBlockPackedValuesRange(DataOutput out, int commonPrefixLength, int start, int end) throws IOException
+        {
+            for (int i = start; i < end; ++i)
+            {
+                out.writeBytes(leafValues, i * bytesPerValue + commonPrefixLength, bytesPerValue - commonPrefixLength);
+            }
+        }
+
+        private int runLen(int start, int end, int byteOffset)
+        {
+            byte b = leafValues[start * bytesPerValue + byteOffset];
+            for (int i = start + 1; i < end; ++i)
+            {
+                byte b2 = leafValues[i * bytesPerValue + byteOffset];
+                assert Byte.toUnsignedInt(b2) >= Byte.toUnsignedInt(b);
+                if (b != b2)
+                {
+                    return i - start;
+                }
+            }
+            return end - start;
+        }
+
+        // The following 3 methods are only used when DEBUG is true:
+
+        private void valueInBounds(byte[] packedValue, int packedValueOffset, byte[] minPackedValue, byte[] maxPackedValue)
+        {
+            if (ByteArrayUtil.compareUnsigned(packedValue,
+                                              packedValueOffset,
+                                              minPackedValue,
+                                              0,
+                                              bytesPerValue) < 0)
+            {
+                throw new AssertionError("value=" + new BytesRef(packedValue, packedValueOffset, bytesPerValue) +
+                                         " is < minPackedValue=" + new BytesRef(minPackedValue));
+            }
+
+            if (ByteArrayUtil.compareUnsigned(packedValue,
+                                              packedValueOffset,
+                                              maxPackedValue, 0,
+                                              bytesPerValue) > 0)
+            {
+                throw new AssertionError("value=" + new BytesRef(packedValue, packedValueOffset, bytesPerValue) +
+                                         " is > maxPackedValue=" + new BytesRef(maxPackedValue));
+            }
+        }
+
+        private void valuesInOrderAndBounds(int count, byte[] minPackedValue, byte[] maxPackedValue, long[] rowIds)
+        {
+            byte[] lastPackedValue = new byte[bytesPerValue];
+            long lastRowId = -1;
+            for (int i = 0; i < count; i++)
+            {
+                valueInOrder(i, lastPackedValue, leafValues, i * bytesPerValue, rowIds[i], lastRowId);
+                lastRowId = rowIds[i];
+
+                // Make sure this value does in fact fall within this leaf cell:
+                valueInBounds(leafValues, i * bytesPerValue, minPackedValue, maxPackedValue);
+            }
+        }
+
+        private void valueInOrder(long ord, byte[] lastPackedValue, byte[] packedValue, int packedValueOffset, long rowId, long lastRowId)
+        {
+            if (ord > 0)
+            {
+                int cmp = ByteArrayUtil.compareUnsigned(lastPackedValue, 0, packedValue, packedValueOffset, bytesPerValue);
+                if (cmp > 0)
+                {
+                    throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) +
+                                             " current value=" + new BytesRef(packedValue, packedValueOffset, bytesPerValue) +
+                                             " ord=" + ord);
+                }
+                if (cmp == 0 && rowId < lastRowId)
+                {
+                    throw new AssertionError("row IDs out of order: last rowID=" + lastRowId + " current rowID=" + rowId + " ord=" + ord);
+                }
+            }
+            System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, bytesPerValue);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMap.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMap.java
new file mode 100644
index 0000000..8fd5bf3
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMap.java
@@ -0,0 +1,39 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.io.IOException;
+
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.packed.DirectWriter;
+
+class LeafOrderMap
+{
+    static void write(final int[] array, int length, int maxValue, final DataOutput out) throws IOException
+    {
+        final int bits = DirectWriter.unsignedBitsRequired(maxValue);
+        final DirectWriter writer = DirectWriter.getInstance(out, length, bits);
+        for (int i = 0; i < length; i++)
+        {
+            assert array[i] <= maxValue;
+
+            writer.add(array[i]);
+        }
+        writer.finish();
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriter.java
new file mode 100644
index 0000000..3f02d5d
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriter.java
@@ -0,0 +1,180 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.MoreObjects;
+
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.disk.format.IndexComponent;
+import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.io.IndexOutputWriter;
+import org.apache.cassandra.index.sai.disk.v1.segment.SegmentMetadata;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * Specialized writer for values, that builds them into a {@link BlockBalancedTreeWriter} with auxiliary
+ * posting lists on eligible tree levels.
+ * <p>
+ * Given a sorted input {@link BlockBalancedTreeIterator}, the flush process is optimised because we don't need to
+ * buffer all point values to sort them.
+ */
+public class NumericIndexWriter
+{
+    public static final int MAX_POINTS_IN_LEAF_NODE = BlockBalancedTreeWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE;
+    private static final int DEFAULT_POSTINGS_SIZE = 128;
+
+    private final BlockBalancedTreeWriter writer;
+    private final IndexDescriptor indexDescriptor;
+    private final IndexContext indexContext;
+    private final int bytesPerValue;
+
+    /**
+     * @param maxSegmentRowId maximum possible segment row ID, used to create `maxRows` for the balanced tree
+     */
+    public NumericIndexWriter(IndexDescriptor indexDescriptor,
+                              IndexContext indexContext,
+                              int bytesPerValue,
+                              long maxSegmentRowId)
+    {
+        this(indexDescriptor, indexContext, MAX_POINTS_IN_LEAF_NODE, bytesPerValue, maxSegmentRowId);
+    }
+
+    @VisibleForTesting
+    public NumericIndexWriter(IndexDescriptor indexDescriptor,
+                              IndexContext indexContext,
+                              int maxPointsInLeafNode,
+                              int bytesPerValue,
+                              long maxSegmentRowId)
+    {
+        checkArgument(maxSegmentRowId >= 0, "[%s] maxSegmentRowId must be non-negative value, but got %s", indexContext.getIndexName(), maxSegmentRowId);
+
+        this.indexDescriptor = indexDescriptor;
+        this.indexContext = indexContext;
+        this.bytesPerValue = bytesPerValue;
+        this.writer = new BlockBalancedTreeWriter(bytesPerValue, maxPointsInLeafNode);
+    }
+
+    @Override
+    public String toString()
+    {
+        return MoreObjects.toStringHelper(this)
+                          .add("indexContext", indexContext)
+                          .add("bytesPerValue", bytesPerValue)
+                          .toString();
+    }
+
+    private static class LeafCallback implements BlockBalancedTreeWriter.Callback
+    {
+        final List<PackedLongValues> leafPostings = new ArrayList<>(DEFAULT_POSTINGS_SIZE);
+
+        public int numLeaves()
+        {
+            return leafPostings.size();
+        }
+
+        @Override
+        public void writeLeafPostings(BlockBalancedTreeWriter.RowIDAndIndex[] leafPostings, int offset, int count)
+        {
+            PackedLongValues.Builder builder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
+
+            for (int i = offset; i < count; ++i)
+            {
+                builder.add(leafPostings[i].rowID);
+            }
+            this.leafPostings.add(builder.build());
+        }
+    }
+
+    /**
+     * Writes a balanced tree and posting lists from a {@link BlockBalancedTreeIterator}.
+     *
+     * @param values sorted {@link BlockBalancedTreeIterator} values to write
+     *
+     * @return metadata describing the location and size of this balanced tree in the overall SSTable balanced tree component file
+     */
+    public SegmentMetadata.ComponentMetadataMap writeCompleteSegment(BlockBalancedTreeIterator values) throws IOException
+    {
+        long treePosition;
+
+        SegmentMetadata.ComponentMetadataMap components = new SegmentMetadata.ComponentMetadataMap();
+
+        LeafCallback leafCallback = new LeafCallback();
+
+        try (IndexOutput treeOutput = indexDescriptor.openPerIndexOutput(IndexComponent.BALANCED_TREE, indexContext, true))
+        {
+            // The SSTable balanced tree component file is opened in append mode, so our offset is the current file pointer.
+            long treeOffset = treeOutput.getFilePointer();
+
+            treePosition = writer.write(treeOutput, values, leafCallback);
+
+            // If the treePosition is less than 0 then we didn't write any values out and the index is empty
+            if (treePosition < 0)
+                return components;
+
+            long treeLength = treeOutput.getFilePointer() - treeOffset;
+
+            Map<String, String> attributes = new LinkedHashMap<>();
+            attributes.put("max_points_in_leaf_node", Integer.toString(writer.getMaxPointsInLeafNode()));
+            attributes.put("num_leaves", Integer.toString(leafCallback.numLeaves()));
+            attributes.put("num_values", Long.toString(writer.getValueCount()));
+            attributes.put("bytes_per_value", Long.toString(writer.getBytesPerValue()));
+
+            components.put(IndexComponent.BALANCED_TREE, treePosition, treeOffset, treeLength, attributes);
+        }
+
+        try (BlockBalancedTreeWalker reader = new BlockBalancedTreeWalker(indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext), treePosition);
+             IndexOutputWriter postingsOutput = indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext, true))
+        {
+            long postingsOffset = postingsOutput.getFilePointer();
+
+            BlockBalancedTreePostingsWriter postingsWriter = new BlockBalancedTreePostingsWriter();
+            reader.traverse(postingsWriter);
+
+            // The balanced tree postings writer already writes its own header & footer.
+            long postingsPosition = postingsWriter.finish(postingsOutput, leafCallback.leafPostings, indexContext);
+
+            Map<String, String> attributes = new LinkedHashMap<>();
+            attributes.put("num_leaf_postings", Integer.toString(postingsWriter.numLeafPostings));
+            attributes.put("num_non_leaf_postings", Integer.toString(postingsWriter.numNonLeafPostings));
+
+            long postingsLength = postingsOutput.getFilePointer() - postingsOffset;
+            components.put(IndexComponent.POSTING_LISTS, postingsPosition, postingsOffset, postingsLength, attributes);
+        }
+
+        return components;
+    }
+
+    /**
+     * @return number of values added
+     */
+    public long getValueCount()
+    {
+        return writer.getValueCount();
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/FilteringPostingList.java b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/FilteringPostingList.java
new file mode 100644
index 0000000..9140e35
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/FilteringPostingList.java
@@ -0,0 +1,101 @@
+/*
+ * 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.index.sai.disk.v1.postings;
+
+import java.io.IOException;
+
+import org.apache.cassandra.index.sai.postings.OrdinalPostingList;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * A wrapper that iterates over a delegate {@link PostingList}, filtering out postings at
+ * positions that are not present in a provided filter.
+ */
+public class FilteringPostingList implements PostingList
+{
+    private final FixedBitSet filter;
+    private final OrdinalPostingList delegate;
+    private final int cardinality;
+    private int position = 0;
+
+    public FilteringPostingList(FixedBitSet filter, OrdinalPostingList delegate)
+    {
+        cardinality = filter.cardinality();
+        this.filter = filter;
+        this.delegate = delegate;
+    }
+
+    @Override
+    public void close()
+    {
+        delegate.close();
+    }
+
+    /**
+     *
+     * @return the segment row ID of the next match
+     */
+    @Override
+    public long nextPosting() throws IOException
+    {
+        while (true)
+        {
+            long segmentRowId = delegate.nextPosting();
+
+            if (segmentRowId == PostingList.END_OF_STREAM)
+            {
+                return PostingList.END_OF_STREAM;
+            }
+
+            if (filter.get(position++))
+            {
+                return segmentRowId;
+            }
+        }
+    }
+
+    @Override
+    public long size()
+    {
+        return cardinality;
+    }
+
+    @Override
+    public  long advance(long targetRowID) throws IOException
+    {
+        long segmentRowId = delegate.advance(targetRowID);
+
+        if (segmentRowId == PostingList.END_OF_STREAM)
+        {
+            return PostingList.END_OF_STREAM;
+        }
+
+        // these are always for leaf balanced tree postings so the max is 1024
+        position = (int)delegate.getOrdinal();
+
+        // If the ordinal of the ID we just read satisfies the filter, just return it...
+        if (filter.get(position - 1))
+        {
+            return segmentRowId;
+        }
+
+        // ...but if the ID doesn't satisfy the filter, get the next match.
+        return nextPosting();
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PackedLongsPostingList.java b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PackedLongsPostingList.java
new file mode 100644
index 0000000..34a6ea8
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PackedLongsPostingList.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.index.sai.disk.v1.postings;
+
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+/**
+ * Adapter class for {@link PackedLongValues} to expose it as {@link PostingList}.
+ */
+public class PackedLongsPostingList implements PostingList
+{
+    private final PackedLongValues.Iterator iterator;
+    private final PackedLongValues values;
+
+    public PackedLongsPostingList(PackedLongValues values)
+    {
+        this.values = values;
+        iterator = values.iterator();
+    }
+
+    @Override
+    public long nextPosting()
+    {
+        if (iterator.hasNext())
+        {
+            return iterator.next();
+        }
+        else
+        {
+            return PostingList.END_OF_STREAM;
+        }
+    }
+
+    @Override
+    public long size()
+    {
+        return values.size();
+    }
+
+    @Override
+    public long advance(long targetRowID)
+    {
+        throw new UnsupportedOperationException();
+    }
+}
+
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsReader.java
index 608d837..0a413a5 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsReader.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsReader.java
@@ -74,7 +74,6 @@
 
     public PostingsReader(IndexInput input, BlocksSummary summary, QueryEventListener.PostingListEventListener listener) throws IOException
     {
-        logger.debug("Opening postings reader for {}", input);
         this.input = input;
         this.seekingInput = new SeekingRandomAccessInput(input);
         this.listener = listener;
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsWriter.java
index 8f12532..8e428d5 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsWriter.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsWriter.java
@@ -43,7 +43,7 @@
 
 /**
  * Encodes, compresses and writes postings lists to disk.
- *
+ * <p>
  * All postings in the posting list are delta encoded, then deltas are divided into blocks for compression.
  * The deltas are based on the final value of the previous block. For the first block in the posting list
  * the first value in the block is written as a VLong prior to block delta encodings.
@@ -114,16 +114,19 @@
         this(indexDescriptor, indexContext, BLOCK_SIZE);
     }
 
+    public PostingsWriter(IndexOutputWriter dataOutput) throws IOException
+    {
+        this(dataOutput, BLOCK_SIZE);
+    }
+
     @VisibleForTesting
     PostingsWriter(IndexDescriptor indexDescriptor, IndexContext indexContext, int blockSize) throws IOException
     {
         this(indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext, true), blockSize);
     }
 
-    private PostingsWriter(IndexOutput dataOutput, int blockSize) throws IOException
+    private PostingsWriter(IndexOutputWriter dataOutput, int blockSize) throws IOException
     {
-        assert dataOutput instanceof IndexOutputWriter;
-        logger.debug("Creating postings writer for output {}", dataOutput);
         this.blockSize = blockSize;
         this.dataOutput = dataOutput;
         startOffset = dataOutput.getFilePointer();
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/IndexSegmentSearcher.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/IndexSegmentSearcher.java
index bb22ff4..b5e8958 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/IndexSegmentSearcher.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/IndexSegmentSearcher.java
@@ -54,12 +54,15 @@
         this.indexContext = indexContext;
     }
 
+    @SuppressWarnings({"resource", "RedundantSuppression"})
     public static IndexSegmentSearcher open(PrimaryKeyMap.Factory primaryKeyMapFactory,
                                             PerColumnIndexFiles indexFiles,
                                             SegmentMetadata segmentMetadata,
                                             IndexContext indexContext) throws IOException
     {
-        return new LiteralIndexSegmentSearcher(primaryKeyMapFactory, indexFiles, segmentMetadata, indexContext);
+        return indexContext.isLiteral()
+               ? new LiteralIndexSegmentSearcher(primaryKeyMapFactory, indexFiles, segmentMetadata, indexContext)
+               : new NumericIndexSegmentSearcher(primaryKeyMapFactory, indexFiles, segmentMetadata, indexContext);
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/NumericIndexSegmentSearcher.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/NumericIndexSegmentSearcher.java
new file mode 100644
index 0000000..9f50695
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/NumericIndexSegmentSearcher.java
@@ -0,0 +1,115 @@
+/*
+ * 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.index.sai.disk.v1.segment;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+
+import com.google.common.base.MoreObjects;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.disk.PrimaryKeyMap;
+import org.apache.cassandra.index.sai.disk.format.IndexComponent;
+import org.apache.cassandra.index.sai.disk.v1.PerColumnIndexFiles;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.BlockBalancedTreeReader;
+import org.apache.cassandra.index.sai.iterators.KeyRangeIterator;
+import org.apache.cassandra.index.sai.metrics.MulticastQueryEventListeners;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.lucene.index.CorruptIndexException;
+
+import static org.apache.cassandra.index.sai.disk.v1.bbtree.BlockBalancedTreeQueries.balancedTreeQueryFrom;
+
+/**
+ * Executes {@link Expression}s against the balanced tree for an individual index segment.
+ */
+public class NumericIndexSegmentSearcher extends IndexSegmentSearcher
+{
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+    private final BlockBalancedTreeReader treeReader;
+    private final QueryEventListener.BalancedTreeEventListener perColumnEventListener;
+
+    NumericIndexSegmentSearcher(PrimaryKeyMap.Factory primaryKeyMapFactory,
+                                PerColumnIndexFiles perIndexFiles,
+                                SegmentMetadata segmentMetadata,
+                                IndexContext indexContext) throws IOException
+    {
+        super(primaryKeyMapFactory, perIndexFiles, segmentMetadata, indexContext);
+
+        final long treePosition = metadata.getIndexRoot(IndexComponent.BALANCED_TREE);
+        if (treePosition < 0)
+            throw new CorruptIndexException(indexContext.logMessage("The tree position is less than zero."), IndexComponent.BALANCED_TREE.name);
+        final long postingsPosition = metadata.getIndexRoot(IndexComponent.POSTING_LISTS);
+        if (postingsPosition < 0)
+            throw new CorruptIndexException(indexContext.logMessage("The postings position is less than zero."), IndexComponent.BALANCED_TREE.name);
+
+        treeReader = new BlockBalancedTreeReader(indexContext,
+                                                 indexFiles.balancedTree(),
+                                                 treePosition,
+                                                 indexFiles.postingLists(),
+                                                 postingsPosition);
+        perColumnEventListener = (QueryEventListener.BalancedTreeEventListener)indexContext.getColumnQueryMetrics();
+    }
+
+    @Override
+    public long indexFileCacheSize()
+    {
+        return treeReader.memoryUsage();
+    }
+
+    @Override
+    @SuppressWarnings({"resource", "RedundantSuppression"})
+    public KeyRangeIterator search(Expression exp, QueryContext context) throws IOException
+    {
+        if (logger.isTraceEnabled())
+            logger.trace(indexContext.logMessage("Searching on expression '{}'..."), exp);
+
+        if (exp.getOp().isEqualityOrRange())
+        {
+            final BlockBalancedTreeReader.IntersectVisitor query = balancedTreeQueryFrom(exp, treeReader.getBytesPerValue());
+            QueryEventListener.BalancedTreeEventListener listener = MulticastQueryEventListeners.of(context, perColumnEventListener);
+            PostingList postingList = treeReader.intersect(query, listener, context);
+            return toIterator(postingList, context);
+        }
+        else
+        {
+            throw new IllegalArgumentException(indexContext.logMessage("Unsupported expression during index query: " + exp));
+        }
+    }
+
+    @Override
+    public String toString()
+    {
+        return MoreObjects.toStringHelper(this)
+                          .add("indexContext", indexContext)
+                          .add("count", treeReader.getPointCount())
+                          .add("bytesPerValue", treeReader.getBytesPerValue())
+                          .toString();
+    }
+
+    @Override
+    public void close()
+    {
+        treeReader.close();
+    }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/Segment.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/Segment.java
index 5f108a6..8c84991 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/Segment.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/Segment.java
@@ -36,7 +36,7 @@
 import org.apache.cassandra.io.util.FileUtils;
 
 /**
- * Each segment represents an on-disk index structure (kdtree/terms/postings) flushed by memory limit or token boundaries.
+ * Each segment represents an on-disk index structure (balanced tree/terms/postings) flushed by memory limit or token boundaries.
  * It also helps to reduce resource consumption for read requests as only segments that intersect with read request data
  * range need to be loaded.
  */
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentBuilder.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentBuilder.java
index dd86662..44ee9fc 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentBuilder.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentBuilder.java
@@ -28,9 +28,11 @@
 
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.index.sai.IndexContext;
-import org.apache.cassandra.index.sai.memory.RAMStringIndexer;
 import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.BlockBalancedTreeRamBuffer;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.NumericIndexWriter;
 import org.apache.cassandra.index.sai.disk.v1.trie.LiteralIndexWriter;
+import org.apache.cassandra.index.sai.memory.RAMStringIndexer;
 import org.apache.cassandra.index.sai.utils.NamedMemoryLimiter;
 import org.apache.cassandra.index.sai.utils.PrimaryKey;
 import org.apache.cassandra.index.sai.utils.TypeUtil;
@@ -54,7 +56,6 @@
 
     /** Minimum flush size, dynamically updated as segment builds are started and completed/aborted. */
     private static volatile long minimumFlushBytes;
-    private final AbstractType<?> termComparator;
     private final NamedMemoryLimiter limiter;
     private final long lastValidSegmentRowID;
     private boolean flushed = false;
@@ -71,10 +72,49 @@
     private ByteBuffer minTerm;
     private ByteBuffer maxTerm;
 
+    final AbstractType<?> termComparator;
     long totalBytesAllocated;
     int rowCount = 0;
     int maxSegmentRowId = -1;
 
+    public static class BlockBalancedTreeSegmentBuilder extends SegmentBuilder
+    {
+        private final byte[] scratch;
+        private final BlockBalancedTreeRamBuffer trieBuffer;
+
+        public BlockBalancedTreeSegmentBuilder(AbstractType<?> termComparator, NamedMemoryLimiter limiter)
+        {
+            super(termComparator, limiter);
+
+            scratch = new byte[TypeUtil.fixedSizeOf(termComparator)];
+            trieBuffer = new BlockBalancedTreeRamBuffer(TypeUtil.fixedSizeOf(termComparator));
+            totalBytesAllocated = this.trieBuffer.memoryUsed();
+        }
+
+        @Override
+        public boolean isEmpty()
+        {
+            return trieBuffer.numRows() == 0;
+        }
+
+        @Override
+        protected long addInternal(ByteBuffer term, int segmentRowId)
+        {
+            TypeUtil.toComparableBytes(term, termComparator, scratch);
+            return trieBuffer.add(segmentRowId, scratch);
+        }
+
+        @Override
+        protected SegmentMetadata.ComponentMetadataMap flushInternal(IndexDescriptor indexDescriptor, IndexContext indexContext) throws IOException
+        {
+            NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+                                                               indexContext,
+                                                               TypeUtil.fixedSizeOf(termComparator),
+                                                               maxSegmentRowId);
+            return writer.writeCompleteSegment(trieBuffer.iterator());
+        }
+    }
+
     public static class RAMStringSegmentBuilder extends SegmentBuilder
     {
         final RAMStringIndexer ramIndexer;
@@ -89,11 +129,13 @@
             totalBytesAllocated = ramIndexer.estimatedBytesUsed();
         }
 
+        @Override
         public boolean isEmpty()
         {
             return ramIndexer.isEmpty();
         }
 
+        @Override
         protected long addInternal(ByteBuffer term, int segmentRowId)
         {
             copyBufferToBytesRef(term, stringBuffer);
@@ -128,7 +170,7 @@
     {
         this.termComparator = termComparator;
         this.limiter = limiter;
-        this.lastValidSegmentRowID = testLastValidSegmentRowId >= 0 ? testLastValidSegmentRowId : LAST_VALID_SEGMENT_ROW_ID;
+        lastValidSegmentRowID = testLastValidSegmentRowId >= 0 ? testLastValidSegmentRowId : LAST_VALID_SEGMENT_ROW_ID;
 
         minimumFlushBytes = limiter.limitBytes() / ACTIVE_BUILDER_COUNT.incrementAndGet();
     }
@@ -204,7 +246,7 @@
 
     /**
      * This method does three things:
-     *
+     * <p>
      * 1. It decrements active builder count and updates the global minimum flush size to reflect that.
      * 2. It releases the builder's memory against its limiter.
      * 3. It defensively marks the builder inactive to make sure nothing bad happens if we try to close it twice.
diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java
index 89b8f7f..f5bca79 100644
--- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java
+++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java
@@ -41,13 +41,9 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(KeyRangeIntersectionIterator.class);
 
-    // The cassandra.sai.intersection.clause.limit (default: 2) controls the maximum number of range iterator that
-    // will be used in the final intersection of a query operation.
-    private static final int INTERSECTION_CLAUSE_LIMIT = CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.getInt();
-
     static
     {
-        logger.info(String.format("Storage attached index intersection clause limit is %d", INTERSECTION_CLAUSE_LIMIT));
+        logger.info(String.format("Storage attached index intersection clause limit is %d", CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.getInt()));
     }
 
     private final List<KeyRangeIterator> ranges;
@@ -133,7 +129,7 @@
 
     public static Builder builder(int size)
     {
-        return builder(size, INTERSECTION_CLAUSE_LIMIT);
+        return new Builder(size);
     }
 
     @VisibleForTesting
@@ -145,6 +141,9 @@
     @VisibleForTesting
     public static class Builder extends KeyRangeIterator.Builder
     {
+        // This controls the maximum number of range iterators that will be used in the final
+        // intersection of a query operation. It is set from cassandra.sai.intersection_clause_limit
+        // and defaults to 2
         private final int limit;
         // tracks if any of the added ranges are disjoint with the other ranges, which is useful
         // in case of intersection, as it gives a direct answer whether the iterator is going
@@ -153,6 +152,11 @@
 
         protected final List<KeyRangeIterator> rangeIterators;
 
+        Builder(int size)
+        {
+            this(size, CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.getInt());
+        }
+
         Builder(int size, int limit)
         {
             super(new IntersectionStatistics());
diff --git a/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java
index d9554e4..e84a3f9 100644
--- a/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java
+++ b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java
@@ -218,7 +218,6 @@
     {
         return isLiteral ? version -> ByteSourceInverse.unescape(ByteSource.peekable(term.asComparableBytes(version)))
                          : term;
-
     }
 
     private ByteSource terminated(ByteSource src)
diff --git a/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java
index 977f5f4..7259ecc 100644
--- a/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java
+++ b/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java
@@ -70,6 +70,61 @@
         }
     }
 
+    public static class BalancedTreeIndexMetrics extends ColumnQueryMetrics implements QueryEventListener.BalancedTreeEventListener
+    {
+        private static final String BALANCED_TREE_POSTINGS_TYPE = "BalancedTreePostings";
+
+        /**
+         * Balanced Tree index metrics.
+         */
+        private final Timer intersectionLatency;
+        private final Meter postingsNumPostings;
+        private final Meter intersectionEarlyExits;
+
+        private final QueryEventListener.PostingListEventListener postingsListener;
+
+        public BalancedTreeIndexMetrics(IndexContext indexContext)
+        {
+            super(indexContext);
+
+            intersectionLatency = Metrics.timer(createMetricName("BalancedTreeIntersectionLatency"));
+            intersectionEarlyExits = Metrics.meter(createMetricName("BalancedTreeIntersectionEarlyExits"));
+
+            postingsNumPostings = Metrics.meter(createMetricName("NumPostings", BALANCED_TREE_POSTINGS_TYPE));
+
+            Meter postingDecodes = Metrics.meter(createMetricName("PostingDecodes", BALANCED_TREE_POSTINGS_TYPE));
+
+            postingsListener = new PostingListEventsMetrics(postingDecodes);
+        }
+
+        @Override
+        public void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit)
+        {
+            intersectionLatency.update(intersectionTotalTime, unit);
+        }
+
+        @Override
+        public void onIntersectionEarlyExit()
+        {
+            intersectionEarlyExits.mark();
+        }
+
+        @Override
+        public void postingListsHit(int count)
+        {
+            postingsNumPostings.mark(count);
+        }
+
+        @Override
+        public void onSegmentHit() { }
+
+        @Override
+        public QueryEventListener.PostingListEventListener postingListEventListener()
+        {
+            return postingsListener;
+        }
+    }
+
     private static class PostingListEventsMetrics implements QueryEventListener.PostingListEventListener
     {
         private final Meter postingDecodes;
diff --git a/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java b/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java
index f0b81cc..5495f2f 100644
--- a/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java
+++ b/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java
@@ -28,6 +28,11 @@
         return new Multicast2TrieIndexEventListener(ctx, listener);
     }
 
+    public static QueryEventListener.BalancedTreeEventListener of(QueryContext ctx, QueryEventListener.BalancedTreeEventListener listener)
+    {
+        return new Multicast2BalancedTreeEventListener(ctx, listener);
+    }
+
     public static class Multicast2TrieIndexEventListener implements QueryEventListener.TrieIndexEventListener
     {
         private final QueryContext ctx;
@@ -62,6 +67,79 @@
         }
     }
 
+    public static class Multicast2BalancedTreeEventListener implements QueryEventListener.BalancedTreeEventListener
+    {
+        private final QueryContext ctx;
+        private final QueryEventListener.BalancedTreeEventListener listener;
+        private final Multicast2BalancedTreePostingListEventListener postingListEventListener;
+
+        private Multicast2BalancedTreeEventListener(QueryContext ctx, QueryEventListener.BalancedTreeEventListener listener)
+        {
+            this.ctx = ctx;
+            this.listener = listener;
+            this.postingListEventListener = new Multicast2BalancedTreePostingListEventListener(ctx, listener.postingListEventListener());
+        }
+
+        @Override
+        public void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit)
+        {
+            listener.onIntersectionComplete(intersectionTotalTime, unit);
+        }
+
+        @Override
+        public void onIntersectionEarlyExit()
+        {
+            listener.onIntersectionEarlyExit();
+        }
+
+        @Override
+        public void postingListsHit(int count)
+        {
+            ctx.balancedTreePostingListsHit++;
+            listener.postingListsHit(count);
+        }
+
+        @Override
+        public void onSegmentHit()
+        {
+            ctx.segmentsHit++;
+            ctx.balancedTreeSegmentsHit++;
+            listener.onSegmentHit();
+        }
+
+        @Override
+        public QueryEventListener.PostingListEventListener postingListEventListener()
+        {
+            return postingListEventListener;
+        }
+    }
+
+    public static class Multicast2BalancedTreePostingListEventListener implements QueryEventListener.PostingListEventListener
+    {
+        private final QueryContext ctx;
+        private final QueryEventListener.PostingListEventListener listener;
+
+        Multicast2BalancedTreePostingListEventListener(QueryContext ctx, QueryEventListener.PostingListEventListener listener)
+        {
+            this.ctx = ctx;
+            this.listener = listener;
+        }
+
+        @Override
+        public void onAdvance()
+        {
+            ctx.balancedTreePostingsSkips++;
+            listener.onAdvance();
+        }
+
+        @Override
+        public void postingDecoded(long postingDecoded)
+        {
+            ctx.balancedTreePostingsDecodes += postingDecoded;
+            listener.postingDecoded(postingDecoded);
+        }
+    }
+
     public static class Multicast2TriePostingListEventListener implements QueryEventListener.PostingListEventListener
     {
         private final QueryContext ctx;
diff --git a/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java b/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java
index 7bae429..db583d8 100644
--- a/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java
+++ b/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java
@@ -24,6 +24,37 @@
  */
 public interface QueryEventListener
 {
+    /**
+     * Collector for balanced tree file related metrics.
+     */
+    interface BalancedTreeEventListener
+    {
+        /**
+         * Per-segment balanced tree index intersection time in given units. Recorded when intersection completes.
+         */
+        void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit);
+
+        /**
+         * When an intersection exits early due to the query shape being completely outside the min/max range.
+         */
+        void onIntersectionEarlyExit();
+
+        /**
+         * How many balanced tree posting list were matched during the intersection.
+         */
+        void postingListsHit(int count);
+
+        /**
+         * When query potentially matches value range within a segment, and we need to do a traversal.
+         */
+        void onSegmentHit();
+
+        /**
+         * Returns events listener for balanced tree postings.
+         */
+        PostingListEventListener postingListEventListener();
+    }
+
     interface TrieIndexEventListener
     {
         /**
@@ -62,13 +93,11 @@
             @Override
             public void onAdvance()
             {
-
             }
 
             @Override
             public void postingDecoded(long postingsDecoded)
             {
-
             }
         };
     }
diff --git a/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java
index 8e7b141..7154df2 100644
--- a/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java
+++ b/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java
@@ -78,6 +78,16 @@
         private final Histogram rowsFiltered;
 
         /**
+         * Balanced tree index metrics.
+         */
+        private final Histogram balancedTreePostingsNumPostings;
+        /**
+         * Balanced tree index posting lists metrics.
+         */
+        private final Histogram balancedTreePostingsSkips;
+        private final Histogram balancedTreePostingsDecodes;
+
+        /**
          * Trie index posting lists metrics.
          */
         private final Histogram postingsSkips;
@@ -92,6 +102,11 @@
             sstablesHit = Metrics.histogram(createMetricName("SSTableIndexesHit"), false);
             segmentsHit = Metrics.histogram(createMetricName("IndexSegmentsHit"), false);
 
+            balancedTreePostingsSkips = Metrics.histogram(createMetricName("BalancedTreePostingsSkips"), false);
+
+            balancedTreePostingsNumPostings = Metrics.histogram(createMetricName("BalancedTreePostingsNumPostings"), false);
+            balancedTreePostingsDecodes = Metrics.histogram(createMetricName("BalancedTreePostingsDecodes"), false);
+
             postingsSkips = Metrics.histogram(createMetricName("PostingsSkips"), false);
             postingsDecodes = Metrics.histogram(createMetricName("PostingsDecodes"), false);
 
@@ -105,6 +120,14 @@
             postingsDecodes.update(events.triePostingsDecodes);
         }
 
+        private void recordNumericIndexCacheMetrics(QueryContext events)
+        {
+            balancedTreePostingsNumPostings.update(events.balancedTreePostingListsHit);
+
+            balancedTreePostingsSkips.update(events.balancedTreePostingsSkips);
+            balancedTreePostingsDecodes.update(events.balancedTreePostingsDecodes);
+        }
+
         public void record(QueryContext queryContext)
         {
             final long totalQueryTimeNs = queryContext.totalQueryTimeNs();
@@ -133,6 +156,11 @@
                 recordStringIndexCacheMetrics(queryContext);
             }
 
+            if (queryContext.balancedTreeSegmentsHit > 0)
+            {
+                recordNumericIndexCacheMetrics(queryContext);
+            }
+
             totalQueriesCompleted.inc();
         }
     }
diff --git a/src/java/org/apache/cassandra/index/sai/plan/Expression.java b/src/java/org/apache/cassandra/index/sai/plan/Expression.java
index 52ebd3d..a36bf31 100644
--- a/src/java/org/apache/cassandra/index/sai/plan/Expression.java
+++ b/src/java/org/apache/cassandra/index/sai/plan/Expression.java
@@ -68,6 +68,11 @@
         {
             return this == EQ || this == CONTAINS_KEY || this == CONTAINS_VALUE;
         }
+
+        public boolean isEqualityOrRange()
+        {
+            return isEquality() || this == RANGE;
+        }
     }
 
     public final AbstractAnalyzer.AnalyzerFactory analyzerFactory;
diff --git a/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java b/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java
index 17d37f0..0a5fdf6 100644
--- a/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java
+++ b/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java
@@ -20,6 +20,7 @@
 import java.util.concurrent.atomic.AtomicLong;
 import javax.annotation.concurrent.ThreadSafe;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,10 +34,11 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(NamedMemoryLimiter.class);
     
-    private final long limitBytes;
     private final AtomicLong bytesUsed = new AtomicLong(0);
     private final String scope;
 
+    private long limitBytes;
+
     public NamedMemoryLimiter(long limitBytes, String scope)
     {
         this.limitBytes = limitBytes;
@@ -76,4 +78,10 @@
     {
         return limitBytes;
     }
+
+    @VisibleForTesting
+    public void setLimitBytes(long bytes)
+    {
+        limitBytes = bytes;
+    }
 }
diff --git a/src/java/org/apache/cassandra/index/sai/utils/TermsIterator.java b/src/java/org/apache/cassandra/index/sai/utils/TermsIterator.java
index 0311e7a..6f3c1fb 100644
--- a/src/java/org/apache/cassandra/index/sai/utils/TermsIterator.java
+++ b/src/java/org/apache/cassandra/index/sai/utils/TermsIterator.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.index.sai.utils;
 
 import java.io.Closeable;
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
 import javax.annotation.concurrent.NotThreadSafe;
@@ -29,7 +28,7 @@
 
 /**
  * Iterator to step through terms to obtain {@link PostingList} for the current term.
- *
+ * <p>
  * Term enumerations are always ordered by their {@link ByteSource}.
  */
 @NotThreadSafe
@@ -38,7 +37,7 @@
     /**
      * Get {@link PostingList} for the current term.
      */
-    PostingList postings() throws IOException;
+    PostingList postings();
 
     ByteBuffer getMinTerm();
 
diff --git a/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java b/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java
index 90276d8..acdaf4c 100644
--- a/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java
+++ b/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java
@@ -30,7 +30,6 @@
 import com.googlecode.concurrenttrees.radix.ConcurrentRadixTree;
 import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.BooleanType;
 import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.CollectionType;
@@ -38,8 +37,9 @@
 import org.apache.cassandra.db.marshal.DecimalType;
 import org.apache.cassandra.db.marshal.InetAddressType;
 import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.db.marshal.ReversedType;
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.StringType;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.ComplexColumnData;
 import org.apache.cassandra.index.sai.plan.Expression;
@@ -63,6 +63,12 @@
      */
     public static final int DECIMAL_APPROXIMATION_BYTES = 24;
 
+    public static final int BIG_INTEGER_APPROXIMATION_BYTES = 20;
+
+    public static final int INET_ADDRESS_SIZE = 16;
+
+    public static final int DEFAULT_FIXED_LENGTH = 16;
+
     private TypeUtil() {}
 
     /**
@@ -130,6 +136,23 @@
         return a == null ? b : (b == null || ByteComparable.compare(b, a, ByteComparable.Version.OSS50) < 0) ? a : b;
     }
 
+    /**
+     * Returns the value length for the given {@link AbstractType}, selecting 16 for types
+     * that officially use VARIABLE_LENGTH but are, in fact, of a fixed length.
+     */
+    public static int fixedSizeOf(AbstractType<?> type)
+    {
+        if (type.isValueLengthFixed())
+            return type.valueLengthIfFixed();
+        else if (isInetAddress(type))
+            return INET_ADDRESS_SIZE;
+        else if (isBigInteger(type))
+            return BIG_INTEGER_APPROXIMATION_BYTES;
+        else if (isBigDecimal(type))
+            return DECIMAL_APPROXIMATION_BYTES;
+        return DEFAULT_FIXED_LENGTH;
+    }
+
     public static AbstractType<?> cellValueType(ColumnMetadata columnMetadata, IndexTarget.Type indexType)
     {
         AbstractType<?> type = columnMetadata.type;
@@ -186,10 +209,39 @@
     {
         if (type instanceof InetAddressType || type instanceof IntegerType || type instanceof DecimalType)
             return ByteSource.optionalFixedLength(ByteBufferAccessor.instance, value);
+        // The LongType.asComparableBytes uses variableLengthInteger which doesn't play well with
+        // the balanced tree because it is expecting fixed length data. So for SAI we use a optionalSignedFixedLengthNumber
+        // to keep all comparable values the same length
+        else if (type instanceof LongType)
+            return ByteSource.optionalSignedFixedLengthNumber(ByteBufferAccessor.instance, value);
         return type.asComparableBytes(value, version);
     }
 
     /**
+     * Fills a byte array with the comparable bytes for a type.
+     * <p>
+     * This method expects a {@code value} parameter generated by calling {@link #asIndexBytes(ByteBuffer, AbstractType)}.
+     * It is not generally safe to pass the output of other serialization methods to this method.  For instance, it is
+     * not generally safe to pass the output of {@link AbstractType#decompose(Object)} as the {@code value} parameter
+     * (there are certain types for which this is technically OK, but that doesn't hold for all types).
+     *
+     * @param value a value buffer returned by {@link #asIndexBytes(ByteBuffer, AbstractType)}
+     * @param type the type associated with the encoded {@code value} parameter
+     * @param bytes this method's output
+     */
+    public static void toComparableBytes(ByteBuffer value, AbstractType<?> type, byte[] bytes)
+    {
+        if (isInetAddress(type))
+            ByteBufferUtil.copyBytes(value, value.hasArray() ? value.arrayOffset() + value.position() : value.position(), bytes, 0, INET_ADDRESS_SIZE);
+        else if (isBigInteger(type))
+            ByteBufferUtil.copyBytes(value, value.hasArray() ? value.arrayOffset() + value.position() : value.position(), bytes, 0, BIG_INTEGER_APPROXIMATION_BYTES);
+        else if (isBigDecimal(type))
+            ByteBufferUtil.copyBytes(value, value.hasArray() ? value.arrayOffset() + value.position() : value.position(), bytes, 0, DECIMAL_APPROXIMATION_BYTES);
+        else
+            ByteSourceInverse.copyBytes(asComparableBytes(value, type, ByteComparable.Version.OSS50), bytes);
+    }
+
+    /**
      * Translates the external value of specific types into a format used by the index.
      */
     public static ByteBuffer asIndexBytes(ByteBuffer value, AbstractType<?> type)
@@ -307,7 +359,7 @@
 
     private static boolean isIPv6(ByteBuffer address)
     {
-        return address.remaining() == 16;
+        return address.remaining() == INET_ADDRESS_SIZE;
     }
 
     /**
@@ -322,7 +374,7 @@
         if (value.remaining() == 4)
         {
             int position = value.hasArray() ? value.arrayOffset() + value.position() : value.position();
-            ByteBuffer mapped = ByteBuffer.allocate(16);
+            ByteBuffer mapped = ByteBuffer.allocate(INET_ADDRESS_SIZE);
             System.arraycopy(IPV4_PREFIX, 0, mapped.array(), 0, IPV4_PREFIX.length);
             ByteBufferUtil.copyBytes(value, position, mapped, IPV4_PREFIX.length, value.remaining());
             return mapped;
@@ -337,7 +389,7 @@
      *
      * The format of the encoding is:
      *
-     *  The first 4 bytes contain the length of the {@link BigInteger} byte array
+     *  The first 4 bytes contain the integer length of the {@link BigInteger} byte array
      *  with the top bit flipped for positive values.
      *
      *  The remaining 16 bytes contain the 16 most significant bytes of the
@@ -350,18 +402,18 @@
     {
         int size = value.remaining();
         int position = value.hasArray() ? value.arrayOffset() + value.position() : value.position();
-        byte[] bytes = new byte[20];
-        if (size < 16)
+        byte[] bytes = new byte[BIG_INTEGER_APPROXIMATION_BYTES];
+        if (size < BIG_INTEGER_APPROXIMATION_BYTES - Integer.BYTES)
         {
             ByteBufferUtil.copyBytes(value, position, bytes, bytes.length - size, size);
             if ((bytes[bytes.length - size] & 0x80) != 0)
-                Arrays.fill(bytes, 4, bytes.length - size, (byte)0xff);
+                Arrays.fill(bytes, Integer.BYTES, bytes.length - size, (byte)0xff);
             else
-                Arrays.fill(bytes, 4, bytes.length - size, (byte)0x00);
+                Arrays.fill(bytes, Integer.BYTES, bytes.length - size, (byte)0x00);
         }
         else
         {
-            ByteBufferUtil.copyBytes(value, position, bytes, 4, 16);
+            ByteBufferUtil.copyBytes(value, position, bytes, Integer.BYTES, BIG_INTEGER_APPROXIMATION_BYTES - Integer.BYTES);
         }
         if ((bytes[4] & 0x80) != 0)
         {
@@ -380,16 +432,16 @@
      */
     public static boolean isLiteral(AbstractType<?> type)
     {
-        return isUTF8OrAscii(type) || isCompositeOrFrozen(type) || baseType(type) instanceof BooleanType;
+        return isString(type) || isCompositeOrFrozen(type) || baseType(type) instanceof BooleanType;
     }
 
     /**
-     * Returns <code>true</code> if given {@link AbstractType} is UTF8 or Ascii
+     * Returns <code>true</code> if given {@link AbstractType} is based on a string, e.g. UTF8 or Ascii
      */
-    public static boolean isUTF8OrAscii(AbstractType<?> type)
+    public static boolean isString(AbstractType<?> type)
     {
         type = baseType(type);
-        return type instanceof UTF8Type || type instanceof AsciiType;
+        return type instanceof StringType;
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java
index 16b6679..4bf9d8c 100644
--- a/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java
+++ b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java
@@ -402,6 +402,19 @@
         return readBytes(byteSource, INITIAL_BUFFER_CAPACITY);
     }
 
+    public static void copyBytes(ByteSource byteSource, byte[] bytes)
+    {
+        int readBytes = 0;
+
+        int data;
+        while ((data = byteSource.next()) != ByteSource.END_OF_STREAM)
+        {
+            if (bytes.length == readBytes)
+                throw new ArrayIndexOutOfBoundsException(String.format("Number of bytes read, %d, exceeds the buffer size of %d.", readBytes + 1, bytes.length));
+            bytes[readBytes++] = (byte) data;
+        }
+    }
+
     /**
      * Ensures the given buffer has capacity for taking data with the given length - if it doesn't, it returns a copy
      * of the buffer, but with double the capacity.
diff --git a/test/distributed/org/apache/cassandra/distributed/test/sai/IndexStreamingTest.java b/test/distributed/org/apache/cassandra/distributed/test/sai/IndexStreamingTest.java
index 3069e80..6b8217f 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/sai/IndexStreamingTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/sai/IndexStreamingTest.java
@@ -19,9 +19,13 @@
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.distributed.Cluster;
@@ -36,19 +40,16 @@
 
 import static org.assertj.core.api.Assertions.assertThat;
 
+@RunWith(Parameterized.class)
 public class IndexStreamingTest extends TestBaseImpl
 {
     // streaming sends events every 65k, so need to make sure that the files are larger than this to hit
     // all cases of the vtable - hence we add a big enough blob column
     private static final ByteBuffer BLOB = ByteBuffer.wrap(new byte[1 << 16]);
-    private static final int NUM_COMPONENTS;
 
     static
     {
         DatabaseDescriptor.clientInitialization();
-        NUM_COMPONENTS = sstableStreamingComponentsCount()
-                         + V1OnDiskFormat.PER_SSTABLE_COMPONENTS.size()
-                         + V1OnDiskFormat.LITERAL_COMPONENTS.size();
     }
 
     private static int sstableStreamingComponentsCount()
@@ -60,42 +61,62 @@
                                        .count() - 1;  // -1 because we don't include the compression component
     }
 
-    @Test
-    public void zeroCopy() throws IOException
+    @SuppressWarnings("DefaultAnnotationParam")
+    @Parameterized.Parameter(0)
+    public boolean isLiteral;
+    @Parameterized.Parameter(1)
+    public boolean isZeroCopyStreaming;
+
+    @Parameterized.Parameters(name = "isLiteral={0}, isZeroCopyStreaming={1}")
+    public static List<Object[]> data()
     {
-        test(true);
+        List<Object[]> result = new ArrayList<>();
+        result.add(new Object[]{ true, true });
+        result.add(new Object[]{ true, false });
+        result.add(new Object[]{ false, true });
+        result.add(new Object[]{ false, false });
+        return result;
     }
 
     @Test
-    public void notZeroCopy() throws IOException
-    {
-        test(false);
-    }
-
-    private void test(boolean zeroCopyStreaming) throws IOException
+    public void testIndexComponentStreaming() throws IOException
     {
         try (Cluster cluster = init(Cluster.build(2)
                                            .withConfig(c -> c.with(Feature.values())
-                                                             .set("stream_entire_sstables", zeroCopyStreaming)
+                                                             .set("stream_entire_sstables", isZeroCopyStreaming)
                                                              .set("streaming_slow_events_log_timeout", "0s"))
                                            .start()))
         {
             cluster.schemaChange(withKeyspace(
-                "CREATE TABLE %s.test (pk int PRIMARY KEY, v text, b blob) WITH compression = { 'enabled' : false };"
+                "CREATE TABLE %s.test (pk int PRIMARY KEY, literal text, numeric int, b blob) WITH compression = { 'enabled' : false };"
             ));
-            cluster.schemaChange(withKeyspace(
-                "CREATE CUSTOM INDEX ON %s.test(v) USING 'StorageAttachedIndex';"
-            ));
+
+            int num_components = isLiteral ? sstableStreamingComponentsCount() +
+                                             V1OnDiskFormat.PER_SSTABLE_COMPONENTS.size() +
+                                             V1OnDiskFormat.LITERAL_COMPONENTS.size()
+                                           : sstableStreamingComponentsCount() +
+                                             V1OnDiskFormat.PER_SSTABLE_COMPONENTS.size() +
+                                             V1OnDiskFormat.NUMERIC_COMPONENTS.size();
+
+            if (isLiteral)
+                cluster.schemaChange(withKeyspace(
+                    "CREATE CUSTOM INDEX ON %s.test(literal) USING 'StorageAttachedIndex';"
+                ));
+            else
+                cluster.schemaChange(withKeyspace(
+                "CREATE CUSTOM INDEX ON %s.test(numeric) USING 'StorageAttachedIndex';"
+                ));
+
             cluster.stream().forEach(i ->
                 i.nodetoolResult("disableautocompaction", KEYSPACE).asserts().success()
             );
             IInvokableInstance first = cluster.get(1);
             IInvokableInstance second = cluster.get(2);
             long sstableCount = 10;
-            long expectedFiles = zeroCopyStreaming ? sstableCount * NUM_COMPONENTS : sstableCount;
+            long expectedFiles = isZeroCopyStreaming ? sstableCount * num_components : sstableCount;
             for (int i = 0; i < sstableCount; i++)
             {
-                first.executeInternal(withKeyspace("insert into %s.test(pk, v, b) values (?, ?, ?)"), i, "v" + i, BLOB);
+                first.executeInternal(withKeyspace("insert into %s.test(pk, literal, numeric, b) values (?, ?, ?, ?)"), i, "v" + i, i, BLOB);
                 first.flush(KEYSPACE);
             }
 
@@ -146,7 +167,8 @@
 
             for (int i = 0; i < sstableCount; i++)
             {
-                Object[][] rs = second.executeInternal(withKeyspace("select pk from %s.test where v = ?"), "v" + i);
+                Object[][] rs = isLiteral ? second.executeInternal(withKeyspace("select pk from %s.test where literal = ?"), "v" + i)
+                                          : second.executeInternal(withKeyspace("select pk from %s.test where numeric = ?"), i);
                 assertThat(rs.length).isEqualTo(1);
                 assertThat(rs[0][0]).isEqualTo(i);
             }
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index d2281cb..afdcfaf 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -778,6 +778,13 @@
         disableCompaction(KEYSPACE);
     }
 
+    public void disableCompaction(String keyspace, String table)
+    {
+        ColumnFamilyStore store = getColumnFamilyStore(keyspace, table);
+        if (store != null)
+            store.disableAutoCompaction();
+    }
+
     public void enableCompaction(String keyspace)
     {
         ColumnFamilyStore store = getCurrentColumnFamilyStore(keyspace);
@@ -1572,7 +1579,7 @@
      * Executes the provided query using the {@link ClientState#forInternalCalls()} as the expected ClientState. Note:
      * this means permissions checking will not apply and queries will proceed regardless of role or guardrails.
      */
-    protected UntypedResultSet executeFormattedQuery(String query, Object... values)
+    public UntypedResultSet executeFormattedQuery(String query, Object... values)
     {
         UntypedResultSet rs;
         if (usePrepared)
diff --git a/test/unit/org/apache/cassandra/index/sai/SAITester.java b/test/unit/org/apache/cassandra/index/sai/SAITester.java
index db8a6ea..903ef1b 100644
--- a/test/unit/org/apache/cassandra/index/sai/SAITester.java
+++ b/test/unit/org/apache/cassandra/index/sai/SAITester.java
@@ -24,8 +24,6 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.RandomAccessFile;
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.channels.FileChannel;
@@ -61,6 +59,7 @@
 import com.datastax.driver.core.ResultSet;
 import com.datastax.driver.core.Session;
 import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -79,7 +78,6 @@
 import org.apache.cassandra.index.sai.disk.format.Version;
 import org.apache.cassandra.index.sai.disk.v1.V1OnDiskFormat;
 import org.apache.cassandra.index.sai.disk.v1.segment.SegmentBuilder;
-import org.apache.cassandra.index.sai.utils.NamedMemoryLimiter;
 import org.apache.cassandra.index.sai.utils.PrimaryKey;
 import org.apache.cassandra.index.sai.utils.ResourceLeakDetector;
 import org.apache.cassandra.inject.Injection;
@@ -114,11 +112,12 @@
     protected static final String CREATE_KEYSPACE_TEMPLATE = "CREATE KEYSPACE IF NOT EXISTS %s WITH replication = " +
                                                              "{'class': 'SimpleStrategy', 'replication_factor': '1'}";
 
-    protected static final String CREATE_TABLE_TEMPLATE = "CREATE TABLE %s (id1 TEXT PRIMARY KEY, v1 TEXT) WITH compaction = " +
+    protected static final String CREATE_TABLE_TEMPLATE = "CREATE TABLE %s (id1 TEXT PRIMARY KEY, v1 INT, v2 TEXT) WITH compaction = " +
                                                           "{'class' : 'SizeTieredCompactionStrategy', 'enabled' : false }";
     protected static final String CREATE_INDEX_TEMPLATE = "CREATE CUSTOM INDEX IF NOT EXISTS ON %%s(%s) USING 'StorageAttachedIndex'";
 
     protected static final ColumnIdentifier V1_COLUMN_IDENTIFIER = ColumnIdentifier.getInterned("v1", true);
+    protected static final ColumnIdentifier V2_COLUMN_IDENTIFIER = ColumnIdentifier.getInterned("v2", true);
 
     protected static final Injections.Counter indexBuildCounter = Injections.newCounter("IndexBuildCounter")
                                                                             .add(newInvokePoint().onClass(CompactionManager.class)
@@ -162,6 +161,9 @@
     public void removeAllInjections()
     {
         Injections.deleteAll();
+        CassandraRelevantProperties.SAI_MINIMUM_POSTINGS_LEAVES.reset();
+        CassandraRelevantProperties.SAI_POSTINGS_SKIP.reset();
+        V1OnDiskFormat.SEGMENT_BUILD_MEMORY_LIMITER.setLimitBytes(V1OnDiskFormat.SEGMENT_BUILD_MEMORY_LIMIT);
     }
 
     public static Randomization getRandom()
@@ -460,17 +462,23 @@
         assertTrue(indexFiles().isEmpty());
     }
 
-    protected void verifyIndexFiles(IndexContext literalIndexContext, int literalFiles)
+    protected void verifyIndexFiles(IndexContext numericIndexContext, IndexContext literalIndexContext, int numericFiles, int literalFiles)
     {
-        verifyIndexFiles(literalIndexContext,
+        verifyIndexFiles(numericIndexContext,
+                         literalIndexContext,
+                         Math.max(numericFiles, literalFiles),
+                         numericFiles,
                          literalFiles,
-                         literalFiles,
+                         numericFiles,
                          literalFiles);
     }
 
-    protected void verifyIndexFiles(IndexContext literalIndexContext,
+    protected void verifyIndexFiles(IndexContext numericIndexContext,
+                                    IndexContext literalIndexContext,
                                     int perSSTableFiles,
+                                    int numericFiles,
                                     int literalFiles,
+                                    int numericCompletionMarkers,
                                     int literalCompletionMarkers)
     {
         Set<File> indexFiles = indexFiles();
@@ -494,6 +502,20 @@
                     assertEquals(stringIndexFiles.toString(), literalFiles, stringIndexFiles.size());
             }
         }
+
+        if (numericIndexContext != null)
+        {
+            for (IndexComponent indexComponent : Version.LATEST.onDiskFormat().perColumnIndexComponents(numericIndexContext))
+            {
+                Set<File> numericIndexFiles = componentFiles(indexFiles,
+                                                             SSTableFormat.Components.Types.CUSTOM.createComponent(Version.LATEST.fileNameFormatter().format(indexComponent,
+                                                                                                                                                             numericIndexContext)));
+                if (isBuildCompletionMarker(indexComponent))
+                    assertEquals(numericCompletionMarkers, numericIndexFiles.size());
+                else
+                    assertEquals(numericIndexFiles.toString(), numericFiles, numericIndexFiles.size());
+            }
+        }
     }
 
     protected void verifySSTableIndexes(String indexName, int count)
@@ -730,19 +752,15 @@
         }
     }
 
-    protected static void setSegmentWriteBufferSpace(final int segmentSize) throws Exception
+    protected static void setBDKPostingsWriterSizing(int minimumPostingsLeaves, int postingsSkip)
     {
-        NamedMemoryLimiter limiter = (NamedMemoryLimiter) V1OnDiskFormat.class.getDeclaredField("SEGMENT_BUILD_MEMORY_LIMITER").get(null);
-        Field limitBytes = limiter.getClass().getDeclaredField("limitBytes");
-        limitBytes.setAccessible(true);
-        Field modifiersField = Field.class.getDeclaredField("modifiers");
-        modifiersField.setAccessible(true);
-        modifiersField.setInt(limitBytes, limitBytes.getModifiers() & ~Modifier.FINAL);
-        limitBytes.set(limiter, segmentSize);
-        limitBytes = V1OnDiskFormat.class.getDeclaredField("SEGMENT_BUILD_MEMORY_LIMIT");
-        limitBytes.setAccessible(true);
-        modifiersField.setInt(limitBytes, limitBytes.getModifiers() & ~Modifier.FINAL);
-        limitBytes.set(limiter, segmentSize);
+        CassandraRelevantProperties.SAI_MINIMUM_POSTINGS_LEAVES.setString(Integer.toString(minimumPostingsLeaves));
+        CassandraRelevantProperties.SAI_POSTINGS_SKIP.setString(Integer.toString(postingsSkip));
+    }
+
+    protected static void setSegmentWriteBufferSpace(final int segmentSize)
+    {
+        V1OnDiskFormat.SEGMENT_BUILD_MEMORY_LIMITER.setLimitBytes(segmentSize);
     }
 
     protected String getSingleTraceStatement(Session session, String query, String contains)
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/BaseDataModel.java b/test/unit/org/apache/cassandra/index/sai/cql/BaseDataModel.java
index e9287e1..654c3e7 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/BaseDataModel.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/BaseDataModel.java
@@ -167,6 +167,11 @@
         tester.createTable(String.format(template, KEYSPACE, nonIndexedTable, keyColumnDefs, normalColumnDefs, primaryKey));
     }
 
+    public void truncateTables(Executor tester) throws Throwable
+    {
+        executeLocal(tester, "TRUNCATE TABLE %s");
+    }
+
     public void createIndexes(Executor tester) throws Throwable
     {
         createIndexes(tester, columns);
@@ -189,6 +194,24 @@
         }
     }
 
+    public void flush(Executor tester) throws Throwable
+    {
+        tester.flush(KEYSPACE, indexedTable);
+        tester.flush(KEYSPACE, nonIndexedTable);
+    }
+
+    public void disableCompaction(Executor tester) throws Throwable
+    {
+        tester.disableCompaction(KEYSPACE, indexedTable);
+        tester.disableCompaction(KEYSPACE, nonIndexedTable);
+    }
+
+    public void compact(Executor tester) throws Throwable
+    {
+        tester.compact(KEYSPACE, indexedTable);
+        tester.compact(KEYSPACE, nonIndexedTable);
+    }
+
     public void insertRows(Executor tester) throws Throwable
     {
         String template = "INSERT INTO %%s (%s, %s) VALUES (%s, %s)";
@@ -210,6 +233,42 @@
         }
     }
 
+    public void updateCells(Executor tester) throws Throwable
+    {
+        executeLocal(tester, String.format("UPDATE %%s SET %s = 9700000000 WHERE p = 0", BIGINT_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = false WHERE p = 1", BOOLEAN_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = '2018-03-10' WHERE p = 2", DATE_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = 8788.06 WHERE p = 3", DOUBLE_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = 2.9 WHERE p = 4", FLOAT_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = '205.204.196.65' WHERE p = 5", INET_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = 27429638 WHERE p = 6", INT_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = 31 WHERE p = 7", SMALLINT_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = 116 WHERE p = 8", TINYINT_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = 'State of Michigan' WHERE p = 9", TEXT_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = '00:20:26' WHERE p = 10", TIME_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = '2009-07-16T00:00:00' WHERE p = 11", TIMESTAMP_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = e37394dc-d17b-11e8-a8d5-f2801f1b9fd1 WHERE p = 12", UUID_COLUMN));
+        executeLocal(tester, String.format("UPDATE %%s SET %s = 1fc81a4c-d17d-11e8-a8d5-f2801f1b9fd1 WHERE p = 13", TIMEUUID_COLUMN));
+    }
+
+    public void deleteCells(Executor tester) throws Throwable
+    {
+        for (int i = 0; i < NORMAL_COLUMNS.size(); i++)
+        {
+            executeLocal(tester, String.format("DELETE %s FROM %%s WHERE p = %s", NORMAL_COLUMNS.get(i).left, i));
+        }
+    }
+
+    public void deleteRows(Executor tester) throws Throwable
+    {
+        String template = "DELETE FROM %%s WHERE p = %d";
+
+        for (int deleted : deletable())
+        {
+            executeLocal(tester, String.format(template, deleted));
+        }
+    }
+
     public void executeLocal(Executor tester, String query, Object... values) throws Throwable
     {
         tester.executeLocal(formatIndexedQuery(query), values);
@@ -270,6 +329,45 @@
         }
 
         @Override
+        public void updateCells(Executor tester) throws Throwable
+        {
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 9700000000 WHERE p = 0 AND c = 0", BIGINT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = false WHERE p = 0 AND c = 1", BOOLEAN_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '2018-03-10' WHERE p = 1 AND c = 0", DATE_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 8788.06 WHERE p = 1 AND c = 1", DOUBLE_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 2.9 WHERE p = 2 AND c = 0", FLOAT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '205.204.196.65' WHERE p = 2 AND c = 1", INET_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 27429638 WHERE p = 3 AND c = 0", INT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 31 WHERE p = 3 AND c = 1", SMALLINT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 116 WHERE p = 4 AND c = 0", TINYINT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 'State of Michigan' WHERE p = 4 AND c = 1", TEXT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '00:20:26' WHERE p = 5 AND c = 0", TIME_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '2009-07-16T00:00:00' WHERE p = 5 AND c = 1", TIMESTAMP_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = e37394dc-d17b-11e8-a8d5-f2801f1b9fd1 WHERE p = 6 AND c = 0", UUID_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 1fc81a4c-d17d-11e8-a8d5-f2801f1b9fd1 WHERE p = 6 AND c = 1", TIMEUUID_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 1896 WHERE p = 7", STATIC_INT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 2020 WHERE p = 100", STATIC_INT_COLUMN)); // static only
+        }
+
+        @Override
+        public void deleteCells(Executor tester) throws Throwable
+        {
+            for (int i = 0; i < NORMAL_COLUMNS.size(); i++)
+            {
+                String[] primaryKey = keys.get(i).split(",");
+                executeLocal(tester, String.format("DELETE %s FROM %%s WHERE p = %s AND c = %s", NORMAL_COLUMNS.get(i).left, primaryKey[0], primaryKey[1]));
+            }
+        }
+
+        @Override
+        public void deleteRows(Executor tester) throws Throwable
+        {
+            executeLocal(tester, "DELETE FROM %s WHERE p = 2 AND c = 0");
+            executeLocal(tester, "DELETE FROM %s WHERE p = 4 AND c = 0");
+            executeLocal(tester, "DELETE FROM %s WHERE p = 6");
+        }
+
+        @Override
         protected Set<Integer> deletable()
         {
             return Sets.newHashSet(4, 8, 12, 13, 100);
@@ -312,6 +410,45 @@
         }
 
         @Override
+        public void updateCells(Executor tester) throws Throwable
+        {
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 9700000000 WHERE p1 = 0 AND p2 = 0", BIGINT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = false WHERE p1 = 0 AND p2 = 1", BOOLEAN_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '2018-03-10' WHERE p1 = 1 AND p2 = 0", DATE_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 8788.06 WHERE p1 = 1 AND p2 = 1", DOUBLE_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 2.9 WHERE p1 = 2 AND p2 = 0", FLOAT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '205.204.196.65' WHERE p1 = 2 AND p2 = 1", INET_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 27429638 WHERE p1 = 3 AND p2 = 0", INT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 31 WHERE p1 = 3 AND p2 = 1", SMALLINT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 116 WHERE p1 = 4 AND p2 = 0", TINYINT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 'State of Michigan' WHERE p1 = 4 AND p2 = 2", TEXT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '00:20:26' WHERE p1 = 5 AND p2 = 3", TIME_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '2009-07-16T00:00:00' WHERE p1 = 5 AND p2 = 1", TIMESTAMP_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = e37394dc-d17b-11e8-a8d5-f2801f1b9fd1 WHERE p1 = 6 AND p2 = 0", UUID_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 1fc81a4c-d17d-11e8-a8d5-f2801f1b9fd1 WHERE p1 = 6 AND p2 = 1", TIMEUUID_COLUMN));
+        }
+
+        @Override
+        public void deleteCells(Executor tester) throws Throwable
+        {
+            for (int i = 0; i < NORMAL_COLUMNS.size(); i++)
+            {
+                String[] primaryKey = keys.get(i).split(",");
+                executeLocal(tester, String.format("DELETE %s FROM %%s WHERE p1 = %s AND p2 = %s",
+                                                   NORMAL_COLUMNS.get(i).left, primaryKey[0], primaryKey[1]));
+            }
+        }
+
+        @Override
+        public void deleteRows(Executor tester) throws Throwable
+        {
+            executeLocal(tester, "DELETE FROM %s WHERE p1 = 2 AND p2 = 0");
+            executeLocal(tester, "DELETE FROM %s WHERE p1 = 4 AND p2 = 1");
+            executeLocal(tester, "DELETE FROM %s WHERE p1 = 6 AND p2 = 2");
+            executeLocal(tester, "DELETE FROM %s WHERE p1 = 8 AND p2 = 0");
+        }
+
+        @Override
         protected Set<Integer> deletable()
         {
             // already overwrites {@code deleteRows()}
@@ -330,6 +467,33 @@
             this.keys = new CompoundPrimaryKeyList(rows.size(), 1);
         }
 
+        @Override
+        public void updateCells(Executor tester) throws Throwable
+        {
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 9700000000 WHERE p = 0 AND c = 0", BIGINT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = false WHERE p = 1 AND c = 0", BOOLEAN_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '2018-03-10' WHERE p = 2 AND c = 0", DATE_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 8788.06 WHERE p = 3 AND c = 0", DOUBLE_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 2.9 WHERE p = 4 AND c = 0", FLOAT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '205.204.196.65' WHERE p = 5 AND c = 0", INET_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 27429638 WHERE p = 6 AND c = 0", INT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 31 WHERE p = 7 AND c = 0", SMALLINT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 116 WHERE p = 8 AND c = 0", TINYINT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 'State of Michigan' WHERE p = 9 AND c = 0", TEXT_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '00:20:26' WHERE p = 10 AND c = 0", TIME_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = '2009-07-16T00:00:00' WHERE p = 11 AND c = 0", TIMESTAMP_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = e37394dc-d17b-11e8-a8d5-f2801f1b9fd1 WHERE p = 12 AND c = 0", UUID_COLUMN));
+            executeLocal(tester, String.format("UPDATE %%s SET %s = 1fc81a4c-d17d-11e8-a8d5-f2801f1b9fd1 WHERE p = 13 AND c = 0", TIMEUUID_COLUMN));
+        }
+
+        @Override
+        public void deleteCells(Executor tester) throws Throwable
+        {
+            for (int i = 0; i < NORMAL_COLUMNS.size(); i++)
+            {
+                executeLocal(tester, String.format("DELETE %s FROM %%s WHERE p = %s AND c = 0", NORMAL_COLUMNS.get(i).left, i));
+            }
+        }
     }
 
     static class SimplePrimaryKeyList extends ForwardingList<String>
@@ -382,6 +546,12 @@
     {
         void createTable(String statement);
 
+        void flush(String keyspace, String table);
+
+        void compact(String keyspace, String table);
+
+        void disableCompaction(String keyspace, String table);
+
         void waitForIndexQueryable(String keyspace, String index);
 
         void executeLocal(String query, Object...values) throws Throwable;
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/IndexQuerySupport.java b/test/unit/org/apache/cassandra/index/sai/cql/IndexQuerySupport.java
index 7573fc9..cd8a047 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/IndexQuerySupport.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/IndexQuerySupport.java
@@ -92,11 +92,100 @@
     {
         dataModel.createTables(executor);
 
+        dataModel.disableCompaction(executor);
+
         dataModel.createIndexes(executor);
 
         // queries against Memtable adjacent in-memory indexes
         dataModel.insertRows(executor);
         executeQueries(dataModel, executor, sets);
+
+        // queries with Memtable flushed to SSTable on disk
+        dataModel.flush(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries across memory and disk indexes
+        dataModel.insertRows(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries w/ multiple SSTable indexes
+        dataModel.flush(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries after compacting to a single SSTable index
+        dataModel.compact(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries against Memtable updates and the existing SSTable index
+        dataModel.updateCells(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries against the newly flushed SSTable index and the existing SSTable index
+        dataModel.flush(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries after compacting updates into to a single SSTable index
+        dataModel.compact(executor);
+        executeQueries(dataModel, executor, sets);
+    }
+
+    public static void rowDeletions(BaseDataModel.Executor executor, BaseDataModel dataModel, List<BaseQuerySet> sets) throws Throwable
+    {
+        dataModel.createTables(executor);
+
+        dataModel.disableCompaction(executor);
+
+        dataModel.createIndexes(executor);
+        dataModel.insertRows(executor);
+        dataModel.flush(executor);
+        dataModel.compact(executor);
+
+        // baseline queries
+        executeQueries(dataModel, executor, sets);
+
+        // queries against Memtable deletes and the existing SSTable index
+        dataModel.deleteRows(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries against the newly flushed SSTable index and the existing SSTable index
+        dataModel.flush(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries after compacting deletes into to a single SSTable index
+        dataModel.compact(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // truncate, reload, and verify that the load is clean
+        dataModel.truncateTables(executor);
+        dataModel.insertRows(executor);
+        executeQueries(dataModel, executor, sets);
+    }
+
+    public static void cellDeletions(BaseDataModel.Executor executor, BaseDataModel dataModel, List<BaseQuerySet> sets) throws Throwable
+    {
+        dataModel.createTables(executor);
+
+        dataModel.disableCompaction(executor);
+
+        dataModel.createIndexes(executor);
+        dataModel.insertRows(executor);
+        dataModel.flush(executor);
+        dataModel.compact(executor);
+
+        // baseline queries
+        executeQueries(dataModel, executor, sets);
+
+        // queries against Memtable deletes and the existing SSTable index
+        dataModel.deleteCells(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries against the newly flushed SSTable index and the existing SSTable index
+        dataModel.flush(executor);
+        executeQueries(dataModel, executor, sets);
+
+        // queries after compacting deletes into to a single SSTable index
+        dataModel.compact(executor);
+        executeQueries(dataModel, executor, sets);
     }
 
     public static void timeToLive(BaseDataModel.Executor executor, BaseDataModel dataModel, List<BaseQuerySet> sets) throws Throwable
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/QueryCellDeletionsTest.java b/test/unit/org/apache/cassandra/index/sai/cql/QueryCellDeletionsTest.java
new file mode 100644
index 0000000..6b42358
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/QueryCellDeletionsTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.index.sai.cql;
+
+import org.junit.Test;
+
+public class QueryCellDeletionsTest extends AbstractQueryTester
+{
+    @Test
+    public void testCellDeletions() throws Throwable
+    {
+        IndexQuerySupport.cellDeletions(executor, dataModel, sets);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/QueryRowDeletionsTest.java b/test/unit/org/apache/cassandra/index/sai/cql/QueryRowDeletionsTest.java
new file mode 100644
index 0000000..a5a843d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/QueryRowDeletionsTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.index.sai.cql;
+
+import org.junit.Test;
+
+public class QueryRowDeletionsTest extends AbstractQueryTester
+{
+    @Test
+    public void testRowDeletions() throws Throwable
+    {
+        IndexQuerySupport.rowDeletions(executor, dataModel, sets);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/SingleNodeExecutor.java b/test/unit/org/apache/cassandra/index/sai/cql/SingleNodeExecutor.java
index 1e86eb4..ea20311 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/SingleNodeExecutor.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/SingleNodeExecutor.java
@@ -43,6 +43,24 @@
     }
 
     @Override
+    public void flush(String keyspace, String table)
+    {
+        tester.flush(keyspace, table);
+    }
+
+    @Override
+    public void compact(String keyspace, String table)
+    {
+        tester.compact(keyspace, table);
+    }
+
+    @Override
+    public void disableCompaction(String keyspace, String table)
+    {
+        tester.disableCompaction(keyspace, table);
+    }
+
+    @Override
     public void waitForIndexQueryable(String keyspace, String index)
     {
         tester.waitForIndexQueryable(keyspace, index);
@@ -51,7 +69,7 @@
     @Override
     public void executeLocal(String query, Object... values) throws Throwable
     {
-        tester.execute(query, values);
+        tester.executeFormattedQuery(query, values);
     }
 
     @Override
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/StorageAttachedIndexDDLTest.java b/test/unit/org/apache/cassandra/index/sai/cql/StorageAttachedIndexDDLTest.java
index 52d364c..fab1161 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/StorageAttachedIndexDDLTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/StorageAttachedIndexDDLTest.java
@@ -47,6 +47,7 @@
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.ReversedType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.index.Index;
@@ -303,7 +304,8 @@
     {
         createTable(CREATE_TABLE_TEMPLATE);
 
-        createIndex("CREATE CUSTOM INDEX ON %s(v1) USING 'org.apache.cassandra.index.sasi.SASIIndex' WITH OPTIONS = {'mode': 'CONTAINS',\n" +
+        createIndex("CREATE CUSTOM INDEX ON %s(v1) USING 'org.apache.cassandra.index.sasi.SASIIndex'");
+        createIndex("CREATE CUSTOM INDEX ON %s(v2) USING 'org.apache.cassandra.index.sasi.SASIIndex' WITH OPTIONS = {'mode': 'CONTAINS',\n" +
                     "'analyzer_class': 'org.apache.cassandra.index.sasi.analyzer.StandardAnalyzer',\n" +
                     "'tokenization_enable_stemming': 'true',\n" +
                     "'tokenization_locale': 'en',\n" +
@@ -311,9 +313,12 @@
                     "'analyzed': 'true',\n" +
                     "'tokenization_normalize_lowercase': 'true'};");
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
 
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0' ALLOW FILTERING");
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(1, rows.all().size());
+
+        rows = executeNet("SELECT id1 FROM %s WHERE v2 like '0'");
         assertEquals(1, rows.all().size());
     }
 
@@ -363,11 +368,12 @@
 
         int rowCount = 10;
         for (int i = 0; i < rowCount; i++)
-            execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0')");
+            execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', " + i + ", '0')");
 
         createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        waitForTableIndexesQueryable();
 
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
         assertEquals(rowCount, rows.all().size());
     }
 
@@ -407,12 +413,12 @@
 
         int rowCount = 10;
         for (int i = 0; i < rowCount; i++)
-            execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0')");
+            execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', " + i + ", '0')");
 
         Injections.inject(forceFlushPause);
         createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1"));
 
-        assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1='0'")).isInstanceOf(ReadFailureException.class);
+        assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1>=0")).isInstanceOf(ReadFailureException.class);
     }
 
     @Test
@@ -423,7 +429,7 @@
 
         int rowCount = 10;
         for (int i = 0; i < rowCount; i++)
-            execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0')");
+            execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', " + i + ", '0')");
         flush();
 
         Injections.inject(failSAIInitialializaion);
@@ -431,7 +437,7 @@
         waitForAssert(() -> assertEquals(1, indexBuildCounter.get()));
         waitForCompactions();
 
-        assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1='0'")).isInstanceOf(ReadFailureException.class);
+        assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1>=0")).isInstanceOf(ReadFailureException.class);
     }
 
     @Test
@@ -439,28 +445,50 @@
     {
         createTable(CREATE_TABLE_TEMPLATE);
 
-        String literalIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        String numericIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        String literalIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+        IndexContext numericIndexContext = createIndexContext(numericIndexName, Int32Type.instance);
         IndexContext literalIndexContext = createIndexContext(literalIndexName, UTF8Type.instance);
-        verifyIndexFiles(literalIndexContext, 0);
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 0, 0);
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0')");
         flush();
-        verifyIndexFiles(literalIndexContext, 1);
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 1);
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(1, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(1, rows.all().size());
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0')");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0')");
         flush();
-        verifyIndexFiles(literalIndexContext, 2);
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 2);
+        verifySSTableIndexes(numericIndexName, 2, 2);
         verifySSTableIndexes(literalIndexName, 2, 2);
-        rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(2, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(2, rows.all().size());
 
+        dropIndex("DROP INDEX %s." + numericIndexName);
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 0, 2);
+        verifySSTableIndexes(numericIndexName, 2, 0);
+        verifySSTableIndexes(literalIndexName, 2, 2);
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
+        assertEquals(2, rows.all().size());
+
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('2', 2, '0')");
+        flush();
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 0, 3);
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
+        assertEquals(3, rows.all().size());
+
         dropIndex("DROP INDEX %s." + literalIndexName);
-        verifyIndexFiles(literalIndexContext, 0);
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 0, 0);
+        verifySSTableIndexes(numericIndexName, 0);
         verifySSTableIndexes(literalIndexName, 0);
 
-        assertZeroSegmentBuilderUsage();
+        assertEquals("Segment memory limiter should revert to zero on drop.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
     }
 
     @Test
@@ -469,20 +497,25 @@
         createTable(CREATE_TABLE_TEMPLATE);
         verifyNoIndexFiles();
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
         flush();
         verifyNoIndexFiles();
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
         flush();
         verifyNoIndexFiles();
 
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
-        verifyIndexFiles(literalIndexContext, 2);
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
+        waitForTableIndexesQueryable();
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 2);
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(2, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(2, rows.all().size());
 
-        assertZeroSegmentBuilderUsage();
+        assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
     }
 
     @Test
@@ -491,20 +524,30 @@
         createTable(CREATE_TABLE_TEMPLATE);
         verifyNoIndexFiles();
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
         flush();
         verifyNoIndexFiles();
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
         flush();
         verifyNoIndexFiles();
 
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
-        verifyIndexFiles(literalIndexContext, 2);
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        waitForTableIndexesQueryable();
+        verifyIndexFiles(numericIndexContext, null, 2, 0);
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
         assertEquals(2, rows.all().size());
 
-        assertZeroSegmentBuilderUsage();
+        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
+        waitForTableIndexesQueryable();
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 2);
+        rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(2, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
+        assertEquals(2, rows.all().size());
+
+        assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
     }
 
     @Test
@@ -513,28 +556,36 @@
         createTable(CREATE_TABLE_TEMPLATE);
         disableCompaction(KEYSPACE);
 
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
         verifyNoIndexFiles();
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
         flush();
-        verifyIndexFiles(literalIndexContext, 1);
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 1);
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(1, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(1, rows.all().size());
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
         flush();
-        verifyIndexFiles(literalIndexContext, 2);
-        rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 2);
+        rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(2, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(2, rows.all().size());
 
         compact();
-        waitForAssert(() -> verifyIndexFiles(literalIndexContext, 1));
+        waitForAssert(() -> verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 1));
 
-        rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(2, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(2, rows.all().size());
 
-        assertZeroSegmentBuilderUsage();
+        assertEquals("Segment memory limiter should revert to zero after compaction.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
     }
 
     @Test
@@ -556,6 +607,7 @@
         if (!concurrentTruncate)
         {
             createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+            createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
         }
 
         // create 100 rows, half in sstable and half in memtable
@@ -564,12 +616,13 @@
         {
             if (i == num / 2)
                 flush();
-            execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0');");
+            execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', 0, '0');");
         }
 
         if (concurrentTruncate)
         {
-            createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+            createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+            createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
             truncate(true);
             waitForTableIndexesQueryable();
         }
@@ -582,8 +635,10 @@
 
         // verify index-view-manager has been cleaned up
         verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V1_COLUMN_IDENTIFIER), 0);
+        verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V2_COLUMN_IDENTIFIER), 0);
 
-        assertZeroSegmentBuilderUsage();
+        assertEquals("Segment memory limiter should revert to zero after truncate.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
     }
 
     @Test
@@ -591,41 +646,51 @@
     {
         // prepare schema and data
         createTable(CREATE_TABLE_TEMPLATE);
-        String literalIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        String numericIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        String stringIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
         flush();
 
         for (CorruptionType corruptionType : CorruptionType.values())
         {
-            verifyRebuildCorruptedFiles(literalIndexName, corruptionType, false);
-            verifyRebuildCorruptedFiles(literalIndexName, corruptionType, true);
+            verifyRebuildCorruptedFiles(numericIndexName, stringIndexName, corruptionType, false);
+            verifyRebuildCorruptedFiles(numericIndexName, stringIndexName, corruptionType, true);
         }
 
-        assertZeroSegmentBuilderUsage();
+        assertEquals("Segment memory limiter should revert to zero following rebuild.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
     }
 
-    private void verifyRebuildCorruptedFiles(String literalIndexName,
+    private void verifyRebuildCorruptedFiles(String numericIndexName,
+                                             String stringIndexName,
                                              CorruptionType corruptionType,
                                              boolean rebuild) throws Throwable
     {
-        IndexContext LiteralIndexContext = createIndexContext(literalIndexName, UTF8Type.instance);
+        IndexContext numericIndexContext = createIndexContext(numericIndexName, Int32Type.instance);
+        IndexContext stringIndexContext = createIndexContext(stringIndexName, UTF8Type.instance);
 
         for (IndexComponent component : Version.LATEST.onDiskFormat().perSSTableIndexComponents())
-            verifyRebuildIndexComponent(LiteralIndexContext, component, null, corruptionType, rebuild);
+            verifyRebuildIndexComponent(numericIndexContext, stringIndexContext, component, null, corruptionType, true, true, rebuild);
 
-        for (IndexComponent component : Version.LATEST.onDiskFormat().perColumnIndexComponents(LiteralIndexContext))
-            verifyRebuildIndexComponent(LiteralIndexContext, component, LiteralIndexContext, corruptionType, rebuild);
+        for (IndexComponent component : Version.LATEST.onDiskFormat().perColumnIndexComponents(numericIndexContext))
+            verifyRebuildIndexComponent(numericIndexContext, stringIndexContext, component, numericIndexContext, corruptionType, false, true, rebuild);
+
+        for (IndexComponent component : Version.LATEST.onDiskFormat().perColumnIndexComponents(stringIndexContext))
+            verifyRebuildIndexComponent(numericIndexContext, stringIndexContext, component, stringIndexContext, corruptionType, true, false, rebuild);
     }
 
-    private void verifyRebuildIndexComponent(IndexContext literalIndexContext,
+    private void verifyRebuildIndexComponent(IndexContext numericIndexContext,
+                                             IndexContext stringIndexContext,
                                              IndexComponent component,
                                              IndexContext corruptionContext,
                                              CorruptionType corruptionType,
+                                             boolean failedStringIndex,
+                                             boolean failedNumericIndex,
                                              boolean rebuild) throws Throwable
     {
-        // The completion markers are valid if they exist on the file system, so we only need to test
+        // The completion markers are valid if they exist on the file system so we only need to test
         // their removal. If we are testing with encryption then we don't want to test any components
         // that are encryptable unless they have been removed because encrypted components aren't
         // checksum validated.
@@ -641,11 +706,15 @@
         int rowCount = 2;
 
         // initial verification
-        verifySSTableIndexes(literalIndexContext.getIndexName(), 1);
-        verifyIndexFiles(literalIndexContext, 1, 1, 1);
-        assertTrue(verifyChecksum(literalIndexContext));
+        verifySSTableIndexes(numericIndexContext.getIndexName(), 1);
+        verifySSTableIndexes(stringIndexContext.getIndexName(), 1);
+        verifyIndexFiles(numericIndexContext, stringIndexContext, 1, 1, 1, 1, 1);
+        assertTrue(verifyChecksum(numericIndexContext));
+        assertTrue(verifyChecksum(numericIndexContext));
 
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(rowCount, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(rowCount, rows.all().size());
 
         // corrupt file
@@ -656,19 +725,17 @@
 
         // If we are removing completion markers then the rest of the components should still have
         // valid checksums.
-        boolean expectedLiteralState = isBuildCompletionMarker(component);
+        boolean expectedNumericState = !failedNumericIndex || isBuildCompletionMarker(component);
+        boolean expectedLiteralState = !failedStringIndex || isBuildCompletionMarker(component);
 
-        assertEquals("Validation for " + component + " should be " + expectedLiteralState + " but was " + !expectedLiteralState,
-                     expectedLiteralState,
-                     validateComponents(literalIndexContext));
-
-        assertEquals("Checksum verification for " + component + " should be " + expectedLiteralState + " but was " + !expectedLiteralState,
-                     expectedLiteralState,
-                     verifyChecksum(literalIndexContext));
+        assertEquals("Checksum verification for " + component + " should be " + expectedNumericState + " but was " + !expectedNumericState,
+                     expectedNumericState,
+                     verifyChecksum(numericIndexContext));
+        assertEquals(expectedLiteralState, verifyChecksum(stringIndexContext));
 
         if (rebuild)
         {
-            rebuildIndexes(literalIndexContext.getIndexName());
+            rebuildIndexes(numericIndexContext.getIndexName(), stringIndexContext.getIndexName());
         }
         else
         {
@@ -676,13 +743,25 @@
             reloadSSTableIndex();
 
             // Verify the index cannot be read:
-            verifySSTableIndexes(literalIndexContext.getIndexName(), Version.LATEST.onDiskFormat().perSSTableIndexComponents().contains(component) ? 0 : 1, 0);
+            verifySSTableIndexes(numericIndexContext.getIndexName(), Version.LATEST.onDiskFormat().perSSTableIndexComponents().contains(component) ? 0 : 1, failedNumericIndex ? 0 : 1);
+            verifySSTableIndexes(stringIndexContext.getIndexName(), Version.LATEST.onDiskFormat().perSSTableIndexComponents().contains(component) ? 0 : 1, failedStringIndex ? 0 : 1);
 
             try
             {
                 // If the corruption is that a file is missing entirely, the index won't be marked non-queryable...
-                rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
-                assertEquals(0, rows.all().size());
+                rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+                assertEquals(failedNumericIndex ? 0 : rowCount, rows.all().size());
+            }
+            catch (ReadFailureException e)
+            {
+                // ...but most kind of corruption will result in the index being non-queryable.
+            }
+
+            try
+            {
+                // If the corruption is that a file is missing entirely, the index won't be marked non-queryable...
+                rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
+                assertEquals(failedStringIndex ? 0 : rowCount, rows.all().size());
             }
             catch (ReadFailureException e)
             {
@@ -694,10 +773,13 @@
         }
 
         // verify indexes are recovered
-        verifySSTableIndexes(literalIndexContext.getIndexName(), 1);
-        verifyIndexFiles(literalIndexContext, 1, 1, 1);
+        verifySSTableIndexes(numericIndexContext.getIndexName(), 1);
+        verifySSTableIndexes(numericIndexContext.getIndexName(), 1);
+        verifyIndexFiles(numericIndexContext, stringIndexContext, 1, 1, 1, 1, 1);
 
-        rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(rowCount, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(rowCount, rows.all().size());
     }
 
@@ -707,9 +789,9 @@
         createTable(CREATE_TABLE_TEMPLATE);
         disableCompaction(KEYSPACE);
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
         flush();
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
         flush();
 
         // Inject failure
@@ -719,17 +801,16 @@
         try
         {
             // Create a new index, which will actuate a build compaction and fail, but leave the node running...
-            IndexContext literalIndexContext = createIndexContext(createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+            IndexContext numericIndexContext = createIndexContext(createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
             // two index builders running in different compaction threads because of parallelised index initial build
             waitForAssert(() -> assertEquals(2, indexBuildCounter.get()));
             waitForCompactionsFinished();
 
-            // Only token/primary key files for the first SSTable in the compaction task should exist, while column-specific files are blown away:
-            verifyIndexFiles(literalIndexContext, 2, 0, 0);
+            // Only token/offset files for the first SSTable in the compaction task should exist, while column-specific files are blown away:
+            verifyIndexFiles(numericIndexContext, null, 2, 0, 0, 0, 0);
 
-            Assertions.assertThat(getNotQueryableIndexes()).isNotEmpty();
-
-            assertZeroSegmentBuilderUsage();
+            assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+            assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
         }
         finally
         {
@@ -738,14 +819,14 @@
     }
 
     @Test
-    public void verifyCleanupFailedTokenPrimaryKeyFiles() throws Throwable
+    public void verifyCleanupFailedPrimaryKeyFiles() throws Throwable
     {
         createTable(CREATE_TABLE_TEMPLATE);
         disableCompaction(KEYSPACE);
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
         flush();
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
         flush();
 
         // Inject failure
@@ -763,9 +844,8 @@
             // SSTable-level token/offset file(s) should be removed, while column-specific files never existed:
             verifyNoIndexFiles();
 
-            Assertions.assertThat(getNotQueryableIndexes()).isNotEmpty();
-
-            assertZeroSegmentBuilderUsage();
+            assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+            assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
         }
         finally
         {
@@ -779,7 +859,8 @@
         createTable(CREATE_TABLE_TEMPLATE);
         disableCompaction(KEYSPACE);
 
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
 
         // flush empty index
         execute("INSERT INTO %s (id1) VALUES ('0');");
@@ -788,19 +869,24 @@
         execute("INSERT INTO %s (id1) VALUES ('1');");
         flush();
 
-        verifyIndexFiles(literalIndexContext, 2, 0, 2);
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 0, 0, 2, 2);
 
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(0, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(0, rows.all().size());
 
         // compact empty index
         compact();
-        waitForAssert(() -> verifyIndexFiles(literalIndexContext, 1, 0, 1));
+        waitForAssert(() -> verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 0, 0, 1, 1));
 
-        rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(0, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(0, rows.all().size());
 
-        assertZeroSegmentBuilderUsage();
+        assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
     }
 
     @Test
@@ -854,24 +940,33 @@
         disableCompaction(KEYSPACE);
 
 
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
+        waitForTableIndexesQueryable();
 
         populateData.run();
         verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V1_COLUMN_IDENTIFIER), 2, 0);
-        verifyIndexFiles(literalIndexContext, 2, 0, 2);
+        verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V2_COLUMN_IDENTIFIER), 2, 0);
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 0, 0, 2, 2);
 
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(0, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(0, rows.all().size());
 
         // compact empty index
         compact();
         verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V1_COLUMN_IDENTIFIER), 1, 0);
-        waitForAssert(() -> verifyIndexFiles(literalIndexContext, 1, 0, 1));
+        verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V2_COLUMN_IDENTIFIER), 1, 0);
+        waitForAssert(() -> verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 0, 0, 1, 1));
 
-        rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(0, rows.all().size());
+        rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
         assertEquals(0, rows.all().size());
 
-        assertZeroSegmentBuilderUsage();
+        assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
     }
 
     @Test
@@ -883,7 +978,7 @@
         int num = 100;
         for (int i = 0; i < num; i++)
         {
-            execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+            execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
         }
         flush();
 
@@ -892,7 +987,7 @@
                                                                    .build();
 
         Injections.inject(delayIndexBuilderCompletion);
-        String indexName = createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        String indexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
         waitForAssert(() -> assertEquals(1, delayIndexBuilderCompletion.getCount()));
 
         dropIndex("DROP INDEX %s." + indexName);
@@ -908,9 +1003,10 @@
 
         // create index again, it should succeed
         indexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        waitForTableIndexesQueryable();
         verifySSTableIndexes(indexName, 1);
 
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
         assertEquals(num, rows.all().size());
     }
 
@@ -926,7 +1022,7 @@
         int sstable = 1;
         for (int i = 0; i < num; i++)
         {
-            execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0');");
+            execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', 0, '0');");
         }
         flush();
 
@@ -936,7 +1032,7 @@
 
         Injections.inject(delayIndexBuilderCompletion);
 
-        IndexContext literalIndexContext = createIndexContext(createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+        IndexContext numericIndexContext = createIndexContext(createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
 
         waitForAssert(() -> assertTrue(getCompactionTasks() > 0), 1000, TimeUnit.MILLISECONDS);
 
@@ -944,6 +1040,8 @@
         int attempt = 20;
         while (getCompactionTasks() > 0 && attempt > 0)
         {
+            System.out.println("Attempt " + attempt + " at stopping the compaction tasks");
+
             // only interrupts active compactions, not pending compactions.
             CompactionManager.instance.stopCompaction(OperationType.INDEX_BUILD.name());
             // let blocked builder to continue, but still block pending builder threads
@@ -958,7 +1056,7 @@
         delayIndexBuilderCompletion.disable();
 
         // initial index builder should have stopped abruptly resulting in the index not being queryable
-        verifyInitialIndexFailed(literalIndexContext.getIndexName());
+        verifyInitialIndexFailed(numericIndexContext.getIndexName());
         Assertions.assertThat(getNotQueryableIndexes()).isNotEmpty();
 
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
@@ -971,18 +1069,20 @@
             assertTrue("Expect index build stopped", view.getIndexes().isEmpty());
         }
 
-        assertZeroSegmentBuilderUsage();
+        assertEquals("Segment memory limiter should revert to zero on interrupted compactions.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
 
         // rebuild index
-        ColumnFamilyStore.rebuildSecondaryIndex(KEYSPACE, currentTable(), literalIndexContext.getIndexName());
+        ColumnFamilyStore.rebuildSecondaryIndex(KEYSPACE, currentTable(), numericIndexContext.getIndexName());
 
-        verifyIndexFiles(literalIndexContext, sstable);
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+        verifyIndexFiles(numericIndexContext, null, sstable, 0);
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
         assertEquals(num, rows.all().size());
 
-        assertZeroSegmentBuilderUsage();
+        assertEquals("Segment memory limiter should revert to zero following rebuild.", 0L, getSegmentBufferUsedBytes());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
 
-        assertTrue(verifyChecksum(literalIndexContext));
+        assertTrue(verifyChecksum(numericIndexContext));
     }
 
     @Test
@@ -992,7 +1092,7 @@
 
         String index = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
 
-        assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %%s WHERE expr(%s, '0')", index)))
+        assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %%s WHERE expr(%s, 0)", index)))
         .isInstanceOf(InvalidQueryException.class)
         .hasMessage(String.format(IndexRestrictions.CUSTOM_EXPRESSION_NOT_SUPPORTED, index));
     }
@@ -1068,7 +1168,7 @@
         assertEquals(Arrays.asList(2L, 1L), toSize.apply(iterator.next()));
     }
 
-    private void assertZeroSegmentBuilderUsage() throws Exception
+    private void assertZeroSegmentBuilderUsage()
     {
         assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
         assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryCellDeletionsTest.java b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryCellDeletionsTest.java
new file mode 100644
index 0000000..39856bd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryCellDeletionsTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.index.sai.cql;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Force generates segments due to a small RAM size on compaction, to test segment splitting
+ */
+public class TinySegmentQueryCellDeletionsTest extends AbstractQueryTester
+{
+    @Before
+    public void setSegmentWriteBufferSpace() throws Throwable
+    {
+        setSegmentWriteBufferSpace(0);
+    }
+
+    @Test
+    public void testCellDeletions() throws Throwable
+    {
+        IndexQuerySupport.cellDeletions(executor, dataModel, sets);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryRowDeletionsTest.java b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryRowDeletionsTest.java
new file mode 100644
index 0000000..5a67230
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryRowDeletionsTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.index.sai.cql;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Force generates segments due to a small RAM size on compaction, to test segment splitting
+ */
+public class TinySegmentQueryRowDeletionsTest extends AbstractQueryTester
+{
+    @Before
+    public void setSegmentWriteBufferSpace() throws Throwable
+    {
+        setSegmentWriteBufferSpace(0);
+    }
+
+    @Test
+    public void testRowDeletions() throws Throwable
+    {
+        IndexQuerySupport.rowDeletions(executor, dataModel, sets);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryTimeToLiveTest.java b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryTimeToLiveTest.java
new file mode 100644
index 0000000..b89a8a8
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryTimeToLiveTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.index.sai.cql;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Force generates segments due to a small RAM size on compaction, to test segment splitting
+ */
+public class TinySegmentQueryTimeToLiveTest extends AbstractQueryTester
+{
+    @Before
+    public void setSegmentWriteBufferSpace() throws Throwable
+    {
+        setSegmentWriteBufferSpace(0);
+    }
+
+    @Test
+    public void testTimeToLive() throws Throwable
+    {
+        IndexQuerySupport.timeToLive(executor, dataModel, sets);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryWriteLifecycleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryWriteLifecycleTest.java
new file mode 100644
index 0000000..b32e1db
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryWriteLifecycleTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.index.sai.cql;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Force generates segments due to a small RAM size on compaction, to test segment splitting
+ */
+public class TinySegmentQueryWriteLifecycleTest extends AbstractQueryTester
+{
+    @Before
+    public void setSegmentWriteBufferSpace() throws Throwable
+    {
+        setSegmentWriteBufferSpace(0);
+    }
+
+    @Test
+    public void testWriteLifecycle() throws Throwable
+    {
+        IndexQuerySupport.writeLifecycle(executor, dataModel, sets);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/AsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/AsciiTest.java
index fdc60ed..485fd1f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/AsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/AsciiTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new DataSet.AsciiDataSet());
+        return generateParameters(new DataSet.AsciiDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/BigintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/BigintTest.java
index 2062c68..603a87e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/BigintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/BigintTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.BigintDataSet());
+        return generateParameters(new DataSet.BigintDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/BooleanTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/BooleanTest.java
index 4f2289e..472fdfd 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/BooleanTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/BooleanTest.java
@@ -27,6 +27,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new DataSet.BooleanDataSet());
+        return generateParameters(new DataSet.BooleanDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/DateTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/DateTest.java
index 14bfedd..7e97717 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/DateTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/DateTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.DateDataSet());
+        return generateParameters(new DataSet.DateDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/DecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/DecimalTest.java
index a4b7589..8dc1071 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/DecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/DecimalTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.DecimalDataSet());
+        return generateParameters(new DataSet.DecimalDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/DoubleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/DoubleTest.java
index 81ce202..4222d7f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/DoubleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/DoubleTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.DoubleDataSet());
+        return generateParameters(new DataSet.DoubleDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/FloatTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/FloatTest.java
index aa185a5..598f95c 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/FloatTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/FloatTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.FloatDataSet());
+        return generateParameters(new DataSet.FloatDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/IndexingTypeSupport.java b/test/unit/org/apache/cassandra/index/sai/cql/types/IndexingTypeSupport.java
index 15acbda..b1d5408 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/IndexingTypeSupport.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/IndexingTypeSupport.java
@@ -52,7 +52,7 @@
         POST_BUILD_QUERY
     }
 
-    protected static Collection<Object[]> generateLiteralParameters(DataSet<?> dataset)
+    protected static Collection<Object[]> generateParameters(DataSet<?> dataset)
     {
         return Arrays.asList(new Object[][]
         {
@@ -69,15 +69,6 @@
         });
     }
 
-    protected static Collection<Object[]> generateNumericParameters(DataSet<?> dataset)
-    {
-        return Arrays.asList(new Object[][]
-        {
-            { dataset, true, Scenario.MEMTABLE_QUERY },
-            { dataset, false, Scenario.MEMTABLE_QUERY },
-        });
-    }
-
     @Before
     public void createTable()
     {
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/InetTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/InetTest.java
index 19c9fe4..a3c2796 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/InetTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/InetTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.InetDataSet());
+        return generateParameters(new DataSet.InetDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/IntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/IntTest.java
index 3fbcefd..a96f2b1 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/IntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/IntTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.IntDataSet());
+        return generateParameters(new DataSet.IntDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/SmallintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/SmallintTest.java
index c960ec4..617752e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/SmallintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/SmallintTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.SmallintDataSet());
+        return generateParameters(new DataSet.SmallintDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TextTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TextTest.java
index 474b7c2..7c40d8e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TextTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TextTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new DataSet.TextDataSet());
+        return generateParameters(new DataSet.TextDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TimeTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TimeTest.java
index a716d8f..316b371 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TimeTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TimeTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.TimeDataSet());
+        return generateParameters(new DataSet.TimeDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TimestampTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TimestampTest.java
index 37293c4..1992c7a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TimestampTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TimestampTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.TimestampDataSet());
+        return generateParameters(new DataSet.TimestampDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TimeuuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TimeuuidTest.java
index d79e408..b5b7dee 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TimeuuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TimeuuidTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.TimeuuidDataSet());
+        return generateParameters(new DataSet.TimeuuidDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TinyintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TinyintTest.java
index edabf52..118d5c7 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TinyintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TinyintTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.TinyintDataSet());
+        return generateParameters(new DataSet.TinyintDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/UuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/UuidTest.java
index 4225ed0..ed11f0e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/UuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/UuidTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.UuidDataSet());
+        return generateParameters(new DataSet.UuidDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/VarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/VarintTest.java
index 6fbe977..5263380 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/VarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/VarintTest.java
@@ -26,6 +26,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new DataSet.VarintDataSet());
+        return generateParameters(new DataSet.VarintDataSet());
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListAsciiTest.java
index 58ee912..03d4d22 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListDecimalTest.java
index eda791e..c239267 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListDecimalTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.DecimalDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.DecimalDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListIntTest.java
index 4161cc9..8d3ab67 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.IntDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.IntDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListVarintTest.java
index 1864b96..98ac32e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.VarintDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.VarintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListAsciiTest.java
index d520751..22c4c9f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.ListDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListBigintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListBigintTest.java
index bafe565..44c856f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListBigintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListBigintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.BigintDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.BigintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDateTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDateTest.java
index 97452db..a79841c 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDateTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDateTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DateDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DateDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDecimalTest.java
index a718539..c52f17d 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDecimalTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DecimalDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DecimalDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDoubleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDoubleTest.java
index 0f9c0a4..64e1363 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDoubleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDoubleTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DoubleDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DoubleDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFloatTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFloatTest.java
index 533a238..8925e24 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFloatTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFloatTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.FloatDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.FloatDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFrozenCollectionTest.java
index 7ef4155..c670acf 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFrozenCollectionTest.java
@@ -32,6 +32,6 @@
     public static Collection<Object[]> generateParameters()
     {
         DataSet<List<Integer>> frozen = new CollectionDataSet.FrozenListDataSet<>(new DataSet.IntDataSet());
-        return generateLiteralParameters(new CollectionDataSet.ListDataSet<>(frozen));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(frozen));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListInetTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListInetTest.java
index 6033ea4..0138975 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListInetTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListInetTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.InetDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.InetDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListIntTest.java
index de3fed3..0f581aa 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.IntDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.IntDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListSmallintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListSmallintTest.java
index 55182ce..3f6cb8b 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListSmallintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListSmallintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.SmallintDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.SmallintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTextTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTextTest.java
index 7f38d27..098a57f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTextTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTextTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TextDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TextDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeTest.java
index bdb0339..d443bf1 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimeDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimeDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimestampTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimestampTest.java
index b1aa9a2..5797f53 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimestampTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimestampTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimestampDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimestampDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeuuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeuuidTest.java
index dd7b8f1..087fc7e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeuuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeuuidTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimeuuidDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimeuuidDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTinyintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTinyintTest.java
index 1ecda14..f4ed256 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTinyintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTinyintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TinyintDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TinyintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListUuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListUuidTest.java
index c315961..49a4841 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListUuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListUuidTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.UuidDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.UuidDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListVarintTest.java
index 4b201ea..17df156 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.VarintDataSet()));
+        return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.VarintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapAsciiTest.java
index f8df465..dc3faf7 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapDecimalTest.java
index 3d110bf..d17c2c3 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapDecimalTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.DecimalDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.DecimalDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapIntTest.java
index 14ab4e2..4d1a393 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapVarintTest.java
index 6424fe0..a393d70 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.VarintDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.VarintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapAsciiTest.java
index 6d56ee5..81650cf 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.MapDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapBigintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapBigintTest.java
index 3af7fde..2e8e434 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapBigintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapBigintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.BigintDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.BigintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDateTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDateTest.java
index 13b5294..d5c2902 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDateTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDateTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DateDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DateDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDecimalTest.java
index 89f67e4..2511921 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDecimalTest.java
@@ -30,7 +30,7 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DecimalDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DecimalDataSet()));
     }
 }
 
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDoubleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDoubleTest.java
index c758b00..67056b2 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDoubleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDoubleTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DoubleDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DoubleDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesAsciiTest.java
index 3ed8435..4526b73 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesFrozenCollectionTest.java
index 2930829..2f06531 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesFrozenCollectionTest.java
@@ -32,6 +32,6 @@
     public static Collection<Object[]> generateParameters()
     {
         DataSet<Map<Integer, Integer>> frozen = new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet());
-        return generateLiteralParameters(new CollectionDataSet.MapEntriesDataSet<>(frozen));
+        return generateParameters(new CollectionDataSet.MapEntriesDataSet<>(frozen));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesIntTest.java
index 97e5d69..71bd9b2 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.IntDataSet()));
+        return generateParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.IntDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesVarintTest.java
index 121128f..852bc7d 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.VarintDataSet()));
+        return generateParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.VarintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFloatTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFloatTest.java
index 4a8b044..31e39c0 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFloatTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFloatTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.FloatDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.FloatDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFrozenCollectionTest.java
index 8787b96..b39a8d3 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFrozenCollectionTest.java
@@ -32,6 +32,6 @@
     public static Collection<Object[]> generateParameters()
     {
         DataSet<Map<Integer, Integer>> frozen = new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet());
-        return generateLiteralParameters(new CollectionDataSet.MapDataSet<>(frozen));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(frozen));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapInetTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapInetTest.java
index d455f84..62def5b 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapInetTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapInetTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.InetDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.InetDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapIntTest.java
index cd0c7c9..5a33360 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.IntDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.IntDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysAsciiTest.java
index aaaf818..5b0837b 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysFrozenCollectionTest.java
index 1ed21d5..e2c4c84 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysFrozenCollectionTest.java
@@ -32,6 +32,6 @@
     public static Collection<Object[]> generateParameters()
     {
         DataSet<Map<Integer, Integer>> frozen = new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet());
-        return generateLiteralParameters(new CollectionDataSet.MapKeysDataSet<>(frozen));
+        return generateParameters(new CollectionDataSet.MapKeysDataSet<>(frozen));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysIntTest.java
index dcd101b..6e75d15 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.IntDataSet()));
+        return generateParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.IntDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysVarintTest.java
index d6aae85..52c395a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.VarintDataSet()));
+        return generateParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.VarintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapSmallintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapSmallintTest.java
index 8ea194e..cd05883 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapSmallintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapSmallintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.SmallintDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.SmallintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTextTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTextTest.java
index e0058a7..29cf454 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTextTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTextTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TextDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TextDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeTest.java
index 82504d9..9923457 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimeDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimeDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimestampTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimestampTest.java
index 40fe6ee..1f5cd8e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimestampTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimestampTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimestampDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimestampDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeuuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeuuidTest.java
index 26e0a7e..2a46698 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeuuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeuuidTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimeuuidDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimeuuidDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTinyintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTinyintTest.java
index d8102c2..c832905 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTinyintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTinyintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TinyintDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TinyintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapUuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapUuidTest.java
index 9f77a37..0963ee0 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapUuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapUuidTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.UuidDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.UuidDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesAsciiTest.java
index 4f3068a..88c8e30 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesFrozenCollectionTest.java
index 9e82234..15e9dfb 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesFrozenCollectionTest.java
@@ -32,6 +32,6 @@
     public static Collection<Object[]> generateParameters()
     {
         DataSet<Map<Integer, Integer>> frozen = new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet());
-        return generateLiteralParameters(new CollectionDataSet.MapValuesDataSet<>(frozen));
+        return generateParameters(new CollectionDataSet.MapValuesDataSet<>(frozen));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesIntTest.java
index 5db3f0d..3bf3840 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.IntDataSet()));
+        return generateParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.IntDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesVarintTest.java
index c49baf5..e5077fb 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapVarintTest.java
index a4fc1e0..d85880a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.VarintDataSet()));
+        return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.VarintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapAsciiTest.java
index e1eccd4..8c5f9f3 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapIntTest.java
index 7d9af16..270f741 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.IntDataSet()));
+        return generateParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.IntDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapVarintTest.java
index 97d0144..9d02cf2 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.VarintDataSet()));
+        return generateParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.VarintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetAsciiTest.java
index 02b8f26..1672d22 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetDecimalTest.java
index aec4946..591bce5 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetDecimalTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.DecimalDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.DecimalDataSet()));
     }
 }
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetIntTest.java
index 4db6b9f..6f43a4f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetVarintTest.java
index 2db3165..9d9d759 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.VarintDataSet()));
+        return generateParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.VarintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetAsciiTest.java
index 93a43c1..e58ad4f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetAsciiTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.SetDataSet<>(new DataSet.AsciiDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetBigintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetBigintTest.java
index a37e907..bdf6fe0 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetBigintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetBigintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.BigintDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.BigintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDateTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDateTest.java
index 9d11fe3..460e70d 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDateTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDateTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DateDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DateDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDecimalTest.java
index 95bccbf..ef495d9 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDecimalTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DecimalDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DecimalDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDoubleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDoubleTest.java
index 05552dd..86a4fa5 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDoubleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDoubleTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DoubleDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DoubleDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFloatTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFloatTest.java
index 82861d3..6c29210 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFloatTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFloatTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.FloatDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.FloatDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFrozenCollectionTest.java
index d632a40..da4aca7 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFrozenCollectionTest.java
@@ -32,6 +32,6 @@
     public static Collection<Object[]> generateParameters()
     {
         DataSet<Set<Integer>> frozen = new CollectionDataSet.FrozenSetDataSet<>(new DataSet.IntDataSet());
-        return generateLiteralParameters(new CollectionDataSet.SetDataSet<>(frozen));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(frozen));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetInetTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetInetTest.java
index b51394f..380ed4c 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetInetTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetInetTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.InetDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.InetDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetIntTest.java
index 900fa71..add9799 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetIntTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.IntDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.IntDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetSmallintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetSmallintTest.java
index a9020c1..fa14363 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetSmallintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetSmallintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.SmallintDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.SmallintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTextTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTextTest.java
index 45dd37b..36e2314 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTextTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTextTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TextDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TextDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeTest.java
index d21b539..21272d1 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimeDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimeDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimestampTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimestampTest.java
index d97cbd9..e18ce5a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimestampTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimestampTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimestampDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimestampDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeuuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeuuidTest.java
index b0edd06..65d5556 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeuuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeuuidTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimeuuidDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimeuuidDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTinyintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTinyintTest.java
index f044717..1a701bf 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTinyintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTinyintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TinyintDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TinyintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetUuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetUuidTest.java
index 52a9790..b4e5541 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetUuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetUuidTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.UuidDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.UuidDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetVarintTest.java
index 07e5a6f..777ebbf 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetVarintTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.VarintDataSet()));
+        return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.VarintDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleCollectionTest.java
index d82e3bd..427ae5d 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleCollectionTest.java
@@ -31,7 +31,7 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new FrozenTupleDataSet(
+        return generateParameters(new FrozenTupleDataSet(
         new CollectionDataSet.ListDataSet<>(new DataSet.AsciiDataSet()),
         new CollectionDataSet.SetDataSet<>(new DataSet.InetDataSet()),
         new CollectionDataSet.MapDataSet<>(new DataSet.BigintDataSet())
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTest.java
index ffae1fd..7733c3c 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new FrozenTupleDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
+        return generateParameters(new FrozenTupleDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTupleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTupleTest.java
index a66bb4c..98db788 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTupleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTupleTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new FrozenTupleDataSet(new FrozenTupleDataSet(new DataSet.AsciiDataSet(), new DataSet.UuidDataSet()), new DataSet.AsciiDataSet()));
+        return generateParameters(new FrozenTupleDataSet(new FrozenTupleDataSet(new DataSet.AsciiDataSet(), new DataSet.UuidDataSet()), new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTCollectionTest.java
index 80ba060..ddd25b2 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTCollectionTest.java
@@ -31,7 +31,7 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new FrozenUDTDataSet(
+        return generateParameters(new FrozenUDTDataSet(
         new CollectionDataSet.ListDataSet<>(new DataSet.AsciiDataSet()),
         new CollectionDataSet.MapDataSet<>(new DataSet.BigintDataSet())));
     }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTTest.java
index 6271f00..2510e0b 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new FrozenUDTDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
+        return generateParameters(new FrozenUDTDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/TupleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/TupleTest.java
index e76f018..d692d8a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/TupleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/TupleTest.java
@@ -30,6 +30,6 @@
     @Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
     public static Collection<Object[]> generateParameters()
     {
-        return generateLiteralParameters(new TupleDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
+        return generateParameters(new TupleDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/SelectiveIntersectionTest.java b/test/unit/org/apache/cassandra/index/sai/disk/SelectiveIntersectionTest.java
index 51c7637..6806110 100644
--- a/test/unit/org/apache/cassandra/index/sai/disk/SelectiveIntersectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/disk/SelectiveIntersectionTest.java
@@ -17,18 +17,15 @@
  */
 package org.apache.cassandra.index.sai.disk;
 
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import com.datastax.driver.core.Session;
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.index.sai.SAITester;
 import org.apache.cassandra.index.sai.disk.v1.postings.PostingsReader;
-import org.apache.cassandra.index.sai.iterators.KeyRangeIntersectionIterator;
 import org.apache.cassandra.inject.Injections;
 
 import static org.apache.cassandra.inject.InvokePointBuilder.newInvokePoint;
@@ -145,13 +142,8 @@
         Assert.assertEquals(postingsReaderOpenCounter.get(), postingsReaderCloseCounter.get());
     }
 
-    private static void setLimits(final int selectivityLimit) throws Exception
+    private static void setLimits(final int selectivityLimit)
     {
-        Field selectivity = KeyRangeIntersectionIterator.class.getDeclaredField("INTERSECTION_CLAUSE_LIMIT");
-        selectivity.setAccessible(true);
-        Field modifiersField = Field.class.getDeclaredField("modifiers");
-        modifiersField.setAccessible(true);
-        modifiersField.setInt(selectivity, selectivity.getModifiers() & ~Modifier.FINAL);
-        selectivity.set(null, selectivityLimit);
+        CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.setString(Integer.toString(selectivityLimit));
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/format/IndexDescriptorTest.java b/test/unit/org/apache/cassandra/index/sai/disk/format/IndexDescriptorTest.java
index 995a876..b6546c9 100644
--- a/test/unit/org/apache/cassandra/index/sai/disk/format/IndexDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/disk/format/IndexDescriptorTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.cassandra.index.sai.disk.format;
 
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
 import java.net.URI;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -45,7 +43,6 @@
 {
     private final TemporaryFolder temporaryFolder = new TemporaryFolder();
     private Descriptor descriptor;
-    private Version latest;
 
     @BeforeClass
     public static void initialise()
@@ -58,21 +55,17 @@
     {
         temporaryFolder.create();
         descriptor = Descriptor.fromFile(new File(temporaryFolder.newFolder().getAbsolutePath() + "/nb-1-big-Data.db"));
-        latest = Version.LATEST;
     }
 
     @After
     public void teardown() throws Throwable
     {
-        setLatestVersion(latest);
         temporaryFolder.delete();
     }
 
     @Test
     public void versionAAPerSSTableComponentIsParsedCorrectly() throws Throwable
     {
-        setLatestVersion(Version.AA);
-
         createFileOnDisk("-SAI+aa+GroupComplete.db");
 
         IndexDescriptor indexDescriptor = IndexDescriptor.create(descriptor, SAITester.EMPTY_COMPARATOR);
@@ -84,8 +77,6 @@
     @Test
     public void versionAAPerIndexComponentIsParsedCorrectly() throws Throwable
     {
-        setLatestVersion(Version.AA);
-
         createFileOnDisk("-SAI+aa+test_index+ColumnComplete.db");
 
         IndexDescriptor indexDescriptor = IndexDescriptor.create(descriptor, SAITester.EMPTY_COMPARATOR);
@@ -95,16 +86,6 @@
         assertTrue(indexDescriptor.hasComponent(IndexComponent.COLUMN_COMPLETION_MARKER, indexContext));
     }
 
-    private void setLatestVersion(Version version) throws Throwable
-    {
-        Field latest = Version.class.getDeclaredField("LATEST");
-        latest.setAccessible(true);
-        Field modifiersField = Field.class.getDeclaredField("modifiers");
-        modifiersField.setAccessible(true);
-        modifiersField.setInt(latest, latest.getModifiers() & ~Modifier.FINAL);
-        latest.set(null, version);
-    }
-
     private void createFileOnDisk(String filename) throws Throwable
     {
         Path path;
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/BalancedTreeIndexSearcherTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/BalancedTreeIndexSearcherTest.java
new file mode 100644
index 0000000..b880a77
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/BalancedTreeIndexSearcherTest.java
@@ -0,0 +1,221 @@
+/*
+ * 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.index.sai.disk.v1;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.NumberType;
+import org.apache.cassandra.db.marshal.ShortType;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.BlockBalancedTreeIndexBuilder;
+import org.apache.cassandra.index.sai.disk.v1.segment.IndexSegmentSearcher;
+import org.apache.cassandra.index.sai.iterators.KeyRangeIterator;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class BalancedTreeIndexSearcherTest extends SAIRandomizedTester
+{
+    private static final short EQ_TEST_LOWER_BOUND_INCLUSIVE = 0;
+    private static final short EQ_TEST_UPPER_BOUND_EXCLUSIVE = 3;
+
+    private static final short RANGE_TEST_LOWER_BOUND_INCLUSIVE = 0;
+    private static final short RANGE_TEST_UPPER_BOUND_EXCLUSIVE = 10;
+
+    @Test
+    public void testRangeQueriesAgainstInt32Index() throws Exception
+    {
+        doTestRangeQueriesAgainstInt32Index();
+    }
+
+    private void doTestRangeQueriesAgainstInt32Index() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildInt32Searcher(newIndexDescriptor(), 0, 10);
+        testRangeQueries(indexSearcher, Int32Type.instance, Int32Type.instance, Integer::valueOf);
+    }
+
+    @Test
+    public void testEqQueriesAgainstInt32Index() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildInt32Searcher(newIndexDescriptor(),
+                                                                                              EQ_TEST_LOWER_BOUND_INCLUSIVE, EQ_TEST_UPPER_BOUND_EXCLUSIVE);
+        testEqQueries(indexSearcher, Int32Type.instance, Int32Type.instance, Integer::valueOf);
+    }
+
+    @Test
+    public void testRangeQueriesAgainstLongIndex() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildLongSearcher(newIndexDescriptor(), 0, 10);
+        testRangeQueries(indexSearcher, LongType.instance, Int32Type.instance, Long::valueOf);
+    }
+
+    @Test
+    public void testEqQueriesAgainstLongIndex() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildLongSearcher(newIndexDescriptor(),
+                                                                                             EQ_TEST_LOWER_BOUND_INCLUSIVE, EQ_TEST_UPPER_BOUND_EXCLUSIVE);
+        testEqQueries(indexSearcher, LongType.instance, Int32Type.instance, Long::valueOf);
+    }
+
+    @Test
+    public void testRangeQueriesAgainstShortIndex() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildShortSearcher(newIndexDescriptor(), (short) 0, (short) 10);
+        testRangeQueries(indexSearcher, ShortType.instance, Int32Type.instance, Function.identity());
+    }
+
+    @Test
+    public void testEqQueriesAgainstShortIndex() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildShortSearcher(newIndexDescriptor(),
+                                                                                              EQ_TEST_LOWER_BOUND_INCLUSIVE, EQ_TEST_UPPER_BOUND_EXCLUSIVE);
+        testEqQueries(indexSearcher, ShortType.instance, Int32Type.instance, Function.identity());
+    }
+
+    @Test
+    public void testRangeQueriesAgainstDecimalIndex() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildDecimalSearcher(newIndexDescriptor(),
+                                                                                                BigDecimal.ZERO, BigDecimal.valueOf(10L));
+        testRangeQueries(indexSearcher, DecimalType.instance, DecimalType.instance, BigDecimal::valueOf,
+                         getLongsOnInterval(21L, 70L));
+    }
+
+    private List<Long> getLongsOnInterval(long lowerInclusive, long upperInclusive)
+    {
+        return LongStream.range(lowerInclusive, upperInclusive + 1L).boxed().collect(Collectors.toList());
+    }
+
+    @Test
+    public void testEqQueriesAgainstDecimalIndex() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildDecimalSearcher(newIndexDescriptor(),
+                                                                                                BigDecimal.valueOf(EQ_TEST_LOWER_BOUND_INCLUSIVE), BigDecimal.valueOf(EQ_TEST_UPPER_BOUND_EXCLUSIVE));
+        testEqQueries(indexSearcher, DecimalType.instance, DecimalType.instance, BigDecimal::valueOf);
+    }
+
+
+    @Test
+    public void testEqQueriesAgainstBigIntegerIndex() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildBigIntegerSearcher(newIndexDescriptor(),
+                                                                                                   BigInteger.valueOf(EQ_TEST_LOWER_BOUND_INCLUSIVE), BigInteger.valueOf(EQ_TEST_UPPER_BOUND_EXCLUSIVE));
+        testEqQueries(indexSearcher, IntegerType.instance, IntegerType.instance, BigInteger::valueOf);
+    }
+
+    @Test
+    public void testRangeQueriesAgainstBigIntegerIndex() throws Exception
+    {
+        IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildBigIntegerSearcher(newIndexDescriptor(),
+                                                                                                   BigInteger.ZERO, BigInteger.valueOf(10L));
+        testRangeQueries(indexSearcher, IntegerType.instance, IntegerType.instance, BigInteger::valueOf);
+    }
+
+    private <T extends Number> void testEqQueries(final IndexSegmentSearcher indexSearcher,
+                                                  final NumberType<T> rawType, final NumberType<?> encodedType,
+                                                  final Function<Short, T> rawValueProducer) throws Exception
+    {
+        try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+        {{
+            operator = IndexOperator.EQ;
+            lower = upper = new Bound(rawType.decompose(rawValueProducer.apply(EQ_TEST_LOWER_BOUND_INCLUSIVE)), encodedType, true);
+        }}, mock(QueryContext.class)))
+        {
+            assertEquals(results.getMinimum(), results.getCurrent());
+            assertTrue(results.hasNext());
+
+            assertEquals(0L, results.next().token().getLongValue());
+        }
+
+        try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+        {{
+            operator = IndexOperator.EQ;
+            lower = upper = new Bound(rawType.decompose(rawValueProducer.apply(EQ_TEST_UPPER_BOUND_EXCLUSIVE)), encodedType, true);
+        }}, mock(QueryContext.class)))
+        {
+            assertFalse(results.hasNext());
+            indexSearcher.close();
+        }
+    }
+
+    private <T extends Number> void testRangeQueries(final IndexSegmentSearcher indexSearcher,
+                                                     final NumberType<T> rawType, final NumberType<?> encodedType,
+                                                     final Function<Short, T> rawValueProducer) throws Exception
+    {
+        List<Long> expectedTokenList = getLongsOnInterval(3L, 7L);
+        testRangeQueries(indexSearcher, rawType, encodedType, rawValueProducer, expectedTokenList);
+    }
+
+
+    private <T extends Number> void testRangeQueries(final IndexSegmentSearcher indexSearcher,
+                                                     final NumberType<T> rawType, final NumberType<?> encodedType,
+                                                     final Function<Short, T> rawValueProducer, List<Long> expectedTokenList) throws Exception
+    {
+        try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+        {{
+            operator = IndexOperator.RANGE;
+
+            lower = new Bound(rawType.decompose(rawValueProducer.apply((short)2)), encodedType, false);
+            upper = new Bound(rawType.decompose(rawValueProducer.apply((short)7)), encodedType, true);
+        }}, mock(QueryContext.class)))
+        {
+            assertEquals(results.getMinimum(), results.getCurrent());
+            assertTrue(results.hasNext());
+
+            List<Long> actualTokenList = Lists.newArrayList(Iterators.transform(results, key -> key.token().getLongValue()));
+            assertEquals(expectedTokenList, actualTokenList);
+        }
+
+        try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+        {{
+            operator = IndexOperator.RANGE;
+            lower = new Bound(rawType.decompose(rawValueProducer.apply(RANGE_TEST_UPPER_BOUND_EXCLUSIVE)), encodedType, true);
+        }}, mock(QueryContext.class)))
+        {
+            assertFalse(results.hasNext());
+        }
+
+        try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+        {{
+            operator = IndexOperator.RANGE;
+            upper = new Bound(rawType.decompose(rawValueProducer.apply(RANGE_TEST_LOWER_BOUND_INCLUSIVE)), encodedType, false);
+        }}, mock(QueryContext.class)))
+        {
+            assertFalse(results.hasNext());
+            indexSearcher.close();
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsScanner.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsScanner.java
index b317f4f..2512343 100644
--- a/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsScanner.java
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsScanner.java
@@ -30,6 +30,7 @@
 import org.apache.cassandra.io.util.FileHandle;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 import org.apache.lucene.store.IndexInput;
@@ -51,11 +52,18 @@
 
     @Override
     @SuppressWarnings({"resource", "RedundantSuppression"})
-    public PostingList postings() throws IOException
+    public PostingList postings()
     {
         assert entry != null;
         final IndexInput input = IndexFileUtils.instance.openInput(postingsFile);
-        return new ScanningPostingsReader(input, new PostingsReader.BlocksSummary(input, entry.right));
+        try
+        {
+            return new ScanningPostingsReader(input, new PostingsReader.BlocksSummary(input, entry.right));
+        }
+        catch (IOException e)
+        {
+            throw Throwables.unchecked(e);
+        }
     }
 
     @Override
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIndexBuilder.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIndexBuilder.java
new file mode 100644
index 0000000..b69e515
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIndexBuilder.java
@@ -0,0 +1,334 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.LongStream;
+import java.util.stream.Stream;
+
+import org.junit.Assert;
+
+import com.carrotsearch.hppc.LongArrayList;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.ShortType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.disk.PrimaryKeyMap;
+import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.v1.segment.NumericIndexSegmentSearcher;
+import org.apache.cassandra.index.sai.disk.v1.PerColumnIndexFiles;
+import org.apache.cassandra.index.sai.disk.v1.segment.IndexSegmentSearcher;
+import org.apache.cassandra.index.sai.disk.v1.segment.SegmentMetadata;
+import org.apache.cassandra.index.sai.memory.MemtableTermsIterator;
+import org.apache.cassandra.index.sai.utils.PrimaryKey;
+import org.apache.cassandra.index.sai.utils.TermsIterator;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.utils.AbstractGuavaIterator;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public class BlockBalancedTreeIndexBuilder
+{
+    public static final PrimaryKeyMap TEST_PRIMARY_KEY_MAP = new PrimaryKeyMap()
+    {
+        private final PrimaryKey.Factory primaryKeyFactory = new PrimaryKey.Factory(null);
+
+        @Override
+        public PrimaryKey primaryKeyFromRowId(long sstableRowId)
+        {
+            return primaryKeyFactory.createTokenOnly(new Murmur3Partitioner.LongToken(sstableRowId));
+        }
+
+        @Override
+        public long rowIdFromPrimaryKey(PrimaryKey key)
+        {
+            return key.token().getLongValue();
+        }
+    };
+    public static final PrimaryKeyMap.Factory TEST_PRIMARY_KEY_MAP_FACTORY = () -> TEST_PRIMARY_KEY_MAP;
+
+
+    private static final BigDecimal ONE_TENTH = BigDecimal.valueOf(1, 1);
+
+    private final IndexDescriptor indexDescriptor;
+    private final AbstractType<?> type;
+    private final AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>> terms;
+    private final int size;
+    private final int minSegmentRowId;
+    private final int maxSegmentRowId;
+
+    public BlockBalancedTreeIndexBuilder(IndexDescriptor indexDescriptor,
+                                         AbstractType<?> type,
+                                         AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>> terms,
+                                         int size,
+                                         int minSegmentRowId,
+                                         int maxSegmentRowId)
+    {
+        this.indexDescriptor = indexDescriptor;
+        this.type = type;
+        this.terms = terms;
+        this.size = size;
+        this.minSegmentRowId = minSegmentRowId;
+        this.maxSegmentRowId = maxSegmentRowId;
+    }
+
+    NumericIndexSegmentSearcher flushAndOpen() throws IOException
+    {
+        final TermsIterator termEnum = new MemtableTermsIterator(null, null, terms);
+        final SegmentMetadata metadata;
+
+        IndexContext columnContext = SAITester.createIndexContext("test", Int32Type.instance);
+        NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+                                                           columnContext,
+                                                           TypeUtil.fixedSizeOf(type),
+                                                           maxSegmentRowId);
+        final SegmentMetadata.ComponentMetadataMap indexMetas = writer.writeCompleteSegment(BlockBalancedTreeIterator.fromTermsIterator(termEnum, type));
+        metadata = new SegmentMetadata(0,
+                                       size,
+                                       minSegmentRowId,
+                                       maxSegmentRowId,
+                                       // min/max is unused for now
+                                       SAITester.TEST_FACTORY.createTokenOnly(Murmur3Partitioner.instance.decorateKey(UTF8Type.instance.fromString("a")).getToken()),
+                                       SAITester.TEST_FACTORY.createTokenOnly(Murmur3Partitioner.instance.decorateKey(UTF8Type.instance.fromString("b")).getToken()),
+                                       UTF8Type.instance.fromString("c"),
+                                       UTF8Type.instance.fromString("d"),
+                                       indexMetas);
+
+        try (PerColumnIndexFiles indexFiles = new PerColumnIndexFiles(indexDescriptor, SAITester.createIndexContext("test", Int32Type.instance)))
+        {
+            IndexSegmentSearcher searcher = IndexSegmentSearcher.open(TEST_PRIMARY_KEY_MAP_FACTORY, indexFiles, metadata, columnContext);
+            assertThat(searcher, is(instanceOf(NumericIndexSegmentSearcher.class)));
+            return (NumericIndexSegmentSearcher) searcher;
+        }
+    }
+
+    /**
+     * Returns a k-d tree index where:
+     * 1. term values have 32b
+     * 2. term value is equal to {@code startTermInclusive} + row id;
+     * 3. tokens and offsets are equal to row id;
+     */
+    public static IndexSegmentSearcher buildInt32Searcher(IndexDescriptor indexDescriptor, int startTermInclusive, int endTermExclusive)
+    throws IOException
+    {
+        final int size = endTermExclusive - startTermInclusive;
+        Assert.assertTrue(size > 0);
+        BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+                                                                                       Int32Type.instance,
+                                                                                       singleOrd(int32Range(startTermInclusive, endTermExclusive), Int32Type.instance, startTermInclusive, size),
+                                                                                       size,
+                                                                                       startTermInclusive,
+                                                                                       endTermExclusive);
+        return indexBuilder.flushAndOpen();
+    }
+
+    public static IndexSegmentSearcher buildDecimalSearcher(IndexDescriptor indexDescriptor, BigDecimal startTermInclusive, BigDecimal endTermExclusive)
+    throws IOException
+    {
+        BigDecimal bigDifference = endTermExclusive.subtract(startTermInclusive);
+        int size = bigDifference.intValueExact() * 10;
+        Assert.assertTrue(size > 0);
+        BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+                                                                                       DecimalType.instance,
+                                                                                       singleOrd(decimalRange(startTermInclusive, endTermExclusive), DecimalType.instance, startTermInclusive.intValueExact() * 10, size),
+                                                                                       size,
+                                                                 startTermInclusive.intValueExact() * 10,
+                                                                 endTermExclusive.intValueExact() * 10);
+        return indexBuilder.flushAndOpen();
+    }
+
+    public static IndexSegmentSearcher buildBigIntegerSearcher(IndexDescriptor indexDescriptor, BigInteger startTermInclusive, BigInteger endTermExclusive)
+    throws IOException
+    {
+        BigInteger bigDifference = endTermExclusive.subtract(startTermInclusive);
+        int size = bigDifference.intValueExact();
+        Assert.assertTrue(size > 0);
+        BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+                                                                                       IntegerType.instance,
+                                                                                       singleOrd(bigIntegerRange(startTermInclusive, endTermExclusive), IntegerType.instance, startTermInclusive.intValueExact(), size),
+                                                                                       size,
+                                                                                       startTermInclusive.intValueExact(),
+                                                                                       endTermExclusive.intValueExact());
+        return indexBuilder.flushAndOpen();
+    }
+
+    /**
+     * Returns a k-d tree index where:
+     * 1. term values have 64b
+     * 2. term value is equal to {@code startTermInclusive} + row id;
+     * 3. tokens and offsets are equal to row id;
+     */
+    public static IndexSegmentSearcher buildLongSearcher(IndexDescriptor indexDescriptor, long startTermInclusive, long endTermExclusive)
+    throws IOException
+    {
+        final long size = endTermExclusive - startTermInclusive;
+        Assert.assertTrue(size > 0);
+        BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+                                                                                       LongType.instance,
+                                                                                       singleOrd(longRange(startTermInclusive, endTermExclusive), LongType.instance, Math.toIntExact(startTermInclusive), Math.toIntExact(size)),
+                                                                                       Math.toIntExact(size),
+                                                                                       Math.toIntExact(startTermInclusive),
+                                                                                       Math.toIntExact(endTermExclusive));
+        return indexBuilder.flushAndOpen();
+    }
+
+    /**
+     * Returns a k-d tree index where:
+     * 1. term values have 16b
+     * 2. term value is equal to {@code startTermInclusive} + row id;
+     * 3. tokens and offsets are equal to row id;
+     */
+    public static IndexSegmentSearcher buildShortSearcher(IndexDescriptor indexDescriptor, short startTermInclusive, short endTermExclusive)
+    throws IOException
+    {
+        final int size = endTermExclusive - startTermInclusive;
+        Assert.assertTrue(size > 0);
+        BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+                                                                                       ShortType.instance,
+                                                                                       singleOrd(shortRange(startTermInclusive, endTermExclusive), ShortType.instance, startTermInclusive, size),
+                                                                                       size,
+                                                                                       startTermInclusive,
+                                                                                       endTermExclusive);
+        return indexBuilder.flushAndOpen();
+    }
+
+    /**
+     * Returns inverted index where each posting list contains exactly one element equal to the terms ordinal number +
+     * given offset.
+     */
+    public static AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>> singleOrd(Iterator<ByteBuffer> terms, AbstractType<?> type, int segmentRowIdOffset, int size)
+    {
+        return new AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>>()
+        {
+            private long currentTerm = 0;
+            private int currentSegmentRowId = segmentRowIdOffset;
+
+            @Override
+            protected Pair<ByteComparable, LongArrayList> computeNext()
+            {
+                if (currentTerm++ >= size)
+                {
+                    return endOfData();
+                }
+
+                LongArrayList postings = new LongArrayList();
+                postings.add(currentSegmentRowId++);
+                assertTrue(terms.hasNext());
+
+                final ByteSource encoded = TypeUtil.asComparableBytes(terms.next(), type, ByteComparable.Version.OSS50);
+                return Pair.create(v -> encoded, postings);
+            }
+        };
+    }
+
+    /**
+     * Returns sequential ordered encoded ints from {@code startInclusive} (inclusive) to {@code endExclusive}
+     * (exclusive) by an incremental step of {@code 1}.
+     */
+    public static Iterator<ByteBuffer> int32Range(int startInclusive, int endExclusive)
+    {
+        return IntStream.range(startInclusive, endExclusive)
+                        .mapToObj(Int32Type.instance::decompose)
+                        .collect(Collectors.toList())
+                        .iterator();
+    }
+
+    /**
+     * Returns sequential ordered encoded longs from {@code startInclusive} (inclusive) to {@code endExclusive}
+     * (exclusive) by an incremental step of {@code 1}.
+     */
+    public static Iterator<ByteBuffer> longRange(long startInclusive, long endExclusive)
+    {
+        return LongStream.range(startInclusive, endExclusive)
+                         .mapToObj(LongType.instance::decompose)
+                         .collect(Collectors.toList())
+                         .iterator();
+    }
+
+    public static Iterator<ByteBuffer> decimalRange(final BigDecimal startInclusive, final BigDecimal endExclusive)
+    {
+        int n = endExclusive.subtract(startInclusive).intValueExact() * 10;
+        final Supplier<BigDecimal> generator = new Supplier<BigDecimal>() {
+            BigDecimal current = startInclusive;
+
+            @Override
+            public BigDecimal get() {
+                BigDecimal result = current;
+                current = current.add(ONE_TENTH);
+                return result;
+            }
+        };
+        return Stream.generate(generator)
+                     .limit(n)
+                     .map(bd -> TypeUtil.asIndexBytes(DecimalType.instance.decompose(bd), DecimalType.instance))
+                     .collect(Collectors.toList())
+                     .iterator();
+    }
+
+    public static Iterator<ByteBuffer> bigIntegerRange(final BigInteger startInclusive, final BigInteger endExclusive)
+    {
+        int n = endExclusive.subtract(startInclusive).intValueExact();
+        final Supplier<BigInteger> generator = new Supplier<BigInteger>() {
+            BigInteger current = startInclusive;
+
+            @Override
+            public BigInteger get() {
+                BigInteger result = current;
+                current = current.add(BigInteger.ONE);
+                return result;
+            }
+        };
+        return Stream.generate(generator)
+                     .limit(n)
+                     .map(bd -> TypeUtil.asIndexBytes(IntegerType.instance.decompose(bd), IntegerType.instance))
+                     .collect(Collectors.toList())
+                     .iterator();
+    }
+
+
+    /**
+     * Returns sequential ordered encoded shorts from {@code startInclusive} (inclusive) to {@code endExclusive}
+     * (exclusive) by an incremental step of {@code 1}.
+     */
+    public static Iterator<ByteBuffer> shortRange(short startInclusive, short endExclusive)
+    {
+        return IntStream.range(startInclusive, endExclusive)
+                        .mapToObj(i -> ShortType.instance.decompose((short) i))
+                        .collect(Collectors.toList())
+                        .iterator();
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriterTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriterTest.java
new file mode 100644
index 0000000..6f1daed
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriterTest.java
@@ -0,0 +1,189 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.agrona.collections.IntArrayList;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.disk.ArrayPostingList;
+import org.apache.cassandra.index.sai.disk.format.IndexComponent;
+import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.io.IndexOutputWriter;
+import org.apache.cassandra.index.sai.disk.v1.postings.PostingsReader;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class BlockBalancedTreePostingsWriterTest extends SAIRandomizedTester
+{
+    private IndexDescriptor indexDescriptor;
+    private IndexContext indexContext;
+
+    @Before
+    public void setup() throws Throwable
+    {
+        indexDescriptor = newIndexDescriptor();
+        indexContext = SAITester.createIndexContext(newIndex(), Int32Type.instance);
+    }
+
+    @Test
+    public void shouldWritePostingsForEligibleNodes() throws Exception
+    {
+        List<PackedLongValues> leaves =
+        Arrays.asList(postings(1, 5, 7), postings(3, 4, 6), postings(2, 8, 10), postings(11, 12, 13));
+
+        setBDKPostingsWriterSizing(1, 2);
+        BlockBalancedTreePostingsWriter writer = new BlockBalancedTreePostingsWriter();
+
+        // should build postings for nodes 2 & 3 (lvl 2) and 8, 10, 12, 14 (lvl 4)
+        writer.onLeaf(64, 1, pathToRoot(1, 2, 4, 8, 16));
+        writer.onLeaf(80, 2, pathToRoot(1, 2, 5, 10, 20));
+        writer.onLeaf(96, 3, pathToRoot(1, 3, 6, 12, 24));
+        writer.onLeaf(112, 4, pathToRoot(1, 3, 7, 14, 28));
+
+        long fp;
+        try (IndexOutputWriter output = indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext))
+        {
+            fp = writer.finish(output, leaves, indexContext);
+        }
+
+        BlockBalancedTreePostingsIndex postingsIndex = new BlockBalancedTreePostingsIndex(indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext), fp);
+        assertEquals(10, postingsIndex.size());
+
+        // Internal postings...
+        assertTrue(postingsIndex.exists(2));
+        assertTrue(postingsIndex.exists(3));
+        assertTrue(postingsIndex.exists(8));
+        assertTrue(postingsIndex.exists(10));
+        assertTrue(postingsIndex.exists(12));
+        assertTrue(postingsIndex.exists(14));
+
+        assertPostingReaderEquals(postingsIndex, 2, 1, 3, 4, 5, 6, 7);
+        assertPostingReaderEquals(postingsIndex, 3, 2, 8, 10, 11, 12, 13);
+        assertPostingReaderEquals(postingsIndex, 8, 1, 5, 7);
+        assertPostingReaderEquals(postingsIndex, 10, 3, 4, 6);
+        assertPostingReaderEquals(postingsIndex, 12, 2, 8, 10);
+        assertPostingReaderEquals(postingsIndex, 14, 11, 12, 13);
+
+        // Leaf postings...
+        assertTrue(postingsIndex.exists(64));
+        assertTrue(postingsIndex.exists(80));
+        assertTrue(postingsIndex.exists(96));
+        assertTrue(postingsIndex.exists(112));
+
+        assertPostingReaderEquals(postingsIndex, 64, 1, 5, 7);
+        assertPostingReaderEquals(postingsIndex, 80, 3, 4, 6);
+        assertPostingReaderEquals(postingsIndex, 96, 2, 8, 10);
+        assertPostingReaderEquals(postingsIndex, 112, 11, 12, 13);
+    }
+
+    @Test
+    public void shouldSkipPostingListWhenSamplingMisses() throws Exception
+    {
+        List<PackedLongValues> leaves = Collections.singletonList(postings(1, 2, 3));
+
+        setBDKPostingsWriterSizing(1, 5);
+        BlockBalancedTreePostingsWriter writer = new BlockBalancedTreePostingsWriter();
+
+        // The tree is too short to have any internal posting lists.
+        writer.onLeaf(16, 1, pathToRoot(1, 2, 4, 8));
+
+        long fp;
+        try (IndexOutputWriter output = indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext))
+        {
+            fp = writer.finish(output, leaves, indexContext);
+        }
+
+        // There is only a single posting list...the leaf posting list.
+        BlockBalancedTreePostingsIndex postingsIndex = new BlockBalancedTreePostingsIndex(indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext), fp);
+        assertEquals(1, postingsIndex.size());
+    }
+
+    @Test
+    public void shouldSkipPostingListWhenTooFewLeaves() throws Exception
+    {
+        List<PackedLongValues> leaves = Collections.singletonList(postings(1, 2, 3));
+
+        setBDKPostingsWriterSizing(2, 2);
+        BlockBalancedTreePostingsWriter writer = new BlockBalancedTreePostingsWriter();
+
+        // The tree is too short to have any internal posting lists.
+        writer.onLeaf(16, 1, pathToRoot(1, 2, 4, 8));
+
+        long fp;
+        try (IndexOutputWriter output = indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext))
+        {
+            fp = writer.finish(output, leaves, indexContext);
+        }
+
+        // There is only a single posting list...the leaf posting list.
+        BlockBalancedTreePostingsIndex postingsIndex = new BlockBalancedTreePostingsIndex(indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext), fp);
+        assertEquals(1, postingsIndex.size());
+    }
+
+    private void assertPostingReaderEquals(BlockBalancedTreePostingsIndex postingsIndex, int nodeID, long... postings) throws IOException
+    {
+        assertPostingReaderEquals(indexDescriptor.openPerIndexInput(IndexComponent.POSTING_LISTS, indexContext),
+                                  postingsIndex.getPostingsFilePointer(nodeID),
+                                  new ArrayPostingList(postings));
+    }
+
+    private void assertPostingReaderEquals(IndexInput input, long offset, PostingList expected) throws IOException
+    {
+        try (PostingsReader reader = new PostingsReader(input, offset, mock(QueryEventListener.PostingListEventListener.class)))
+        {
+            assertPostingListEquals(expected, reader);
+        }
+    }
+
+    private PackedLongValues postings(int... postings)
+    {
+        final PackedLongValues.Builder builder = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
+        for (int posting : postings)
+        {
+            builder.add(posting);
+        }
+        return builder.build();
+    }
+
+    private IntArrayList pathToRoot(int... nodes)
+    {
+        final IntArrayList path = new IntArrayList();
+        for (int node : nodes)
+        {
+            path.add(node);
+        }
+        return path;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueriesTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueriesTest.java
new file mode 100644
index 0000000..a055236
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueriesTest.java
@@ -0,0 +1,220 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
+
+import static org.apache.lucene.index.PointValues.Relation.CELL_CROSSES_QUERY;
+import static org.apache.lucene.index.PointValues.Relation.CELL_INSIDE_QUERY;
+import static org.apache.lucene.index.PointValues.Relation.CELL_OUTSIDE_QUERY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class BlockBalancedTreeQueriesTest extends SAIRandomizedTester
+{
+    @Test
+    public void testMatchesAll()
+    {
+        Expression expression = new Expression(SAITester.createIndexContext("meh", Int32Type.instance));
+        BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+        for (int visit = 0; visit < between(100, 1000); visit++)
+            assertTrue(query.contains(toSortableBytes(nextInt(Integer.MAX_VALUE))));
+
+        for (int compare = 0; compare < between(100, 1000); compare++)
+            assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(between(0, Integer.MAX_VALUE/2)),
+                                                          toSortableBytes(between(Integer.MAX_VALUE/2, Integer.MAX_VALUE))));
+    }
+
+    @Test
+    public void testInclusiveLowerBound()
+    {
+        int lowerBound = between(-10, 10);
+        Expression expression = buildExpression(Operator.GTE, lowerBound);
+        BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+        assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+        assertTrue(query.contains(toSortableBytes(lowerBound)));
+        assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 2), toSortableBytes(lowerBound - 1)));
+        assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(lowerBound + 1)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+    }
+
+    @Test
+    public void testExclusiveLowerBound()
+    {
+        int lowerBound = between(-10, 10);
+        Expression expression = buildExpression(Operator.GT, lowerBound);
+        BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+        assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+        assertFalse(query.contains(toSortableBytes(lowerBound)));
+        assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+        assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound + 1), toSortableBytes(lowerBound + 2)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(lowerBound + 1)));
+    }
+
+    @Test
+    public void testInclusiveUpperBound()
+    {
+        int upperBound = between(-10, 10);
+        Expression expression = buildExpression(Operator.LTE, upperBound);
+        BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+        assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+        assertTrue(query.contains(toSortableBytes(upperBound)));
+        assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound + 1), toSortableBytes(upperBound + 2)));
+        assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(upperBound - 1), toSortableBytes(upperBound)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+    }
+
+    @Test
+    public void testExclusiveUpperBound()
+    {
+        int upper = between(-10, 10);
+        Expression expression = buildExpression(Operator.LT, upper);
+        BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+        assertTrue(query.contains(toSortableBytes(upper - 1)));
+        assertFalse(query.contains(toSortableBytes(upper)));
+        assertFalse(query.contains(toSortableBytes(upper + 1)));
+
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upper), toSortableBytes(upper + 1)));
+        assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(upper - 2), toSortableBytes(upper - 1)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upper - 1), toSortableBytes(upper)));
+    }
+
+    @Test
+    public void testInclusiveLowerAndUpperBound()
+    {
+        int lowerBound = between(-15, 15);
+        int upperBound = lowerBound + 5;
+        Expression expression = buildExpression(Operator.GTE, lowerBound).add(Operator.LTE, Int32Type.instance.decompose(upperBound));
+        BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+        assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+        assertTrue(query.contains(toSortableBytes(lowerBound)));
+        assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+        assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+        assertTrue(query.contains(toSortableBytes(upperBound)));
+        assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 2), toSortableBytes(lowerBound - 1)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+        assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(upperBound)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound + 1), toSortableBytes(upperBound + 2)));
+    }
+
+    @Test
+    public void testExclusiveLowerAndUpperBound()
+    {
+        int lowerBound = between(-15, 15);
+        int upperBound = lowerBound + 5;
+        Expression expression = buildExpression(Operator.GT, lowerBound).add(Operator.LT, Int32Type.instance.decompose(upperBound));
+        BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+        assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+        assertFalse(query.contains(toSortableBytes(lowerBound)));
+        assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+        assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+        assertFalse(query.contains(toSortableBytes(upperBound)));
+        assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(lowerBound + 1)));
+        assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound + 1), toSortableBytes(upperBound - 1)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound - 1), toSortableBytes(upperBound)));
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+    }
+
+    @Test
+    public void testExclusiveLowerAndInclusiveUpperBound()
+    {
+        int lowerBound = between(-15, 15);
+        int upperBound = lowerBound + 5;
+        Expression expression = buildExpression(Operator.GT, lowerBound).add(Operator.LTE, Int32Type.instance.decompose(upperBound));
+        BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+        assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+        assertFalse(query.contains(toSortableBytes(lowerBound)));
+        assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+        assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+        assertTrue(query.contains(toSortableBytes(upperBound)));
+        assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(lowerBound + 1)));
+        assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound + 1), toSortableBytes(upperBound)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound + 1), toSortableBytes(upperBound + 2)));
+    }
+
+    @Test
+    public void testInclusiveLowerAndExclusiveUpperBound()
+    {
+        int lowerBound = between(-15, 15);
+        int upperBound = lowerBound + 5;
+        Expression expression = buildExpression(Operator.GTE, lowerBound).add(Operator.LT, Int32Type.instance.decompose(upperBound));
+        BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+        assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+        assertTrue(query.contains(toSortableBytes(lowerBound)));
+        assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+        assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+        assertFalse(query.contains(toSortableBytes(upperBound)));
+        assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 2), toSortableBytes(lowerBound - 1)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+        assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(upperBound - 1)));
+        assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound - 1), toSortableBytes(upperBound)));
+        assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+    }
+
+    private byte[] toSortableBytes(int value)
+    {
+        byte[] buffer = new byte[4];
+        ByteSource source = Int32Type.instance.asComparableBytes(Int32Type.instance.decompose(value), ByteComparable.Version.OSS50);
+        ByteSourceInverse.copyBytes(source, buffer);
+        return buffer;
+    }
+
+    private Expression buildExpression(Operator op, int value)
+    {
+        Expression expression = new Expression(SAITester.createIndexContext("meh", Int32Type.instance));
+        expression.add(op, Int32Type.instance.decompose(value));
+        return expression;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBufferTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBufferTest.java
new file mode 100644
index 0000000..da5faa9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBufferTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.StreamSupport;
+
+import org.junit.Test;
+
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+import static org.junit.Assert.assertEquals;
+
+public class BlockBalancedTreeRamBufferTest extends SAIRandomizedTester
+{
+    @Test
+    public void shouldReturnValuesInSortedValue()
+    {
+        int numRows = nextInt(100, 1000);
+
+        // Generate a random unsorted list of integers
+        List<Integer> values = IntStream.generate(() -> nextInt(0, 1000))
+                                        .distinct()
+                                        .limit(numRows)
+                                        .boxed()
+                                        .collect(Collectors.toList());
+
+        BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+
+        byte[] scratch = new byte[Integer.BYTES];
+        values.forEach(v -> {
+            TypeUtil.toComparableBytes(Int32Type.instance.decompose(v), Int32Type.instance, scratch);
+            buffer.add(0, scratch);
+        });
+
+        Iterable<Pair<byte[], PostingList>> iterable = buffer::iterator;
+
+        List<Integer> result = StreamSupport.stream(iterable.spliterator(), false).mapToInt(pair -> unpackValue(pair.left)).boxed().collect(Collectors.toList());
+
+        Collections.sort(values);
+
+        assertEquals(values, result);
+    }
+
+    private static int unpackValue(byte[] value)
+    {
+        return Int32Type.instance.compose(Int32Type.instance.fromComparableBytes(ByteSource.peekable(ByteSource.fixedLength(value)),
+                                                                                 ByteComparable.Version.OSS50));
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReaderTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReaderTest.java
new file mode 100644
index 0000000..984a064
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReaderTest.java
@@ -0,0 +1,289 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.disk.format.IndexComponent;
+import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.v1.segment.SegmentMetadata;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.util.NumericUtils;
+
+import static org.apache.lucene.index.PointValues.Relation.CELL_CROSSES_QUERY;
+import static org.apache.lucene.index.PointValues.Relation.CELL_INSIDE_QUERY;
+import static org.apache.lucene.index.PointValues.Relation.CELL_OUTSIDE_QUERY;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class BlockBalancedTreeReaderTest extends SAIRandomizedTester
+{
+    private final BlockBalancedTreeReader.IntersectVisitor NONE_MATCH = new BlockBalancedTreeReader.IntersectVisitor()
+    {
+        @Override
+        public boolean contains(byte[] packedValue)
+        {
+            return false;
+        }
+
+        @Override
+        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+        {
+            return CELL_OUTSIDE_QUERY;
+        }
+    };
+
+    private final BlockBalancedTreeReader.IntersectVisitor ALL_MATCH = new BlockBalancedTreeReader.IntersectVisitor()
+    {
+        @Override
+        public boolean contains(byte[] packedValue)
+        {
+            return true;
+        }
+
+        @Override
+        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+        {
+            return CELL_INSIDE_QUERY;
+        }
+    };
+
+    private IndexDescriptor indexDescriptor;
+    private IndexContext indexContext;
+
+    @Before
+    public void setup() throws Throwable
+    {
+        indexDescriptor = newIndexDescriptor();
+        indexContext = SAITester.createIndexContext(newIndex(), Int32Type.instance);
+    }
+
+    @Test
+    public void testFilteringIntersection() throws Exception
+    {
+        int numRows = 1000;
+
+        final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+
+        byte[] scratch = new byte[4];
+        for (int docID = 0; docID < numRows; docID++)
+        {
+            NumericUtils.intToSortableBytes(docID, scratch, 0);
+            buffer.add(docID, scratch);
+        }
+
+        final BlockBalancedTreeReader reader = finishAndOpenReader(4, buffer);
+
+        Expression expression = new Expression(indexContext);
+        expression.add(Operator.GT, Int32Type.instance.decompose(444));
+        expression.add(Operator.LT, Int32Type.instance.decompose(555));
+        PostingList intersection = performIntersection(reader, BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4));
+        assertNotNull(intersection);
+        assertEquals(110, intersection.size());
+        for (long posting = 445; posting < 555; posting++)
+            assertEquals(posting, intersection.nextPosting());
+    }
+
+    @Test
+    public void testAdvance() throws Exception
+    {
+        final int numRows = between(1000, 2000);
+        final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+
+        byte[] scratch = new byte[4];
+        for (int docID = 0; docID < numRows; docID++)
+        {
+            NumericUtils.intToSortableBytes(docID, scratch, 0);
+            buffer.add(docID, scratch);
+        }
+
+        final BlockBalancedTreeReader reader = finishAndOpenReader(2, buffer);
+
+        PostingList intersection = performIntersection(reader, NONE_MATCH);
+        assertNull(intersection);
+
+        intersection = performIntersection(reader, ALL_MATCH);
+        assertEquals(numRows, intersection.size());
+        assertEquals(100, intersection.advance(100));
+        assertEquals(200, intersection.advance(200));
+        assertEquals(300, intersection.advance(300));
+        assertEquals(400, intersection.advance(400));
+        assertEquals(401, intersection.advance(401));
+        long expectedRowID = 402;
+        for (long id = intersection.nextPosting(); expectedRowID < 500; id = intersection.nextPosting())
+        {
+            assertEquals(expectedRowID++, id);
+        }
+        assertEquals(PostingList.END_OF_STREAM, intersection.advance(numRows + 1));
+
+        intersection.close();
+    }
+
+    @Test
+    public void testSameValuesInLeaf() throws Exception
+    {
+        // While a bit synthetic this test is designed to test that the
+        // BlockBalancedTreeReader.FilteringIntersection.buildPostingsFilterForSingleValueLeaf
+        // method is exercised in a test. To do this we need to ensure that
+        // we have at least one leaf that has all the same value and that
+        // all of that leaf is requested in a query.
+        final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+        byte[] scratch = new byte[4];
+
+        for (int docID = 0; docID < 10; docID++)
+        {
+            NumericUtils.intToSortableBytes(docID, scratch, 0);
+            buffer.add(docID, scratch);
+        }
+
+        for (int docID = 10; docID < 20; docID++)
+        {
+            NumericUtils.intToSortableBytes(10, scratch, 0);
+            buffer.add(docID, scratch);
+        }
+
+        for (int docID = 20; docID < 30; docID++)
+        {
+            NumericUtils.intToSortableBytes(docID, scratch, 0);
+            buffer.add(docID, scratch);
+        }
+
+        final BlockBalancedTreeReader reader = finishAndOpenReader(5, buffer);
+
+        PostingList postingList = performIntersection(reader, buildQuery(8, 15));
+
+        assertEquals(8, postingList.nextPosting());
+        assertEquals(9, postingList.nextPosting());
+        assertEquals(10, postingList.nextPosting());
+        assertEquals(11, postingList.nextPosting());
+        assertEquals(12, postingList.nextPosting());
+        assertEquals(13, postingList.nextPosting());
+        assertEquals(14, postingList.nextPosting());
+        assertEquals(15, postingList.nextPosting());
+        assertEquals(16, postingList.nextPosting());
+        assertEquals(17, postingList.nextPosting());
+        assertEquals(18, postingList.nextPosting());
+        assertEquals(19, postingList.nextPosting());
+        assertEquals(PostingList.END_OF_STREAM, postingList.nextPosting());
+    }
+
+    @Test
+    public void testResourcesReleaseWhenQueryDoesntMatchAnything() throws Exception
+    {
+        final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+        byte[] scratch = new byte[4];
+        for (int docID = 0; docID < 1000; docID++)
+        {
+            NumericUtils.intToSortableBytes(docID, scratch, 0);
+            buffer.add(docID, scratch);
+        }
+        // add a gap between 1000 and 1100
+        for (int docID = 1000; docID < 2000; docID++)
+        {
+            NumericUtils.intToSortableBytes(docID + 100, scratch, 0);
+            buffer.add(docID, scratch);
+        }
+
+        final BlockBalancedTreeReader reader = finishAndOpenReader(50, buffer);
+
+        final PostingList intersection = performIntersection(reader, buildQuery(1017, 1096));
+        assertNull(intersection);
+    }
+
+    private PostingList performIntersection(BlockBalancedTreeReader reader, BlockBalancedTreeReader.IntersectVisitor visitor)
+    {
+        QueryEventListener.BalancedTreeEventListener balancedTreeEventListener = mock(QueryEventListener.BalancedTreeEventListener.class);
+        when(balancedTreeEventListener.postingListEventListener()).thenReturn(mock(QueryEventListener.PostingListEventListener.class));
+        return reader.intersect(visitor, balancedTreeEventListener, mock(QueryContext.class));
+    }
+
+    private BlockBalancedTreeReader.IntersectVisitor buildQuery(int queryMin, int queryMax)
+    {
+        return new BlockBalancedTreeReader.IntersectVisitor()
+        {
+            @Override
+            public boolean contains(byte[] packedValue)
+            {
+                int x = NumericUtils.sortableBytesToInt(packedValue, 0);
+                return x >= queryMin && x <= queryMax;
+            }
+
+            @Override
+            public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+            {
+                int min = NumericUtils.sortableBytesToInt(minPackedValue, 0);
+                int max = NumericUtils.sortableBytesToInt(maxPackedValue, 0);
+                assert max >= min;
+
+                if (max < queryMin || min > queryMax)
+                {
+                    return Relation.CELL_OUTSIDE_QUERY;
+                }
+                else if (min >= queryMin && max <= queryMax)
+                {
+                    return CELL_INSIDE_QUERY;
+                }
+                else
+                {
+                    return CELL_CROSSES_QUERY;
+                }
+            }
+        };
+    }
+
+    private BlockBalancedTreeReader finishAndOpenReader(int maxPointsPerLeaf, BlockBalancedTreeRamBuffer buffer) throws Exception
+    {
+        setBDKPostingsWriterSizing(8, 2);
+        final NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+                                                                 indexContext,
+                                                                 maxPointsPerLeaf,
+                                                                 Integer.BYTES,
+                                                                 Math.toIntExact(buffer.numRows()));
+
+        final SegmentMetadata.ComponentMetadataMap metadata = writer.writeCompleteSegment(buffer.iterator());
+        final long treePosition = metadata.get(IndexComponent.BALANCED_TREE).root;
+        assertThat(treePosition, is(greaterThan(0L)));
+        final long postingsPosition = metadata.get(IndexComponent.POSTING_LISTS).root;
+        assertThat(postingsPosition, is(greaterThan(0L)));
+
+        FileHandle treeHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext);
+        FileHandle treePostingsHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext);
+        return new BlockBalancedTreeReader(indexContext,
+                                           treeHandle,
+                                           treePosition,
+                                           treePostingsHandle,
+                                           postingsPosition);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeTest.java
new file mode 100644
index 0000000..710da74
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.util.function.IntFunction;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.ByteBuffersIndexOutput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.util.NumericUtils;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class BlockBalancedTreeTest extends SAIRandomizedTester
+{
+    private ByteBuffersDataOutput dataOutput;
+
+    @Before
+    public void setupDataOutput()
+    {
+        dataOutput = new ByteBuffersDataOutput();
+    }
+
+    @Test
+    public void testEmptyTree() throws Exception
+    {
+        long treeFilePointer = writeBalancedTree(0, 100, rowID -> rowID);
+
+        assertEquals(-1, treeFilePointer);
+    }
+
+    @Test
+    public void testSingleLeaf() throws Exception
+    {
+        BlockBalancedTreeWalker.TraversalState state = generateBalancedTree(100, 100, rowID -> rowID);
+
+        assertEquals(1, state.numLeaves);
+        assertEquals(1, state.treeDepth);
+        assertEquals(100, state.valueCount);
+        assertTrue(state.atLeafNode());
+
+        recursiveAssertTraversal(state, -1);
+
+        assertEquals(state.treeDepth, state.maxLevel + 1);
+    }
+
+    @Test
+    public void testTreeWithSameValue() throws Exception
+    {
+        BlockBalancedTreeWalker.TraversalState state = generateBalancedTree(100, 4, rowID -> 1);
+
+        recursiveAssertTraversal(state, -1);
+
+        assertEquals(state.treeDepth, state.maxLevel + 1);
+    }
+
+    @Test
+    public void testTreeDepthNeverMoreThanNumberOfLeaves() throws Exception
+    {
+        int leafSize = 4;
+        for (int numLeaves = 1; numLeaves < 1000; numLeaves++)
+        {
+            int numRows = leafSize * numLeaves;
+
+            BlockBalancedTreeWalker.TraversalState state = generateBalancedTree(numRows, leafSize, rowID -> rowID);
+
+            assertEquals(numLeaves, state.numLeaves);
+            assertTrue(state.treeDepth <= state.numLeaves);
+
+            recursiveAssertTraversal(state, -1);
+
+            assertEquals(state.treeDepth, state.maxLevel + 1);
+        }
+    }
+
+    @Test
+    public void randomisedTreeTest() throws Exception
+    {
+        int loops = nextInt(10, 1000);
+
+        for (int loop = 0; loop < loops; loop++)
+        {
+            int leafSize = nextInt(2, 512);
+            int numRows = nextInt(1000, 10000);
+
+            BlockBalancedTreeWalker.TraversalState state = generateBalancedTree(numRows, leafSize, rowID -> nextInt(0, numRows / 2));
+
+            recursiveAssertTraversal(state, -1);
+
+            assertEquals(state.treeDepth, state.maxLevel + 1);
+        }
+    }
+
+    private long recursiveAssertTraversal(BlockBalancedTreeWalker.TraversalState state, long lastLeafBlockFP)
+    {
+        if (state.atLeafNode())
+        {
+            assertTrue(state.nodeExists());
+            assertTrue(state.getLeafBlockFP() > lastLeafBlockFP);
+            return state.getLeafBlockFP();
+        }
+        else
+        {
+            state.pushLeft();
+            lastLeafBlockFP = recursiveAssertTraversal(state, lastLeafBlockFP);
+            state.pop();
+
+            state.pushRight();
+            lastLeafBlockFP = recursiveAssertTraversal(state, lastLeafBlockFP);
+            state.pop();
+
+            return lastLeafBlockFP;
+        }
+    }
+
+    private BlockBalancedTreeWalker.TraversalState generateBalancedTree(int numRows, int leafSize, IntFunction<Integer> valueProvider) throws Exception
+    {
+        long treeOffset = writeBalancedTree(numRows, leafSize, valueProvider);
+
+        DataInput input = dataOutput.toDataInput();
+
+        input.skipBytes(treeOffset);
+        return new BlockBalancedTreeWalker.TraversalState(input);
+    }
+
+    private long writeBalancedTree(int numRows, int leafSize, IntFunction<Integer> valueProvider) throws Exception
+    {
+        final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+
+        byte[] scratch = new byte[4];
+        for (int rowID = 0; rowID < numRows; rowID++)
+        {
+            NumericUtils.intToSortableBytes(valueProvider.apply(rowID), scratch, 0);
+            buffer.add(rowID, scratch);
+        }
+
+        BlockBalancedTreeWriter writer = new BlockBalancedTreeWriter(4, leafSize);
+        ByteBuffersIndexOutput output = new ByteBuffersIndexOutput(dataOutput, "test", "test");
+        return writer.write(output, buffer.iterator(), (leafPostings, offset, count) -> {});
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMapTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMapTest.java
new file mode 100644
index 0000000..86e8608
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMapTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.nio.ByteBuffer;
+
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import org.apache.cassandra.index.sai.disk.ResettableByteBuffersIndexOutput;
+import org.apache.cassandra.index.sai.disk.io.SeekingRandomAccessInput;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.packed.DirectReader;
+import org.apache.lucene.util.packed.DirectWriter;
+
+import static org.junit.Assert.assertEquals;
+
+public class LeafOrderMapTest extends SAIRandomizedTester
+{
+    @Test
+    public void test() throws Exception
+    {
+        int[] array = new int[1024];
+        for (int x=0; x < array.length; x++)
+        {
+            array[x] = x;
+        }
+        shuffle(array);
+
+        ResettableByteBuffersIndexOutput out = new ResettableByteBuffersIndexOutput("");
+
+        LeafOrderMap.write(array, array.length, array.length - 1, out);
+
+        IndexInput input = new ByteBuffersIndexInput(new ByteBuffersDataInput(Lists.newArrayList(ByteBuffer.wrap(out.toArrayCopy()))), "");
+
+        final byte bits = (byte) DirectWriter.unsignedBitsRequired(array.length - 1);
+
+        for (int index = 0; index < array.length; index++)
+        {
+            LongValues reader = DirectReader.getInstance(new SeekingRandomAccessInput(input), bits);
+
+            int value = Math.toIntExact(reader.get(index));
+
+            assertEquals(array[index], value);
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriterTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriterTest.java
new file mode 100644
index 0000000..acaff3b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriterTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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.index.sai.disk.v1.bbtree;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import com.carrotsearch.hppc.LongArrayList;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.disk.format.IndexComponent;
+import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.v1.segment.SegmentMetadata;
+import org.apache.cassandra.index.sai.memory.MemtableTermsIterator;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.cassandra.index.sai.utils.TermsIterator;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.AbstractGuavaIterator;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.NumericUtils;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class NumericIndexWriterTest extends SAIRandomizedTester
+{
+    private IndexDescriptor indexDescriptor;
+    private IndexContext indexContext;
+
+    @Before
+    public void setup() throws Throwable
+    {
+        indexDescriptor = newIndexDescriptor();
+        indexContext = SAITester.createIndexContext(newIndex(), Int32Type.instance);
+    }
+
+    @Test
+    public void shouldFlushFromRamBuffer() throws Exception
+    {
+        final BlockBalancedTreeRamBuffer ramBuffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+        final int numRows = 120;
+        int currentValue = numRows;
+        for (int i = 0; i < numRows; ++i)
+        {
+            byte[] scratch = new byte[Integer.BYTES];
+            NumericUtils.intToSortableBytes(currentValue--, scratch, 0);
+            ramBuffer.add(i, scratch);
+        }
+
+        int rowCount = ramBuffer.numRows();
+
+        SegmentMetadata.ComponentMetadataMap indexMetas;
+
+        NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+                                                           indexContext,
+                                                           Integer.BYTES,
+                                                           rowCount);
+        indexMetas = writer.writeCompleteSegment(ramBuffer.iterator());
+
+        final FileHandle treeHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext);
+        final FileHandle treePostingsHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext);
+
+        try (BlockBalancedTreeReader reader = new BlockBalancedTreeReader(indexContext,
+                                                                          treeHandle,
+                                                                          indexMetas.get(IndexComponent.BALANCED_TREE).root,
+                                                                          treePostingsHandle,
+                                                                          indexMetas.get(IndexComponent.POSTING_LISTS).root))
+        {
+            final Counter visited = Counter.newCounter();
+            try (final PostingList ignored = reader.intersect(new BlockBalancedTreeReader.IntersectVisitor()
+            {
+                @Override
+                public boolean contains(byte[] packedValue)
+                {
+                    // we should read point values in reverse order after sorting
+                    assertEquals(1 + visited.get(), NumericUtils.sortableBytesToInt(packedValue, 0));
+                    visited.addAndGet(1);
+                    return true;
+                }
+
+                @Override
+                public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+                {
+                    return PointValues.Relation.CELL_CROSSES_QUERY;
+                }
+            }, mockEventListener(), mock(QueryContext.class)))
+            {
+                assertEquals(numRows, visited.get());
+            }
+        }
+    }
+
+    @Test
+    public void shouldFlushFromMemtable() throws Exception
+    {
+        final int maxSegmentRowId = 100;
+        final TermsIterator termEnum = buildTermEnum(0, maxSegmentRowId);
+
+        SegmentMetadata.ComponentMetadataMap indexMetas;
+        NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+                                                           indexContext,
+                                                           TypeUtil.fixedSizeOf(Int32Type.instance),
+                                                           maxSegmentRowId);
+        indexMetas = writer.writeCompleteSegment(BlockBalancedTreeIterator.fromTermsIterator(termEnum, Int32Type.instance));
+
+        final FileHandle treeHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext);
+        final FileHandle treePostingsHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext);
+
+        try (BlockBalancedTreeReader reader = new BlockBalancedTreeReader(indexContext,
+                                                                          treeHandle,
+                                                                          indexMetas.get(IndexComponent.BALANCED_TREE).root,
+                                                                          treePostingsHandle,
+                                                                          indexMetas.get(IndexComponent.POSTING_LISTS).root
+        ))
+        {
+            final Counter visited = Counter.newCounter();
+            try (final PostingList ignored = reader.intersect(new BlockBalancedTreeReader.IntersectVisitor()
+            {
+                @Override
+                public boolean contains(byte[] packedValue)
+                {
+                    final ByteComparable actualTerm = ByteComparable.fixedLength(packedValue);
+                    final ByteComparable expectedTerm = ByteComparable.of(Math.toIntExact(visited.get()));
+                    assertEquals("Point value mismatch after visiting " + visited.get() + " entries.", 0,
+                                 ByteComparable.compare(actualTerm, expectedTerm, ByteComparable.Version.OSS50));
+
+                    visited.addAndGet(1);
+                    return true;
+                }
+
+                @Override
+                public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+                {
+                    return PointValues.Relation.CELL_CROSSES_QUERY;
+                }
+            }, mockEventListener(), mock(QueryContext.class)))
+            {
+                assertEquals(maxSegmentRowId, visited.get());
+            }
+        }
+    }
+
+    private QueryEventListener.BalancedTreeEventListener mockEventListener()
+    {
+        QueryEventListener.BalancedTreeEventListener balancedTreeEventListener = mock(QueryEventListener.BalancedTreeEventListener.class);
+        when(balancedTreeEventListener.postingListEventListener()).thenReturn(mock(QueryEventListener.PostingListEventListener.class));
+        return balancedTreeEventListener;
+    }
+
+    private TermsIterator buildTermEnum(int startTermInclusive, int endTermExclusive)
+    {
+        final ByteBuffer minTerm = Int32Type.instance.decompose(startTermInclusive);
+        final ByteBuffer maxTerm = Int32Type.instance.decompose(endTermExclusive);
+
+        final AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>> iterator = new AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>>()
+        {
+            private int currentTerm = startTermInclusive;
+            private int currentRowId = 0;
+
+            @Override
+            protected Pair<ByteComparable, LongArrayList> computeNext()
+            {
+                if (currentTerm >= endTermExclusive)
+                {
+                    return endOfData();
+                }
+                final ByteBuffer term = Int32Type.instance.decompose(currentTerm++);
+                final LongArrayList postings = new LongArrayList();
+                postings.add(currentRowId++);
+                final ByteSource encoded = Int32Type.instance.asComparableBytes(term, ByteComparable.Version.OSS50);
+                return Pair.create(v -> encoded, postings);
+            }
+        };
+
+        return new MemtableTermsIterator(minTerm, maxTerm, iterator);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/CompactionTest.java b/test/unit/org/apache/cassandra/index/sai/functional/CompactionTest.java
index dfb22ac..e13cca5 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/CompactionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/CompactionTest.java
@@ -24,6 +24,7 @@
 import java.util.Collections;
 
 import com.google.common.collect.Lists;
+import org.junit.Assert;
 import org.junit.Test;
 
 import com.datastax.driver.core.exceptions.InvalidQueryException;
@@ -71,19 +72,19 @@
     public void testAntiCompaction() throws Throwable
     {
         createTable(CREATE_TABLE_TEMPLATE);
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
         verifyNoIndexFiles();
 
         // create 100 rows in 1 sstable
         int num = 100;
         for (int i = 0; i < num; i++)
-            execute( "INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+            execute( "INSERT INTO %s (id1, v1) VALUES (?, 0)", Integer.toString(i));
         flush();
 
         // verify 1 sstable index
-        assertNumRows(num, "SELECT * FROM %%s WHERE v1 = '0'");
-        verifyIndexFiles(literalIndexContext, 1);
-        verifySSTableIndexes(literalIndexContext.getIndexName(), 1);
+        assertNumRows(num, "SELECT * FROM %%s WHERE v1 >= 0");
+        verifyIndexFiles(numericIndexContext, null, 1, 0);
+        verifySSTableIndexes(numericIndexContext.getIndexName(), 1);
 
         // split sstable into repaired and unrepaired
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
@@ -108,24 +109,26 @@
         }
 
         // verify 2 sstable indexes
-        assertNumRows(num, "SELECT * FROM %%s WHERE v1 = '0'");
-        waitForAssert(() -> verifyIndexFiles(literalIndexContext, 2));
-        verifySSTableIndexes(literalIndexContext.getIndexName(), 2);
+        assertNumRows(num, "SELECT * FROM %%s WHERE v1 >= 0");
+        waitForAssert(() -> verifyIndexFiles(numericIndexContext, null, 2, 0));
+        verifySSTableIndexes(numericIndexContext.getIndexName(), 2);
 
         // index components are included after anti-compaction
         verifyIndexComponentsIncludedInSSTable();
     }
 
     @Test
-    public void testConcurrentQueryWithCompaction() throws Throwable
+    public void testConcurrentQueryWithCompaction()
     {
         createTable(CREATE_TABLE_TEMPLATE);
         String v1IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        String v2IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+        waitForTableIndexesQueryable();
 
         int num = 10;
         for (int i = 0; i < num; i++)
         {
-            execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+            execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
             flush();
         }
 
@@ -134,7 +137,8 @@
             {
                 try
                 {
-                    assertNumRows(num, "SELECT id1 FROM %s WHERE v1='0'");
+                    assertNumRows(num, "SELECT id1 FROM %s WHERE v1>=0");
+                    assertNumRows(num, "SELECT id1 FROM %s WHERE v2='0'");
                 }
                 catch (Throwable e)
                 {
@@ -146,6 +150,7 @@
         compactionTest.start();
 
         verifySSTableIndexes(v1IndexName, num);
+        verifySSTableIndexes(v2IndexName, num);
     }
 
     @Test
@@ -153,6 +158,7 @@
     {
         createTable(CREATE_TABLE_TEMPLATE);
         String v1IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        String v2IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
 
         int sstables = 2;
         int num = 10;
@@ -161,7 +167,7 @@
             if (i == num / sstables)
                 flush();
 
-            execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+            execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
         }
         flush();
 
@@ -186,11 +192,10 @@
         }
         catch (Throwable e)
         {
-            Throwable cause = e;
-            while (cause.getCause() != null)
-                cause = cause.getCause();
+            while (e.getCause() != null)
+                e = e.getCause();
 
-            assertTrue(String.format("Expected %s, but got %s", errMessage, cause.getMessage()), cause.getMessage().contains(errMessage));
+            assertTrue(String.format("Expected %s, but got %s", errMessage, e.getMessage()), e.getMessage().contains(errMessage));
         }
         finally
         {
@@ -200,8 +205,10 @@
         assertNotEquals(0, earlyOpenCounter.get());
 
         // verify indexes are working
-        assertNumRows(num, "SELECT id1 FROM %%s WHERE v1='0'");
+        assertNumRows(num, "SELECT id1 FROM %%s WHERE v1=0");
+        assertNumRows(num, "SELECT id1 FROM %%s WHERE v2='0'");
         verifySSTableIndexes(v1IndexName, sstables);
+        verifySSTableIndexes(v2IndexName, sstables);
     }
 
     @Test
@@ -214,7 +221,7 @@
         int num = 100;
         for (int i = 0; i < num; i++)
         {
-            execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+            execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
         }
         flush();
 
@@ -229,34 +236,36 @@
             Injections.inject(compactionLatch);
 
             TestWithConcurrentVerification compactionTask = new TestWithConcurrentVerification(
-                    () -> {
-                        try
-                        {
-                            upgradeSSTables();
-                            fail("Expected CompactionInterruptedException");
-                        }
-                        catch (Exception e)
-                        {
-                            assertTrue("Expected CompactionInterruptedException, but got " + e,
-                                       Throwables.isCausedBy(e, CompactionInterruptedException.class::isInstance));
-                        }
-                    },
-                    () -> {
-                        try
-                        {
-                            waitForAssert(() -> assertEquals(1, compactionLatch.getCount()));
+            () -> {
+                try
+                {
+                    upgradeSSTables();
+                    fail("Expected CompactionInterruptedException");
+                }
+                catch (Exception e)
+                {
+                    assertTrue("Expected CompactionInterruptedException, but got " + e,
+                               Throwables.isCausedBy(e, CompactionInterruptedException.class::isInstance));
+                }
+            },
+            () -> {
+                try
+                {
+                    waitForAssert(() -> Assert.assertEquals(1, compactionLatch.getCount()));
 
-                            // build indexes on SSTables that will be compacted soon
-                            createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+                    // build indexes on SSTables that will be compacted soon
+                    createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+                    createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+                    waitForTableIndexesQueryable();
 
-                            // continue in-progress compaction
-                            compactionLatch.countDown();
-                        }
-                        catch (Exception e)
-                        {
-                            throw new RuntimeException(e);
-                        }
-                    }, -1 // run verification task once
+                    // continue in-progress compaction
+                    compactionLatch.countDown();
+                }
+                catch (Exception e)
+                {
+                    throw new RuntimeException(e);
+                }
+            }, -1 // run verification task once
             );
 
             compactionTask.start();
@@ -266,8 +275,10 @@
             compactionLatch.disable();
         }
 
-        assertNumRows(num, "SELECT id1 FROM %%s WHERE v1='0'");
+        assertNumRows(num, "SELECT id1 FROM %%s WHERE v1>=0");
+        assertNumRows(num, "SELECT id1 FROM %%s WHERE v2='0'");
         verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V1_COLUMN_IDENTIFIER), sstables);
+        verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V2_COLUMN_IDENTIFIER), sstables);
     }
 
     @Test
@@ -275,12 +286,14 @@
     {
         createTable(CREATE_TABLE_TEMPLATE);
         String v1IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        String v2IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+
 
         // Load data into a single SSTable...
         int num = 100;
         for (int i = 0; i < num; i++)
         {
-            execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+            execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
         }
         flush();
 
@@ -288,32 +301,33 @@
         assertNotEquals(0, getDiskUsage());
 
         Injections.Barrier compactionLatch =
-                Injections.newBarrier("pause_compaction_for_drop", 2, false)
-                          .add(InvokePointBuilder.newInvokePoint().onClass(SSTableIndexWriter.class).onMethod("addRow"))
-                          .build();
+        Injections.newBarrier("pause_compaction_for_drop", 2, false)
+                  .add(InvokePointBuilder.newInvokePoint().onClass(SSTableIndexWriter.class).onMethod("addRow"))
+                  .build();
         try
         {
             // pause in-progress compaction
             Injections.inject(compactionLatch);
 
             TestWithConcurrentVerification compactionTask = new TestWithConcurrentVerification(
-                    this::upgradeSSTables,
-                    () -> {
-                        try
-                        {
-                            waitForAssert(() -> assertEquals(1, compactionLatch.getCount()));
+            this::upgradeSSTables,
+            () -> {
+                try
+                {
+                    waitForAssert(() -> Assert.assertEquals(1, compactionLatch.getCount()));
 
-                            // drop all indexes
-                            dropIndex("DROP INDEX %s." + v1IndexName);
+                    // drop all indexes
+                    dropIndex("DROP INDEX %s." + v1IndexName);
+                    dropIndex("DROP INDEX %s." + v2IndexName);
 
-                            // continue in-progress compaction
-                            compactionLatch.countDown();
-                        }
-                        catch (Throwable e)
-                        {
-                            throw new RuntimeException(e);
-                        }
-                    }, -1 // run verification task once
+                    // continue in-progress compaction
+                    compactionLatch.countDown();
+                }
+                catch (Throwable e)
+                {
+                    throw new RuntimeException(e);
+                }
+            }, -1 // run verification task once
             );
 
             compactionTask.start();
@@ -330,8 +344,11 @@
 
         // verify indexes are dropped
         // verify indexes are dropped
-        assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1='0'"))
-                .isInstanceOf(InvalidQueryException.class)
-                .hasMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE);
+        assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1>=0"))
+        .isInstanceOf(InvalidQueryException.class)
+        .hasMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE);
+        assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v2='0'"))
+        .isInstanceOf(InvalidQueryException.class)
+        .hasMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE);
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/DiskSpaceTest.java b/test/unit/org/apache/cassandra/index/sai/functional/DiskSpaceTest.java
index 152d472..1d052c1 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/DiskSpaceTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/DiskSpaceTest.java
@@ -33,7 +33,7 @@
         int rows = 1000;
         for (int j = 0; j < rows; j++)
         {
-            execute("INSERT INTO %s (id1, v1) VALUES (?, '1')", Integer.toString(j));
+            execute("INSERT INTO %s (id1, v1) VALUES (?, 1)", Integer.toString(j));
         }
         flush();
 
@@ -41,6 +41,7 @@
 
         // create index, disk space should include index components
         String indexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        waitForTableIndexesQueryable();
 
         long indexSize = indexDiskSpaceUse();
         long sstableSizeWithIndex = totalDiskSpaceUsed();
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/DropTableTest.java b/test/unit/org/apache/cassandra/index/sai/functional/DropTableTest.java
index 191525e..b5322d3 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/DropTableTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/DropTableTest.java
@@ -50,11 +50,13 @@
     {
         createTable(CREATE_TABLE_TEMPLATE);
         createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+        waitForTableIndexesQueryable();
 
         int rows = 100;
         for (int j = 0; j < rows; j++)
         {
-            execute("INSERT INTO %s (id1, v1) VALUES (?, '1')", Integer.toString(j));
+            execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 1 , '1')", Integer.toString(j));
         }
         flush();
 
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/FailureTest.java b/test/unit/org/apache/cassandra/index/sai/functional/FailureTest.java
index 893e304..16b3164 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/FailureTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/FailureTest.java
@@ -39,18 +39,18 @@
     public void shouldMakeIndexNonQueryableOnSSTableContextFailureDuringFlush() throws Throwable
     {
         createTable(CREATE_TABLE_TEMPLATE);
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
-        execute("INSERT INTO %s (id1, v1) VALUES ('2', '2')");
+        execute("INSERT INTO %s (id1, v1) VALUES ('1', 1)");
+        execute("INSERT INTO %s (id1, v1) VALUES ('2', 2)");
         flush();
 
-        assertEquals(1, execute("SELECT id1 FROM %s WHERE v1='1'").size());
+        assertEquals(1, execute("SELECT id1 FROM %s WHERE v1 > 1").size());
 
-        verifyIndexFiles(literalIndexContext, 1, 1, 1);
-        verifySSTableIndexes(literalIndexContext.getIndexName(), 1, 1);
+        verifyIndexFiles(numericIndexContext, null, 1, 1, 0, 1, 0);
+        verifySSTableIndexes(numericIndexContext.getIndexName(), 1, 1);
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('3', '3')");
+        execute("INSERT INTO %s (id1, v1) VALUES ('3', 3)");
 
         Injection ssTableContextCreationFailure = newFailureOnEntry("context_failure_on_flush", SSTableContext.class, "<init>", RuntimeException.class);
         Injections.inject(ssTableContextCreationFailure);
@@ -58,7 +58,7 @@
         flush();
 
         // Verify that, while the node is still operational, the index is not.
-        Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1='1'"))
+        Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1 > 1"))
                   .isInstanceOf(IndexNotAvailableException.class);
 
         ssTableContextCreationFailure.disable();
@@ -66,28 +66,28 @@
         // Now verify that a restart actually repairs the index...
         simulateNodeRestart();
 
-        verifyIndexFiles(literalIndexContext, 2);
-        verifySSTableIndexes(literalIndexContext.getIndexName(), 2, 2);
+        verifyIndexFiles(numericIndexContext, null, 2, 0);
+        verifySSTableIndexes(numericIndexContext.getIndexName(), 2, 2);
 
-        assertEquals(1, execute("SELECT id1 FROM %s WHERE v1='1'").size());
+        assertEquals(2, execute("SELECT id1 FROM %s WHERE v1 > 1").size());
     }
 
     @Test
     public void shouldMakeIndexNonQueryableOnSSTableContextFailureDuringCompaction() throws Throwable
     {
         createTable(CREATE_TABLE_TEMPLATE);
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
+        execute("INSERT INTO %s (id1, v1) VALUES ('1', 1)");
         flush();
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('2', '2')");
+        execute("INSERT INTO %s (id1, v1) VALUES ('2', 2)");
         flush();
 
-        assertEquals(1, execute("SELECT id1 FROM %s WHERE v1='1'").size());
+        assertEquals(1, execute("SELECT id1 FROM %s WHERE v1 > 1").size());
 
-        verifyIndexFiles(literalIndexContext, 2, 2, 2);
-        verifySSTableIndexes(literalIndexContext.getIndexName(), 2, 2);
+        verifyIndexFiles(numericIndexContext, null, 2, 2, 0, 2, 0);
+        verifySSTableIndexes(numericIndexContext.getIndexName(), 2, 2);
 
         Injection ssTableContextCreationFailure = newFailureOnEntry("context_failure_on_compaction", SSTableContext.class, "<init>", RuntimeException.class);
         Injections.inject(ssTableContextCreationFailure);
@@ -95,7 +95,7 @@
         compact();
 
         // Verify that the index is not available.
-        Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1='1'"))
+        Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1 > 1"))
                   .isInstanceOf(IndexNotAvailableException.class);
     }
 
@@ -104,22 +104,26 @@
     {
         createTable(CREATE_TABLE_TEMPLATE);
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
-        execute("INSERT INTO %s (id1, v1) VALUES ('2', '2')");
+        execute("INSERT INTO %s (id1, v1) VALUES ('1', 1)");
+        execute("INSERT INTO %s (id1, v1) VALUES ('2', 2)");
+
+        // We need to reference SSTableContext first or the failure injection fails
+        // because byteman can't find the class.
+        SSTableContext.class.getName();
 
         Injection ssTableContextCreationFailure = newFailureOnEntry("context_failure_on_creation", SSTableContext.class, "<init>", RuntimeException.class);
         Injections.inject(ssTableContextCreationFailure);
 
-        String v1IndexName = createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+        String v2IndexName = createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v2"));
 
         // Verify that the initial index build fails...
-        verifyInitialIndexFailed(v1IndexName);
+        verifyInitialIndexFailed(v2IndexName);
 
         verifyNoIndexFiles();
-        verifySSTableIndexes(v1IndexName, 0);
+        verifySSTableIndexes(v2IndexName, 0);
 
         // ...and then verify that, while the node is still operational, the index is not.
-        Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1='1'"))
+        Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v2 = '1'"))
                   .isInstanceOf(IndexNotAvailableException.class);
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/FlushingTest.java b/test/unit/org/apache/cassandra/index/sai/functional/FlushingTest.java
index 7c067a5..79a126d 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/FlushingTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/FlushingTest.java
@@ -23,38 +23,57 @@
 import org.junit.Test;
 
 import com.datastax.driver.core.ResultSet;
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.index.sai.IndexContext;
 import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.NumericIndexWriter;
 
 import static org.junit.Assert.assertEquals;
 
 public class FlushingTest extends SAITester
 {
     @Test
+    public void testFlushingLargeStaleMemtableIndex() throws Throwable
+    {
+        createTable(CREATE_TABLE_TEMPLATE);
+        createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+
+        int overwrites = NumericIndexWriter.MAX_POINTS_IN_LEAF_NODE + 1;
+        for (int j = 0; j < overwrites; j++)
+        {
+            execute("INSERT INTO %s (id1, v1) VALUES ('1', ?)", j);
+        }
+
+        flush();
+
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+        assertEquals(1, rows.all().size());
+    }
+
+    @Test
     public void testFlushingOverwriteDelete() throws Throwable
     {
         createTable(CREATE_TABLE_TEMPLATE);
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
 
         int sstables = 3;
         for (int j = 0; j < sstables; j++)
         {
-            execute("INSERT INTO %s (id1, v1) VALUES (?, '1')", Integer.toString(j));
+            execute("INSERT INTO %s (id1, v1) VALUES (?, 1)", Integer.toString(j));
             execute("DELETE FROM %s WHERE id1 = ?", Integer.toString(j));
             flush();
         }
 
-        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1 = '1'");
+        ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1 >= 0");
         assertEquals(0, rows.all().size());
-        verifyIndexFiles(literalIndexContext, sstables, 0, 3);
-        verifySSTableIndexes(literalIndexContext.getIndexName(), sstables, 0);
+        verifyIndexFiles(numericIndexContext, null, sstables, 0, 0, sstables, 0);
+        verifySSTableIndexes(numericIndexContext.getIndexName(), sstables, 0);
 
         compact();
-        waitForAssert(() -> verifyIndexFiles(literalIndexContext, 1, 0, 1));
+        waitForAssert(() -> verifyIndexFiles(numericIndexContext, null, 1, 0, 0, 1, 0));
 
-        rows = executeNet("SELECT id1 FROM %s WHERE v1 = '1'");
+        rows = executeNet("SELECT id1 FROM %s WHERE v1 >= 0");
         assertEquals(0, rows.all().size());
-        verifySSTableIndexes(literalIndexContext.getIndexName(), 1, 0);
+        verifySSTableIndexes(numericIndexContext.getIndexName(), 1, 0);
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java b/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java
index b8049b7..9e348dc 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java
@@ -42,7 +42,7 @@
 public class GroupComponentsTest extends SAITester
 {
     @Test
-    public void testInvalidateWithoutObsolete() throws Throwable
+    public void testInvalidateWithoutObsolete()
     {
         createTable("CREATE TABLE %s (pk int primary key, value text)");
         createIndex("CREATE CUSTOM INDEX ON %s(value) USING 'StorageAttachedIndex'");
@@ -67,7 +67,7 @@
     }
 
     @Test
-    public void getLiveComponentsForEmptyIndex() throws Throwable
+    public void getLiveComponentsForEmptyIndex()
     {
         createTable("CREATE TABLE %s (pk int primary key, value text)");
         createIndex("CREATE CUSTOM INDEX ON %s(value) USING 'StorageAttachedIndex'");
@@ -88,7 +88,7 @@
     }
 
     @Test
-    public void getLiveComponentsForPopulatedIndex() throws Throwable
+    public void getLiveComponentsForPopulatedIndex()
     {
         createTable("CREATE TABLE %s (pk int primary key, value text)");
         IndexContext indexContext = createIndexContext(createIndex("CREATE CUSTOM INDEX ON %s(value) USING 'StorageAttachedIndex'"), UTF8Type.instance);
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/NodeRestartTest.java b/test/unit/org/apache/cassandra/index/sai/functional/NodeRestartTest.java
index 1766b69..8ed9b6b 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/NodeRestartTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/NodeRestartTest.java
@@ -23,6 +23,7 @@
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.index.sai.IndexContext;
 import org.apache.cassandra.index.sai.SAITester;
@@ -92,7 +93,7 @@
         waitForAssert(() -> Assert.assertEquals(1, initTaskLatch.getCount()));
 
         // If we do not make the index queryable before it starts accepting queries, this will fail:
-        assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+        assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
 
         // Allow the init task to run, and then wait for it to finish...
         initTaskLatch.countDown();
@@ -108,19 +109,23 @@
         createTable(CREATE_TABLE_TEMPLATE);
         verifyNoIndexFiles();
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
         flush();
 
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
-        verifyIndexFiles(literalIndexContext, 1);
-        assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
+        waitForTableIndexesQueryable();
+        verifyIndexFiles(numericIndexContext,literalIndexContext, 1, 1);
+        assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
+        assertNumRows(1, "SELECT * FROM %%s WHERE v2 = '0'");
         assertValidationCount(0, 0);
 
         simulateNodeRestart();
 
-        verifyIndexFiles(literalIndexContext, 1);
+        verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 1);
 
-        assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+        assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
+        assertNumRows(1, "SELECT * FROM %%s WHERE v2 = '0'");
 
         waitForTableIndexesQueryable();
 
@@ -158,7 +163,7 @@
 
         // This will fail if the pre-join task doesn't skip validation (after the init task has already run):
         assertValidationCount(0, 0);
-        assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+        assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
     }
 
     void createSingleRowIndex() throws Throwable
@@ -166,12 +171,13 @@
         createTable(CREATE_TABLE_TEMPLATE);
         verifyNoIndexFiles();
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0')");
         flush();
 
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
-        verifyIndexFiles(literalIndexContext, 1);
-        assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        waitForTableIndexesQueryable();
+        verifyIndexFiles(numericIndexContext, null, 1, 0);
+        assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
         assertValidationCount(0, 0);
     }
 }
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/SnapshotTest.java b/test/unit/org/apache/cassandra/index/sai/functional/SnapshotTest.java
index 769435d..cd20955 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/SnapshotTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/SnapshotTest.java
@@ -21,7 +21,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.index.sai.IndexContext;
 import org.apache.cassandra.index.sai.SAITester;
 import org.apache.cassandra.inject.Injections;
@@ -50,60 +50,60 @@
         verifyNoIndexFiles();
 
         // Insert some initial data and create the index over it
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+        execute("INSERT INTO %s (id1, v1) VALUES ('0', 0);");
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        waitForTableIndexesQueryable();
         flush();
-        verifyIndexFiles(literalIndexContext, 1, 1, 1);
+        verifyIndexFiles(numericIndexContext, null, 1, 1, 0, 1, 0);
         // Note: This test will fail here if it is run on its own because the per-index validation
         // is run if the node is starting up but validatation isn't done once the node is started
         assertValidationCount(0, 0);
         resetValidationCount();
 
         // Add some data into a second sstable
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+        execute("INSERT INTO %s (id1, v1) VALUES ('1', 0);");
         flush();
-        verifyIndexFiles(literalIndexContext, 2, 2, 2);
+        verifyIndexFiles(numericIndexContext, null, 2, 2, 0, 2, 0);
         assertValidationCount(0, 0);
 
         // Take a snapshot recording the index files last modified date
         String snapshot = "s";
-        int numSnapshottedSSTables = snapshot(snapshot);
-        assertEquals(1, numSnapshottedSSTables);
+        assertEquals(1, snapshot(snapshot));
         long snapshotLastModified = indexFilesLastModified();
 
-        // File.lastModified result can be truncated one second resolution, which can be less than the index build
+        // File.lastModified result can be truncated one second resolution, which can be lesser than the index build
         // time, so we sleep for that time to guarantee that the modification date any of overridden index file will be
         // different to that of the original file
         Thread.sleep(1000);
 
         // Add some data into a third sstable, out of the scope of our snapshot
-        execute("INSERT INTO %s (id1, v1) VALUES ('2', '0');");
+        execute("INSERT INTO %s (id1, v1) VALUES ('2', 0);");
         flush();
-        verifyIndexFiles(literalIndexContext, 3, 3, 3);
-        assertNumRows(3, "SELECT * FROM %%s WHERE v1 = '0'");
+        verifyIndexFiles(numericIndexContext, null, 3, 3, 0, 3, 0);
+        assertNumRows(3, "SELECT * FROM %%s WHERE v1 >= 0");
         assertValidationCount(0, 0);
 
         // Truncate the table
         truncate(false);
-        waitForAssert(this::verifyNoIndexFiles);
-        assertNumRows(0, "SELECT * FROM %%s WHERE v1 = '0'");
+        waitForAssert(() -> verifyNoIndexFiles());
+        assertNumRows(0, "SELECT * FROM %%s WHERE v1 >= 0");
         assertValidationCount(0, 0);
 
         // Restore the snapshot, only the two first sstables should be restored
         restoreSnapshot(snapshot);
-        verifyIndexFiles(literalIndexContext, 2, 2, 2);
+        verifyIndexFiles(numericIndexContext, null, 2, 2, 0, 2, 0);
         assertEquals(snapshotLastModified, indexFilesLastModified());
-        assertNumRows(2, "SELECT * FROM %%s WHERE v1 = '0'");
+        assertNumRows(2, "SELECT * FROM %%s WHERE v1 >= 0");
         assertValidationCount(2, 2); // newly loaded
 
         // index components are included after restore
         verifyIndexComponentsIncludedInSSTable();
 
         // Rebuild the index to verify that the index files are overridden
-        rebuildIndexes(literalIndexContext.getIndexName());
-        verifyIndexFiles(literalIndexContext, 2);
+        rebuildIndexes(numericIndexContext.getIndexName());
+        verifyIndexFiles(numericIndexContext, null, 2, 0);
         assertNotEquals(snapshotLastModified, indexFilesLastModified());
-        assertNumRows(2, "SELECT * FROM %%s WHERE v1 = '0'");
+        assertNumRows(2, "SELECT * FROM %%s WHERE v1 >= 0");
         assertValidationCount(2, 2); // compaction should not validate
 
         // index components are included after rebuild
@@ -117,19 +117,20 @@
         verifyNoIndexFiles();
 
         // Insert some initial data
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+        execute("INSERT INTO %s (id1, v1) VALUES ('0', 0);");
         flush();
 
         // Add some data into a second sstable
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+        execute("INSERT INTO %s (id1, v1) VALUES ('1', 0);");
         flush();
 
         // index components are not included
         verifyIndexComponentsNotIncludedInSSTable();
 
         // create index
-        IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
-        verifyIndexFiles(literalIndexContext, 2);
+        IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+        waitForTableIndexesQueryable();
+        verifyIndexFiles(numericIndexContext, null, 2, 0);
         assertValidationCount(0, 0);
 
         // index components are included after initial build
@@ -137,8 +138,7 @@
 
         // Take a snapshot recording the index files last modified date
         String snapshot = "s";
-        int numSnapshottedSSTables = snapshot(snapshot);
-        assertEquals(1, numSnapshottedSSTables);
+        assertEquals(1, snapshot(snapshot));
         long snapshotLastModified = indexFilesLastModified();
 
         // File.lastModified result can be truncated one second resolution, which can be lesser than the index build
@@ -148,15 +148,15 @@
 
         // Truncate the table
         truncate(false);
-        waitForAssert(this::verifyNoIndexFiles);
-        assertNumRows(0, "SELECT * FROM %%s WHERE v1 = '0'");
+        waitForAssert(() -> verifyNoIndexFiles());
+        assertNumRows(0, "SELECT * FROM %%s WHERE v1 >= 0");
         assertValidationCount(0, 0);
 
         // Restore the snapshot
         restoreSnapshot(snapshot);
-        verifyIndexFiles(literalIndexContext, 2);
+        verifyIndexFiles(numericIndexContext, null, 2, 0);
         assertEquals(snapshotLastModified, indexFilesLastModified());
-        assertNumRows(2, "SELECT * FROM %%s WHERE v1 = '0'");
+        assertNumRows(2, "SELECT * FROM %%s WHERE v1 >= 0");
         assertValidationCount(2, 2); // newly loaded
 
         // index components are included after restore snapshot
diff --git a/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java b/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java
index 7b28aaf..57ad70e 100644
--- a/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java
@@ -54,7 +54,7 @@
         int sstables = 10;
         for (int i = 0; i < sstables; i++)
         {
-            execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+            execute("INSERT INTO %s (id1, v1) VALUES ('0', 0)");
             flush();
         }
 
diff --git a/test/unit/org/apache/cassandra/index/sai/metrics/SegmentFlushingFailureTester.java b/test/unit/org/apache/cassandra/index/sai/metrics/SegmentFlushingFailureTester.java
index 6bc6c99..378bbd4 100644
--- a/test/unit/org/apache/cassandra/index/sai/metrics/SegmentFlushingFailureTester.java
+++ b/test/unit/org/apache/cassandra/index/sai/metrics/SegmentFlushingFailureTester.java
@@ -97,14 +97,14 @@
 
         assertEquals(expectedBytesLimit(), getSegmentBufferSpaceLimit());
         assertEquals("Segment buffer memory tracker should start at zero!", 0L, getSegmentBufferUsedBytes());
-        assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0')");
         flush();
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '1')");
         flush();
 
-        ResultSet rows = executeNet("SELECT * FROM %s WHERE v1 = '0'");
+        ResultSet rows = executeNet("SELECT * FROM %s WHERE v1 = 0");
         assertEquals(1, rows.all().size());
 
         compact();
@@ -116,9 +116,9 @@
         assertTrue(memoryTrackingCounter.get() > 0);
 
         assertEquals("Global memory tracker should have reverted to zero.", 0L, getSegmentBufferUsedBytes());
-        assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
 
-        rows = executeNet("SELECT * FROM %s WHERE v1 = '0'");
+        rows = executeNet("SELECT * FROM %s WHERE v1 = 0");
         assertEquals(1, rows.all().size());
     }
 
@@ -147,17 +147,17 @@
 
         assertEquals(expectedBytesLimit(), getSegmentBufferSpaceLimit());
         assertEquals("Segment buffer memory tracker should start at zero!", 0L, getSegmentBufferUsedBytes());
-        assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
 
-        execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0')");
         flush();
-        execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
+        execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '1')");
         flush();
 
         // Verify that we abort exactly once and zero the memory tracker:
         verifyCompactionIndexBuilds(1, failure, currentTable());
 
-        String select = String.format("SELECT * FROM %%s WHERE %s = %s", column, "'0'");
+        String select = String.format("SELECT * FROM %%s WHERE %s = %s", column, column.equals("v1") ? "0" : "'0'");
 
         assertThatThrownBy(() -> executeNet(select)).isInstanceOf(ReadFailureException.class);
     }
@@ -172,26 +172,26 @@
 
         assertEquals(expectedBytesLimit(), getSegmentBufferSpaceLimit());
         assertEquals("Segment buffer memory tracker should start at zero!", 0L, getSegmentBufferUsedBytes());
-        assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
+        assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
 
-        execute("INSERT INTO " + KEYSPACE + '.' + table1 + "(id1, v1) VALUES ('0', '0')");
+        execute("INSERT INTO " + KEYSPACE + "." + table1 + "(id1, v1, v2) VALUES ('0', 0, '0')");
         flush(KEYSPACE, table1);
-        execute("INSERT INTO " + KEYSPACE + '.' + table1 + "(id1, v1) VALUES ('1', '1')");
+        execute("INSERT INTO " + KEYSPACE + "." + table1 + "(id1, v1, v2) VALUES ('1', 1, '1')");
         flush(KEYSPACE, table1);
 
-        execute("INSERT INTO " + KEYSPACE + '.' + table2 + "(id1, v1) VALUES ('0', '0')");
+        execute("INSERT INTO " + KEYSPACE + "." + table2 + "(id1, v1, v2) VALUES ('0', 0, '0')");
         flush(KEYSPACE, table2);
-        execute("INSERT INTO " + KEYSPACE + '.' + table2 + "(id1, v1) VALUES ('1', '1')");
+        execute("INSERT INTO " + KEYSPACE + "." + table2 + "(id1, v1, v2) VALUES ('1', 1, '1')");
         flush(KEYSPACE, table2);
 
         // Start compaction against both tables/indexes and verify that they are aborted safely:
         verifyCompactionIndexBuilds(2, segmentFlushFailure, table1, table2);
 
-        assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %s.%s WHERE v1 = '0'", KEYSPACE, table1)))
-                .isInstanceOf(ReadFailureException.class);
+        assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %s WHERE v1 = 0", KEYSPACE + "." + table1)))
+        .isInstanceOf(ReadFailureException.class);
 
-        assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %s.%s WHERE v1 = '0'", KEYSPACE, table2)))
-                .isInstanceOf(ReadFailureException.class);
+        assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %s WHERE v1 = 0", KEYSPACE + "." + table2)))
+        .isInstanceOf(ReadFailureException.class);
     }
 
     private void verifyCompactionIndexBuilds(int aborts, Injection failure, String... tables) throws Throwable
diff --git a/test/unit/org/apache/cassandra/index/sai/utils/SAIRandomizedTester.java b/test/unit/org/apache/cassandra/index/sai/utils/SAIRandomizedTester.java
index f8db85f..636f603 100644
--- a/test/unit/org/apache/cassandra/index/sai/utils/SAIRandomizedTester.java
+++ b/test/unit/org/apache/cassandra/index/sai/utils/SAIRandomizedTester.java
@@ -121,6 +121,11 @@
         return between(min, max - 1);
     }
 
+    public static int between(int min, int max)
+    {
+        return getRandom().nextIntBetween(min, max - 1);
+    }
+
     public static long between(long min, long max)
     {
         return randomLongBetween(min, max);
diff --git a/test/unit/org/apache/cassandra/index/sai/utils/TypeUtilTest.java b/test/unit/org/apache/cassandra/index/sai/utils/TypeUtilTest.java
index 9f91c27..69922ee 100644
--- a/test/unit/org/apache/cassandra/index/sai/utils/TypeUtilTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/utils/TypeUtilTest.java
@@ -63,8 +63,8 @@
             boolean isLiteral = cql3Type == CQL3Type.Native.ASCII || cql3Type == CQL3Type.Native.TEXT || cql3Type == CQL3Type.Native.VARCHAR || cql3Type == CQL3Type.Native.BOOLEAN;
             assertEquals(isLiteral, TypeUtil.isLiteral(type));
             assertEquals(TypeUtil.isLiteral(type), TypeUtil.isLiteral(reversedType));
-            assertEquals(isUTF8OrAscii, TypeUtil.isUTF8OrAscii(type));
-            assertEquals(TypeUtil.isUTF8OrAscii(type), TypeUtil.isUTF8OrAscii(reversedType));
+            assertEquals(isUTF8OrAscii, TypeUtil.isString(type));
+            assertEquals(TypeUtil.isString(type), TypeUtil.isString(reversedType));
         }
     }