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

[cassandra] branch trunk updated: ByteComparable API

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

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


The following commit(s) were added to refs/heads/trunk by this push:
     new e966c45afc ByteComparable API
e966c45afc is described below

commit e966c45afcf8bef47df245ccb851386e5ce60505
Author: jacek-lewandowski <ja...@datastax.com>
AuthorDate: Fri Nov 6 14:59:56 2020 +0100

    ByteComparable API
    
    Provides an API for converting all values of types that can be used in
    primary keys to byte sequences that can be compared lexicographically
    by unsigned byte value (i.e. byte-comparable sequences) and back.
    
    patch by Branimir Lambov, Dimitar Dimitrov and Jacek Lewandowski;
    reviewed by Caleb Rackliffe, Dimitar Dimitrov, Jacek Lewandowski and Aleksey Yeschenko for CASSANDRA-6936
---
 CHANGES.txt                                        |    1 +
 src/java/org/apache/cassandra/cql3/CQL3Type.java   |    2 +-
 src/java/org/apache/cassandra/cql3/Tuples.java     |    7 +-
 src/java/org/apache/cassandra/cql3/UserTypes.java  |    2 +-
 .../cassandra/cql3/conditions/ColumnCondition.java |    4 +-
 .../cassandra/cql3/selection/FieldSelector.java    |    3 +-
 .../apache/cassandra/db/BufferDecoratedKey.java    |   26 +
 .../apache/cassandra/db/ClusteringComparator.java  |  271 +++++
 .../org/apache/cassandra/db/ClusteringPrefix.java  |   57 +-
 src/java/org/apache/cassandra/db/DataRange.java    |   29 +
 src/java/org/apache/cassandra/db/DecoratedKey.java |   72 +-
 .../apache/cassandra/db/NativeDecoratedKey.java    |   36 +-
 .../org/apache/cassandra/db/PartitionPosition.java |   25 +-
 .../org/apache/cassandra/db/SystemKeyspace.java    |    3 +-
 .../db/columniterator/SSTableIterator.java         |    1 +
 .../db/columniterator/SSTableReversedIterator.java |    1 +
 .../cassandra/db/marshal/AbstractTimeUUIDType.java |   69 +-
 .../apache/cassandra/db/marshal/AbstractType.java  |   89 +-
 .../apache/cassandra/db/marshal/BooleanType.java   |   27 +-
 .../cassandra/db/marshal/ByteArrayAccessor.java    |    7 +
 .../db/marshal/ByteArrayObjectFactory.java         |   35 +-
 .../cassandra/db/marshal/ByteBufferAccessor.java   |    7 +
 .../db/marshal/ByteBufferObjectFactory.java        |   25 +-
 .../org/apache/cassandra/db/marshal/ByteType.java  |   17 +
 .../cassandra/db/marshal/CollectionType.java       |   90 ++
 .../apache/cassandra/db/marshal/CompositeType.java |   95 +-
 .../org/apache/cassandra/db/marshal/DateType.java  |   16 +
 .../apache/cassandra/db/marshal/DecimalType.java   |  204 ++++
 .../apache/cassandra/db/marshal/DoubleType.java    |   15 +
 .../cassandra/db/marshal/DynamicCompositeType.java |  207 ++++
 .../org/apache/cassandra/db/marshal/EmptyType.java |   14 +
 .../org/apache/cassandra/db/marshal/FloatType.java |   15 +
 .../org/apache/cassandra/db/marshal/Int32Type.java |   15 +
 .../apache/cassandra/db/marshal/IntegerType.java   |  307 +++++
 .../cassandra/db/marshal/LexicalUUIDType.java      |   43 +
 .../org/apache/cassandra/db/marshal/ListType.java  |   55 +-
 .../org/apache/cassandra/db/marshal/LongType.java  |   25 +
 .../org/apache/cassandra/db/marshal/MapType.java   |   74 +-
 .../db/marshal/PartitionerDefinedOrder.java        |   31 +
 .../apache/cassandra/db/marshal/ReversedType.java  |   62 ++
 .../org/apache/cassandra/db/marshal/SetType.java   |   18 +-
 .../org/apache/cassandra/db/marshal/ShortType.java |   16 +
 .../cassandra/db/marshal/SimpleDateType.java       |   18 +
 .../org/apache/cassandra/db/marshal/TimeType.java  |   18 +
 .../apache/cassandra/db/marshal/TimestampType.java |   15 +
 .../org/apache/cassandra/db/marshal/TupleType.java |  126 ++-
 .../org/apache/cassandra/db/marshal/UUIDType.java  |   65 ++
 .../org/apache/cassandra/db/marshal/UserType.java  |    2 +-
 .../apache/cassandra/db/marshal/ValueAccessor.java |    8 +-
 .../apache/cassandra/db/rows/EncodingStats.java    |    2 +-
 .../cassandra/dht/ByteOrderedPartitioner.java      |   14 +
 .../org/apache/cassandra/dht/LocalPartitioner.java |   15 +
 .../apache/cassandra/dht/Murmur3Partitioner.java   |   15 +
 .../cassandra/dht/OrderPreservingPartitioner.java  |   14 +
 .../apache/cassandra/dht/RandomPartitioner.java    |   15 +
 src/java/org/apache/cassandra/dht/Token.java       |   55 +-
 .../cassandra/serializers/BooleanSerializer.java   |    4 +-
 .../serializers/CollectionSerializer.java          |    5 -
 .../cassandra/serializers/MapSerializer.java       |    4 +-
 .../cassandra/serializers/SetSerializer.java       |    4 +-
 .../service/paxos/PaxosRepairHistory.java          |    3 +-
 .../utils/bytecomparable/ByteComparable.java       |  163 +++
 .../utils/bytecomparable/ByteComparable.md         |  693 ++++++++++++
 .../cassandra/utils/bytecomparable/ByteSource.java |  853 ++++++++++++++
 .../utils/bytecomparable/ByteSourceInverse.java    |  471 ++++++++
 .../AbstractTypeByteSourceDecodingBench.java       |  140 +++
 test/unit/org/apache/cassandra/Util.java           |  194 ++++
 .../cql3/validation/entities/TupleTypeTest.java    |    6 +-
 .../cql3/validation/entities/UserTypesTest.java    |   50 +
 .../db/marshal/DynamicCompositeTypeTest.java       |    9 +-
 .../cassandra/db/marshal/TypeValidationTest.java   |    2 +-
 .../org/apache/cassandra/dht/KeyCollisionTest.java |    9 +
 .../apache/cassandra/dht/LengthPartitioner.java    |    7 +
 .../apache/cassandra/transport/SerDeserTest.java   |    2 +-
 .../bytecomparable/AbstractTypeByteSourceTest.java | 1015 +++++++++++++++++
 .../bytecomparable/ByteSourceComparisonTest.java   | 1178 ++++++++++++++++++++
 .../bytecomparable/ByteSourceConversionTest.java   |  784 +++++++++++++
 .../bytecomparable/ByteSourceInverseTest.java      |  397 +++++++
 .../bytecomparable/ByteSourceSequenceTest.java     |  784 +++++++++++++
 .../utils/bytecomparable/ByteSourceTestBase.java   |  513 +++++++++
 .../bytecomparable/DecoratedKeyByteSourceTest.java |   85 ++
 81 files changed, 9730 insertions(+), 145 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 6a7eae004d..fc0e2dc864 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.2
+ * Add ByteComparable API (CASSANDRA-6936)
  * Add guardrail for maximum replication factor (CASSANDRA-17500)
  * Increment CQLSH to version 6.2.0 for release 4.2 (CASSANDRA-17646)
  * Adding support to perform certificate based internode authentication (CASSANDRA-17661)
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java
index 1d792b2453..1c20e6b0ff 100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Type.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java
@@ -201,7 +201,7 @@ public interface CQL3Type
 
             StringBuilder target = new StringBuilder();
             buffer = buffer.duplicate();
-            int size = CollectionSerializer.readCollectionSize(buffer, version);
+            int size = CollectionSerializer.readCollectionSize(buffer, ByteBufferAccessor.instance, version);
             buffer.position(buffer.position() + CollectionSerializer.sizeOfCollectionSize(size, version));
 
             switch (type.kind)
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index b8acd5954a..6e028c274d 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -154,14 +154,14 @@ public class Tuples
 
         public static Value fromSerialized(ByteBuffer bytes, TupleType type)
         {
-            ByteBuffer[] values = type.split(bytes);
+            ByteBuffer[] values = type.split(ByteBufferAccessor.instance, bytes);
             if (values.length > type.size())
             {
                 throw new InvalidRequestException(String.format(
                         "Tuple value contained too many fields (expected %s, got %s)", type.size(), values.length));
             }
 
-            return new Value(type.split(bytes));
+            return new Value(type.split(ByteBufferAccessor.instance, bytes));
         }
 
         public ByteBuffer get(ProtocolVersion protocolVersion)
@@ -272,7 +272,8 @@ public class Tuples
                 // type.split(bytes)
                 List<List<ByteBuffer>> elements = new ArrayList<>(l.size());
                 for (Object element : l)
-                    elements.add(Arrays.asList(tupleType.split(type.getElementsType().decompose(element))));
+                    elements.add(Arrays.asList(tupleType.split(ByteBufferAccessor.instance,
+                                                               type.getElementsType().decompose(element))));
                 return new InValue(elements);
             }
             catch (MarshalException e)
diff --git a/src/java/org/apache/cassandra/cql3/UserTypes.java b/src/java/org/apache/cassandra/cql3/UserTypes.java
index b023a8a0b8..a63420fca3 100644
--- a/src/java/org/apache/cassandra/cql3/UserTypes.java
+++ b/src/java/org/apache/cassandra/cql3/UserTypes.java
@@ -217,7 +217,7 @@ public abstract class UserTypes
         public static Value fromSerialized(ByteBuffer bytes, UserType type)
         {
             type.validate(bytes);
-            return new Value(type, type.split(bytes));
+            return new Value(type, type.split(ByteBufferAccessor.instance, bytes));
         }
 
         public ByteBuffer get(ProtocolVersion protocolVersion)
diff --git a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
index e3f463a255..68cf2d3782 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
@@ -650,8 +650,8 @@ public abstract class ColumnCondition
 
             Cell<?> cell = getCell(row, column);
             return cell == null
-                      ? null
-                      : userType.split(cell.buffer())[userType.fieldPosition(field)];
+                   ? null
+                   : userType.split(ByteBufferAccessor.instance, cell.buffer())[userType.fieldPosition(field)];
         }
 
         private boolean isSatisfiedBy(ByteBuffer rowValue)
diff --git a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
index 0c62397681..ddcc868cf3 100644
--- a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
@@ -27,6 +27,7 @@ import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -108,7 +109,7 @@ final class FieldSelector extends Selector
         ByteBuffer value = selected.getOutput(protocolVersion);
         if (value == null)
             return null;
-        ByteBuffer[] buffers = type.split(value);
+        ByteBuffer[] buffers = type.split(ByteBufferAccessor.instance, value);
         return field < buffers.length ? buffers[field] : null;
     }
 
diff --git a/src/java/org/apache/cassandra/db/BufferDecoratedKey.java b/src/java/org/apache/cassandra/db/BufferDecoratedKey.java
index d375162240..ae3e9d44e0 100644
--- a/src/java/org/apache/cassandra/db/BufferDecoratedKey.java
+++ b/src/java/org/apache/cassandra/db/BufferDecoratedKey.java
@@ -19,7 +19,9 @@ package org.apache.cassandra.db;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 
 public class BufferDecoratedKey extends DecoratedKey
 {
@@ -36,4 +38,28 @@ public class BufferDecoratedKey extends DecoratedKey
     {
         return key;
     }
+
+    /**
+     * A factory method that translates the given byte-comparable representation to a {@link BufferDecoratedKey}
+     * instance. If the given byte comparable doesn't represent the encoding of a buffer decorated key, anything from a
+     * wide variety of throwables may be thrown (e.g. {@link AssertionError}, {@link IndexOutOfBoundsException},
+     * {@link IllegalStateException}, etc.).
+     *
+     * @param byteComparable A byte-comparable representation (presumably of a {@link BufferDecoratedKey} instance).
+     * @param version The encoding version used for the given byte comparable.
+     * @param partitioner The partitioner of the encoded decorated key. Needed in order to correctly decode the token
+     *                    bytes of the key.
+     * @return A new {@link BufferDecoratedKey} instance, corresponding to the given byte-comparable representation. If
+     * we were to call {@link #asComparableBytes(Version)} on the returned object, we should get a {@link ByteSource}
+     * equal to the one of the input byte comparable.
+     */
+    public static BufferDecoratedKey fromByteComparable(ByteComparable byteComparable,
+                                                        Version version,
+                                                        IPartitioner partitioner)
+    {
+        return DecoratedKey.fromByteComparable(byteComparable,
+                                               version,
+                                               partitioner,
+                                               (token, keyBytes) -> new BufferDecoratedKey(token, ByteBuffer.wrap(keyBytes)));
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/ClusteringComparator.java b/src/java/org/apache/cassandra/db/ClusteringComparator.java
index fdc450813f..c1aebfad77 100644
--- a/src/java/org/apache/cassandra/db/ClusteringComparator.java
+++ b/src/java/org/apache/cassandra/db/ClusteringComparator.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.db;
 
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Objects;
@@ -31,6 +32,15 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.serializers.MarshalException;
 
 import org.apache.cassandra.io.sstable.IndexInfo;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+import static org.apache.cassandra.utils.bytecomparable.ByteSource.EXCLUDED;
+import static org.apache.cassandra.utils.bytecomparable.ByteSource.NEXT_COMPONENT;
+import static org.apache.cassandra.utils.bytecomparable.ByteSource.NEXT_COMPONENT_EMPTY;
+import static org.apache.cassandra.utils.bytecomparable.ByteSource.NEXT_COMPONENT_EMPTY_REVERSED;
+import static org.apache.cassandra.utils.bytecomparable.ByteSource.NEXT_COMPONENT_NULL;
+import static org.apache.cassandra.utils.bytecomparable.ByteSource.TERMINATOR;
 
 /**
  * A comparator of clustering prefixes (or more generally of {@link Clusterable}}.
@@ -232,6 +242,267 @@ public class ClusteringComparator implements Comparator<Clusterable>
         }
     }
 
+    /**
+     * Produce a prefix-free byte-comparable representation of the given value, i.e. such a sequence of bytes that any
+     * pair x, y of valid values of this type
+     *   compare(x, y) == compareLexicographicallyUnsigned(asByteComparable(x), asByteComparable(y))
+     * and
+     *   asByteComparable(x) is not a prefix of asByteComparable(y)
+     */
+    public <V> ByteComparable asByteComparable(ClusteringPrefix<V> clustering)
+    {
+        return new ByteComparableClustering<>(clustering);
+    }
+
+    /**
+     * A prefix-free byte-comparable representation for a clustering or prefix.
+     *
+     * Adds a NEXT_COMPONENT byte before each component (allowing inclusive/exclusive bounds over incomplete prefixes
+     * of that length) and finishes with a suitable byte for the clustering kind. Also deals with null entries.
+     *
+     * Since all types' encodings are weakly prefix-free, this is guaranteed to be prefix-free as long as the
+     * bound/ClusteringPrefix terminators are different from the separator byte. It is okay for the terminator for
+     * Clustering to be the same as the separator, as all Clusterings must be completely specified.
+     *
+     * See also {@link AbstractType#asComparableBytes}.
+     *
+     * Some examples:
+     *    "A", 0005, Clustering     -> 40 4100 40 0005 40
+     *    "B", 0006, InclusiveEnd   -> 40 4200 40 0006 60
+     *    "A", ExclusiveStart       -> 40 4100 60
+     *    "", null, Clustering      -> 40 00 3F 40
+     *    "", 0000, Clustering      -> 40 00 40 0000 40
+     *    BOTTOM                    -> 20
+     */
+    private class ByteComparableClustering<V> implements ByteComparable
+    {
+        private final ClusteringPrefix<V> src;
+
+        ByteComparableClustering(ClusteringPrefix<V> src)
+        {
+            this.src = src;
+        }
+
+        @Override
+        public ByteSource asComparableBytes(Version version)
+        {
+            return new ByteSource()
+            {
+                private ByteSource current = null;
+                private int srcnum = -1;
+
+                @Override
+                public int next()
+                {
+                    if (current != null)
+                    {
+                        int b = current.next();
+                        if (b > END_OF_STREAM)
+                            return b;
+                        current = null;
+                    }
+
+                    int sz = src.size();
+                    if (srcnum == sz)
+                        return END_OF_STREAM;
+
+                    ++srcnum;
+                    if (srcnum == sz)
+                        return src.kind().asByteComparableValue(version);
+
+                    final V nextComponent = src.get(srcnum);
+                    // We can have a null as the clustering component (this is a relic of COMPACT STORAGE, but also
+                    // can appear in indexed partitions with no rows but static content),
+                    if (nextComponent == null)
+                    {
+                        if (version != Version.LEGACY)
+                            return NEXT_COMPONENT_NULL; // always sorts before non-nulls, including for reversed types
+                        else
+                        {
+                            // legacy version did not permit nulls in clustering keys and treated these as null values
+                            return subtype(srcnum).isReversed() ? NEXT_COMPONENT_EMPTY_REVERSED : NEXT_COMPONENT_EMPTY;
+                        }
+                    }
+
+                    current = subtype(srcnum).asComparableBytes(src.accessor(), nextComponent, version);
+                    // and also null values for some types (e.g. int, varint but not text) that are encoded as empty
+                    // buffers.
+                    if (current == null)
+                        return subtype(srcnum).isReversed() ? NEXT_COMPONENT_EMPTY_REVERSED : NEXT_COMPONENT_EMPTY;
+
+                    return NEXT_COMPONENT;
+                }
+            };
+        }
+
+        public String toString()
+        {
+            return src.clusteringString(subtypes());
+        }
+    }
+
+    /**
+     * Produces a clustering from the given byte-comparable value. The method will throw an exception if the value
+     * does not correctly encode a clustering of this type, including if it encodes a position before or after a
+     * clustering (i.e. a bound/boundary).
+     *
+     * @param accessor Accessor to use to construct components.
+     * @param comparable The clustering encoded as a byte-comparable sequence.
+     */
+    public <V> Clustering<V> clusteringFromByteComparable(ValueAccessor<V> accessor, ByteComparable comparable)
+    {
+        ByteComparable.Version version = ByteComparable.Version.OSS42;
+        ByteSource.Peekable orderedBytes = ByteSource.peekable(comparable.asComparableBytes(version));
+
+        // First check for special cases (partition key only, static clustering) that can do without buffers.
+        int sep = orderedBytes.next();
+        switch (sep)
+        {
+        case TERMINATOR:
+            assert size() == 0 : "Terminator should be after " + size() + " components, got 0";
+            return accessor.factory().clustering();
+        case EXCLUDED:
+            return accessor.factory().staticClustering();
+        default:
+            // continue with processing
+        }
+
+        int cc = 0;
+        V[] components = accessor.createArray(size());
+
+        while (true)
+        {
+            switch (sep)
+            {
+            case NEXT_COMPONENT_NULL:
+                components[cc] = null;
+                break;
+            case NEXT_COMPONENT_EMPTY:
+            case NEXT_COMPONENT_EMPTY_REVERSED:
+                components[cc] = subtype(cc).fromComparableBytes(accessor, null, version);
+                break;
+            case NEXT_COMPONENT:
+                // Decode the next component, consuming bytes from orderedBytes.
+                components[cc] = subtype(cc).fromComparableBytes(accessor, orderedBytes, version);
+                break;
+            case TERMINATOR:
+                assert cc == size() : "Terminator should be after " + size() + " components, got " + cc;
+                return accessor.factory().clustering(components);
+            case EXCLUDED:
+                throw new AssertionError("Unexpected static terminator after the first component");
+            default:
+                throw new AssertionError("Unexpected separator " + Integer.toHexString(sep) + " in Clustering encoding");
+            }
+            ++cc;
+            sep = orderedBytes.next();
+        }
+    }
+
+    /**
+     * Produces a clustering bound from the given byte-comparable value. The method will throw an exception if the value
+     * does not correctly encode a bound position of this type, including if it encodes an exact clustering.
+     *
+     * Note that the encoded clustering position cannot specify the type of bound (i.e. start/end/boundary) because to
+     * correctly compare clustering positions the encoding must be the same for the different types (e.g. the position
+     * for a exclusive end and an inclusive start is the same, before the exact clustering). The type must be supplied
+     * separately (in the bound... vs boundary... call and isEnd argument).
+     *
+     * @param accessor Accessor to use to construct components.
+     * @param comparable The clustering position encoded as a byte-comparable sequence.
+     * @param isEnd true if the bound marks the end of a range, false is it marks the start.
+     */
+    public <V> ClusteringBound<V> boundFromByteComparable(ValueAccessor<V> accessor,
+                                                          ByteComparable comparable,
+                                                          boolean isEnd)
+    {
+        ByteComparable.Version version = ByteComparable.Version.OSS42;
+        ByteSource.Peekable orderedBytes = ByteSource.peekable(comparable.asComparableBytes(version));
+
+        int sep = orderedBytes.next();
+        int cc = 0;
+        V[] components = accessor.createArray(size());
+
+        while (true)
+        {
+            switch (sep)
+            {
+            case NEXT_COMPONENT_NULL:
+                components[cc] = null;
+                break;
+            case NEXT_COMPONENT_EMPTY:
+            case NEXT_COMPONENT_EMPTY_REVERSED:
+                components[cc] = subtype(cc).fromComparableBytes(accessor, null, version);
+                break;
+            case NEXT_COMPONENT:
+                // Decode the next component, consuming bytes from orderedBytes.
+                components[cc] = subtype(cc).fromComparableBytes(accessor, orderedBytes, version);
+                break;
+            case ByteSource.LT_NEXT_COMPONENT:
+                return accessor.factory().bound(isEnd ? ClusteringPrefix.Kind.EXCL_END_BOUND
+                                                      : ClusteringPrefix.Kind.INCL_START_BOUND,
+                                                Arrays.copyOf(components, cc));
+            case ByteSource.GT_NEXT_COMPONENT:
+                return accessor.factory().bound(isEnd ? ClusteringPrefix.Kind.INCL_END_BOUND
+                                                      : ClusteringPrefix.Kind.EXCL_START_BOUND,
+                                                Arrays.copyOf(components, cc));
+            default:
+                throw new AssertionError("Unexpected separator " + Integer.toHexString(sep) + " in ClusteringBound encoding");
+            }
+            ++cc;
+            sep = orderedBytes.next();
+        }
+    }
+
+    /**
+     * Produces a clustering boundary from the given byte-comparable value. The method will throw an exception if the
+     * value does not correctly encode a bound position of this type, including if it encodes an exact clustering.
+     *
+     * Note that the encoded clustering position cannot specify the type of bound (i.e. start/end/boundary) because to
+     * correctly compare clustering positions the encoding must be the same for the different types (e.g. the position
+     * for a exclusive end and an inclusive start is the same, before the exact clustering). The type must be supplied
+     * separately (in the bound... vs boundary... call and isEnd argument).
+     *
+     * @param accessor Accessor to use to construct components.
+     * @param comparable The clustering position encoded as a byte-comparable sequence.
+     */
+    public <V> ClusteringBoundary<V> boundaryFromByteComparable(ValueAccessor<V> accessor, ByteComparable comparable)
+    {
+        ByteComparable.Version version = ByteComparable.Version.OSS42;
+        ByteSource.Peekable orderedBytes = ByteSource.peekable(comparable.asComparableBytes(version));
+
+        int sep = orderedBytes.next();
+        int cc = 0;
+        V[] components = accessor.createArray(size());
+
+        while (true)
+        {
+            switch (sep)
+            {
+            case NEXT_COMPONENT_NULL:
+                components[cc] = null;
+                break;
+            case NEXT_COMPONENT_EMPTY:
+            case NEXT_COMPONENT_EMPTY_REVERSED:
+                components[cc] = subtype(cc).fromComparableBytes(accessor, null, version);
+                break;
+            case NEXT_COMPONENT:
+                // Decode the next component, consuming bytes from orderedBytes.
+                components[cc] = subtype(cc).fromComparableBytes(accessor, orderedBytes, version);
+                break;
+            case ByteSource.LT_NEXT_COMPONENT:
+                return accessor.factory().boundary(ClusteringPrefix.Kind.EXCL_END_INCL_START_BOUNDARY,
+                                                   Arrays.copyOf(components, cc));
+            case ByteSource.GT_NEXT_COMPONENT:
+                return accessor.factory().boundary(ClusteringPrefix.Kind.INCL_END_EXCL_START_BOUNDARY,
+                                                   Arrays.copyOf(components, cc));
+            default:
+                throw new AssertionError("Unexpected separator " + Integer.toHexString(sep) + " in ClusteringBoundary encoding");
+            }
+            ++cc;
+            sep = orderedBytes.next();
+        }
+    }
+
     /**
      * A comparator for rows.
      *
diff --git a/src/java/org/apache/cassandra/db/ClusteringPrefix.java b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
index a1291c889f..c7a2782ece 100644
--- a/src/java/org/apache/cassandra/db/ClusteringPrefix.java
+++ b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
+import java.util.function.ToIntFunction;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.config.*;
@@ -34,6 +35,8 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteArrayUtil;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 
 /**
  * A clustering prefix is the unit of what a {@link ClusteringComparator} can compare.
@@ -62,14 +65,19 @@ public interface ClusteringPrefix<V> extends IMeasurableMemory, Clusterable<V>
     {
         // WARNING: the ordering of that enum matters because we use ordinal() in the serialization
 
-        EXCL_END_BOUND              (0, -1),
-        INCL_START_BOUND            (0, -1),
-        EXCL_END_INCL_START_BOUNDARY(0, -1),
-        STATIC_CLUSTERING           (1, -1),
-        CLUSTERING                  (2,  0),
-        INCL_END_EXCL_START_BOUNDARY(3,  1),
-        INCL_END_BOUND              (3,  1),
-        EXCL_START_BOUND            (3,  1);
+        EXCL_END_BOUND              (0, -1, v -> ByteSource.LT_NEXT_COMPONENT),
+        INCL_START_BOUND            (0, -1, v -> ByteSource.LT_NEXT_COMPONENT),
+        EXCL_END_INCL_START_BOUNDARY(0, -1, v -> ByteSource.LT_NEXT_COMPONENT),
+        STATIC_CLUSTERING           (1, -1, v -> v == Version.LEGACY
+                                                 ? ByteSource.LT_NEXT_COMPONENT + 1
+                                                 : ByteSource.EXCLUDED),
+        CLUSTERING                  (2,  0, v -> v == Version.LEGACY
+                                                 ? ByteSource.NEXT_COMPONENT
+                                                 : ByteSource.TERMINATOR),
+        INCL_END_EXCL_START_BOUNDARY(3,  1, v -> ByteSource.GT_NEXT_COMPONENT),
+        INCL_END_BOUND              (3,  1, v -> ByteSource.GT_NEXT_COMPONENT),
+        EXCL_START_BOUND            (3,  1, v -> ByteSource.GT_NEXT_COMPONENT);
+
 
         private final int comparison;
 
@@ -79,10 +87,13 @@ public interface ClusteringPrefix<V> extends IMeasurableMemory, Clusterable<V>
          */
         public final int comparedToClustering;
 
-        Kind(int comparison, int comparedToClustering)
+        public final ToIntFunction<Version> asByteComparable;
+
+        Kind(int comparison, int comparedToClustering, ToIntFunction<Version> asByteComparable)
         {
             this.comparison = comparison;
             this.comparedToClustering = comparedToClustering;
+            this.asByteComparable = asByteComparable;
         }
 
         /**
@@ -197,6 +208,16 @@ public interface ClusteringPrefix<V> extends IMeasurableMemory, Clusterable<V>
                  ? (this == INCL_END_EXCL_START_BOUNDARY ? INCL_END_BOUND : EXCL_END_BOUND)
                  : (this == INCL_END_EXCL_START_BOUNDARY ? EXCL_START_BOUND : INCL_START_BOUND);
         }
+
+        /*
+         * Returns a terminator value for this clustering type that is suitable for byte comparison.
+         * Inclusive starts / exclusive ends need a lower value than ByteSource.NEXT_COMPONENT and the clustering byte,
+         * exclusive starts / inclusive ends -- a higher.
+         */
+        public int asByteComparableValue(Version version)
+        {
+            return asByteComparable.applyAsInt(version);
+        }
     }
 
     default boolean isBottom()
@@ -308,6 +329,24 @@ public interface ClusteringPrefix<V> extends IMeasurableMemory, Clusterable<V>
             values[i] = accessor().toBuffer(get(i));
         return CompositeType.build(ByteBufferAccessor.instance, values);
     }
+
+    /**
+     * Produce a human-readable representation of the clustering given the list of types.
+     * Easier to access than metadata for debugging.
+     */
+    public default String clusteringString(List<AbstractType<?>> types)
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(kind()).append('(');
+        for (int i = 0; i < size(); i++)
+        {
+            if (i > 0)
+                sb.append(", ");
+            sb.append(types.get(i).getString(get(i), accessor()));
+        }
+        return sb.append(')').toString();
+    }
+
     /**
      * The values of this prefix as an array.
      * <p>
diff --git a/src/java/org/apache/cassandra/db/DataRange.java b/src/java/org/apache/cassandra/db/DataRange.java
index 52162be72a..9912ac56e9 100644
--- a/src/java/org/apache/cassandra/db/DataRange.java
+++ b/src/java/org/apache/cassandra/db/DataRange.java
@@ -27,6 +27,7 @@ import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 
 /**
  * Groups both the range of partitions to query, and the clustering index filter to
@@ -138,6 +139,34 @@ public class DataRange
         return keyRange.right;
     }
 
+    /**
+     * The start of the partition key range queried by this {@code DataRange}.
+     *
+     * @return the start of the partition key range expressed as a ByteComparable.
+     */
+    public ByteComparable startAsByteComparable()
+    {
+        PartitionPosition bound = keyRange.left;
+        if (bound.isMinimum())
+            return null;
+
+        return bound.asComparableBound(keyRange.inclusiveLeft());
+    }
+
+    /**
+     * The end of the partition key range queried by this {@code DataRange}.
+     *
+     * @return the end of the partition key range expressed as a ByteComparable.
+     */
+    public ByteComparable stopAsByteComparable()
+    {
+        PartitionPosition bound = keyRange.right;
+        if (bound.isMinimum())
+            return null;
+
+        return bound.asComparableBound(!keyRange.inclusiveRight());
+    }
+
     /**
      * Whether the underlying clustering index filter is a names filter or not.
      *
diff --git a/src/java/org/apache/cassandra/db/DecoratedKey.java b/src/java/org/apache/cassandra/db/DecoratedKey.java
index 4dd87d0e2c..569c86d9d8 100644
--- a/src/java/org/apache/cassandra/db/DecoratedKey.java
+++ b/src/java/org/apache/cassandra/db/DecoratedKey.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.Comparator;
 import java.util.List;
 import java.util.StringJoiner;
+import java.util.function.BiFunction;
 
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.dht.IPartitioner;
@@ -29,8 +30,11 @@ import org.apache.cassandra.dht.Token.KeyBound;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.MurmurHash;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 import org.apache.cassandra.utils.IFilter.FilterKey;
+import org.apache.cassandra.utils.MurmurHash;
 
 /**
  * Represents a decorated key, handy for certain operations
@@ -102,6 +106,37 @@ public abstract class DecoratedKey implements PartitionPosition, FilterKey
         return cmp == 0 ? ByteBufferUtil.compareUnsigned(key, otherKey.getKey()) : cmp;
     }
 
+    @Override
+    public ByteSource asComparableBytes(Version version)
+    {
+        // Note: In the legacy version one encoding could be a prefix of another as the escaping is only weakly
+        // prefix-free (see ByteSourceTest.testDecoratedKeyPrefixes()).
+        // The OSS42 version avoids this by adding a terminator.
+        return ByteSource.withTerminatorMaybeLegacy(version,
+                                                    ByteSource.END_OF_STREAM,
+                                                    token.asComparableBytes(version),
+                                                    keyComparableBytes(version));
+    }
+
+    @Override
+    public ByteComparable asComparableBound(boolean before)
+    {
+        return version ->
+        {
+            assert (version != Version.LEGACY) : "Decorated key bounds are not supported by the legacy encoding.";
+
+            return ByteSource.withTerminator(
+                    before ? ByteSource.LT_NEXT_COMPONENT : ByteSource.GT_NEXT_COMPONENT,
+                    token.asComparableBytes(version),
+                    keyComparableBytes(version));
+        };
+    }
+
+    protected ByteSource keyComparableBytes(Version version)
+    {
+        return ByteSource.of(getKey(), version);
+    }
+
     public IPartitioner getPartitioner()
     {
         return getToken().getPartitioner();
@@ -169,4 +204,39 @@ public abstract class DecoratedKey implements PartitionPosition, FilterKey
         ByteBuffer key = getKey();
         MurmurHash.hash3_x64_128(key, key.position(), key.remaining(), 0, dest);
     }
+
+    /**
+     * A template factory method for creating decorated keys from their byte-comparable representation.
+     */
+    static <T extends DecoratedKey> T fromByteComparable(ByteComparable byteComparable,
+                                                         Version version,
+                                                         IPartitioner partitioner,
+                                                         BiFunction<Token, byte[], T> decoratedKeyFactory)
+    {
+        ByteSource.Peekable peekable = ByteSource.peekable(byteComparable.asComparableBytes(version));
+        // Decode the token from the first component of the multi-component sequence representing the whole decorated key.
+        Token token = partitioner.getTokenFactory().fromComparableBytes(ByteSourceInverse.nextComponentSource(peekable), version);
+        // Decode the key bytes from the second component.
+        byte[] keyBytes = ByteSourceInverse.getUnescapedBytes(ByteSourceInverse.nextComponentSource(peekable));
+        // Consume the terminator byte.
+        int terminator = peekable.next();
+        assert terminator == ByteSource.TERMINATOR : "Decorated key encoding must end in terminator.";
+        // Instantiate a decorated key from the decoded token and key bytes, using the provided factory method.
+        return decoratedKeyFactory.apply(token, keyBytes);
+    }
+
+    public static byte[] keyFromByteSource(ByteSource.Peekable peekableByteSource,
+                                           Version version,
+                                           IPartitioner partitioner)
+    {
+        assert version != Version.LEGACY;   // reverse translation is not supported for LEGACY version.
+        // Decode the token from the first component of the multi-component sequence representing the whole decorated key.
+        // We won't use it, but the decoding also positions the byte source after it.
+        partitioner.getTokenFactory().fromComparableBytes(ByteSourceInverse.nextComponentSource(peekableByteSource), version);
+        // Decode the key bytes from the second component.
+        byte[] keyBytes = ByteSourceInverse.getUnescapedBytes(ByteSourceInverse.nextComponentSource(peekableByteSource));
+        int terminator = peekableByteSource.next();
+        assert terminator == ByteSource.TERMINATOR : "Decorated key encoding must end in terminator.";
+        return keyBytes;
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
index add5218977..e9a564a509 100644
--- a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
+++ b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
@@ -20,7 +20,11 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
+import net.nicoulaj.compilecommand.annotations.Inline;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.MemoryUtil;
 import org.apache.cassandra.utils.memory.NativeAllocator;
@@ -41,8 +45,38 @@ public class NativeDecoratedKey extends DecoratedKey
         MemoryUtil.setBytes(peer + 4, key);
     }
 
+    public NativeDecoratedKey(Token token, NativeAllocator allocator, OpOrder.Group writeOp, byte[] keyBytes)
+    {
+        super(token);
+        assert keyBytes != null;
+
+        int size = keyBytes.length;
+        this.peer = allocator.allocate(4 + size, writeOp);
+        MemoryUtil.setInt(peer, size);
+        MemoryUtil.setBytes(peer + 4, keyBytes, 0, size);
+    }
+
+    @Inline
+    int length()
+    {
+        return MemoryUtil.getInt(peer);
+    }
+
+    @Inline
+    long address()
+    {
+        return this.peer + 4;
+    }
+
+    @Override
     public ByteBuffer getKey()
     {
-        return MemoryUtil.getByteBuffer(peer + 4, MemoryUtil.getInt(peer), ByteOrder.BIG_ENDIAN);
+        return MemoryUtil.getByteBuffer(address(), length(), ByteOrder.BIG_ENDIAN);
+    }
+
+    @Override
+    protected ByteSource keyComparableBytes(Version version)
+    {
+        return ByteSource.ofMemory(address(), length(), version);
     }
 }
diff --git a/src/java/org/apache/cassandra/db/PartitionPosition.java b/src/java/org/apache/cassandra/db/PartitionPosition.java
index 3b45c6c0e2..5e1d6184e2 100644
--- a/src/java/org/apache/cassandra/db/PartitionPosition.java
+++ b/src/java/org/apache/cassandra/db/PartitionPosition.java
@@ -24,8 +24,10 @@ import java.nio.ByteBuffer;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 
-public interface PartitionPosition extends RingPosition<PartitionPosition>
+public interface PartitionPosition extends RingPosition<PartitionPosition>, ByteComparable
 {
     public static enum Kind
     {
@@ -54,6 +56,27 @@ public interface PartitionPosition extends RingPosition<PartitionPosition>
     public Kind kind();
     public boolean isMinimum();
 
+    /**
+     * Produce a prefix-free byte-comparable representation of the key, i.e. such a sequence of bytes that any pair x, y
+     * of valid positions (with the same key column types and partitioner),
+     *   x.compareTo(y) == compareLexicographicallyUnsigned(x.asComparableBytes(), y.asComparableBytes())
+     * and
+     *   x.asComparableBytes() is not a prefix of y.asComparableBytes()
+     *
+     * We use a two-component tuple for decorated keys, and a one-component tuple for key bounds, where the terminator
+     * byte is chosen to yield the correct comparison result. No decorated key can be a prefix of another (per the tuple
+     * encoding), and no key bound can be a prefix of one because it uses a terminator byte that is different from the
+     * tuple separator.
+     */
+    public abstract ByteSource asComparableBytes(Version version);
+
+    /**
+     * Produce a byte-comparable representation for the position before or after the key.
+     * This does nothing for token boundaries (which are already at a position between valid keys), and changes
+     * the terminator byte for keys.
+     */
+    public abstract ByteComparable asComparableBound(boolean before);
+
     public static class RowPositionSerializer implements IPartitionerDependentSerializer<PartitionPosition>
     {
         /*
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 6fbbc3e621..a1013e7955 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -65,6 +65,7 @@ import org.apache.cassandra.cql3.functions.UuidFcts;
 import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.commitlog.CommitLogPosition;
 import org.apache.cassandra.db.compaction.CompactionHistoryTabularData;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
@@ -1875,7 +1876,7 @@ public final class SystemKeyspace
             TupleType tupleType = new TupleType(Lists.newArrayList(UTF8Type.instance, LongType.instance));
             for (ByteBuffer bb : top)
             {
-                ByteBuffer[] components = tupleType.split(bb);
+                ByteBuffer[] components = tupleType.split(ByteBufferAccessor.instance, bb);
                 String keyStr = UTF8Type.instance.compose(components[0]);
                 long value = LongType.instance.compose(components[1]);
                 topPartitions.add(new TopPartitionTracker.TopPartition(metadata.partitioner.decorateKey(metadata.partitionKeyType.fromString(keyStr)), value));
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
index d4362f775b..ddcbcf929b 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
@@ -48,6 +48,7 @@ public class SSTableIterator extends AbstractSSTableIterator
         super(sstable, file, key, indexEntry, slices, columns, ifile);
     }
 
+    @SuppressWarnings("resource") // caller to close
     protected Reader createReaderInternal(RowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
     {
         return indexEntry.isIndexed()
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index a60aafa771..37db6d9754 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -52,6 +52,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         super(sstable, file, key, indexEntry, slices, columns, ifile);
     }
 
+    @SuppressWarnings("resource") // caller to close
     protected Reader createReaderInternal(RowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
     {
         return indexEntry.isIndexed()
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractTimeUUIDType.java b/src/java/org/apache/cassandra/db/marshal/AbstractTimeUUIDType.java
index b6b10d582d..38af812377 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractTimeUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractTimeUUIDType.java
@@ -24,9 +24,11 @@ import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.Constants;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.UUIDSerializer;
 import org.apache.cassandra.utils.TimeUUID;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUIDAsBytes;
@@ -44,6 +46,7 @@ public abstract class AbstractTimeUUIDType<T> extends TemporalType<T>
         return true;
     }
 
+    @Override
     public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
         // Compare for length
@@ -58,12 +61,12 @@ public abstract class AbstractTimeUUIDType<T> extends TemporalType<T>
 
         long msb1 = accessorL.getLong(left, 0);
         long msb2 = accessorR.getLong(right, 0);
+        verifyVersion(msb1);
+        verifyVersion(msb2);
+
         msb1 = reorderTimestampBytes(msb1);
         msb2 = reorderTimestampBytes(msb2);
 
-        assert (msb1 & topbyte(0xf0L)) == topbyte(0x10L);
-        assert (msb2 & topbyte(0xf0L)) == topbyte(0x10L);
-
         int c = Long.compare(msb1, msb2);
         if (c != 0)
             return c;
@@ -75,6 +78,40 @@ public abstract class AbstractTimeUUIDType<T> extends TemporalType<T>
         return Long.compare(lsb1, lsb2);
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        if (accessor.isEmpty(data))
+            return null;
+
+        long hiBits = accessor.getLong(data, 0);
+        verifyVersion(hiBits);
+        ByteBuffer swizzled = ByteBuffer.allocate(16);
+        swizzled.putLong(0, TimeUUIDType.reorderTimestampBytes(hiBits));
+        swizzled.putLong(8, accessor.getLong(data, 8) ^ 0x8080808080808080L);
+
+        return ByteSource.fixedLength(swizzled);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        // Optional-style encoding of empty values as null sources
+        if (comparableBytes == null)
+            return accessor.empty();
+
+        // The non-lexical UUID bits are stored as an unsigned fixed-length 128-bit integer.
+        long hiBits = ByteSourceInverse.getUnsignedFixedLengthAsLong(comparableBytes, 8);
+        long loBits = ByteSourceInverse.getUnsignedFixedLengthAsLong(comparableBytes, 8);
+
+        hiBits = reorderBackTimestampBytes(hiBits);
+        verifyVersion(hiBits);
+        // In addition, TimeUUIDType also touches the low bits of the UUID (see CASSANDRA-8730 and DB-1758).
+        loBits ^= 0x8080808080808080L;
+
+        return UUIDType.makeUuidBytes(accessor, hiBits, loBits);
+    }
+
     // takes as input 8 signed bytes in native machine order
     // returns the first byte unchanged, and the following 7 bytes converted to an unsigned representation
     // which is the same as a 2's complement long in native format
@@ -83,16 +120,30 @@ public abstract class AbstractTimeUUIDType<T> extends TemporalType<T>
         return signedBytes ^ 0x0080808080808080L;
     }
 
-    private static long topbyte(long topbyte)
+    private void verifyVersion(long hiBits)
     {
-        return topbyte << 56;
+        long version = (hiBits >>> 12) & 0xF;
+        if (version != 1)
+            throw new MarshalException(String.format("Invalid UUID version %d for timeuuid",
+                                                     version));
     }
 
     protected static long reorderTimestampBytes(long input)
     {
-        return    (input <<  48)
-                  | ((input <<  16) & 0xFFFF00000000L)
-                  |  (input >>> 32);
+        return (input <<  48)
+               | ((input <<  16) & 0xFFFF00000000L)
+               |  (input >>> 32);
+    }
+
+    protected static long reorderBackTimestampBytes(long input)
+    {
+        // In a time-based UUID the high bits are significantly more shuffled than in other UUIDs - if [X] represents a
+        // 16-bit tuple, [1][2][3][4] should become [3][4][2][1].
+        // See the UUID Javadoc (and more specifically the high bits layout of a Leach-Salz UUID) to understand the
+        // reasoning behind this bit twiddling in the first place (in the context of comparisons).
+        return (input << 32)
+               | ((input >>> 16) & 0xFFFF0000L)
+               | (input >>> 48);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 74d4006664..8f54cb65a6 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -40,6 +40,9 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 import org.github.jamm.Unmetered;
 
 import static org.apache.cassandra.db.marshal.AbstractType.ComparisonType.CUSTOM;
@@ -55,6 +58,8 @@ import static org.apache.cassandra.db.marshal.AbstractType.ComparisonType.CUSTOM
 @Unmetered
 public abstract class AbstractType<T> implements Comparator<ByteBuffer>, AssignmentTestable
 {
+    private final static int VARIABLE_LENGTH = -1;
+
     public final Comparator<ByteBuffer> reverseComparator;
 
     public enum ComparisonType
@@ -449,11 +454,28 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
     }
 
     /**
-     * The length of values for this type if all values are of fixed length, -1 otherwise.
+     * The length of values for this type if all values are of fixed length, -1 otherwise. This has an impact on
+     * serialization.
+     * <lu>
+     *  <li> see {@link #writeValue} </li>
+     *  <li> see {@link #read} </li>
+     *  <li> see {@link #writtenLength} </li>
+     *  <li> see {@link #skipValue} </li>
+     * </lu>
      */
     public int valueLengthIfFixed()
     {
-        return -1;
+        return VARIABLE_LENGTH;
+    }
+
+    /**
+     * Checks if all values are of fixed length.
+     *
+     * @return {@code true} if all values are of fixed length, {@code false} otherwise.
+     */
+    public final boolean isValueLengthFixed()
+    {
+        return valueLengthIfFixed() != VARIABLE_LENGTH;
     }
 
     // This assumes that no empty values are passed
@@ -598,6 +620,69 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
         return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
     }
 
+    /**
+     * Produce a byte-comparable representation of the given value, i.e. a sequence of bytes that compares the same way
+     * using lexicographical unsigned byte comparison as the original value using the type's comparator.
+     *
+     * We use a slightly stronger requirement to be able to use the types in tuples. Precisely, for any pair x, y of
+     * non-equal valid values of this type and any bytes b1, b2 between 0x10 and 0xEF,
+     * (+ stands for concatenation)
+     *   compare(x, y) == compareLexicographicallyUnsigned(asByteComparable(x)+b1, asByteComparable(y)+b2)
+     * (i.e. the values compare like the original type, and an added 0x10-0xEF byte at the end does not change that) and:
+     *   asByteComparable(x)+b1 is not a prefix of asByteComparable(y)      (weakly prefix free)
+     * (i.e. a valid representation of a value may be a prefix of another valid representation of a value only if the
+     * following byte in the latter is smaller than 0x10 or larger than 0xEF). These properties are trivially true if
+     * the encoding compares correctly and is prefix free, but also permits a little more freedom that enables somewhat
+     * more efficient encoding of arbitrary-length byte-comparable blobs.
+     *
+     * Depending on the type, this method can be called for null or empty input, in which case the output is allowed to
+     * be null (the clustering/tuple encoding will accept and handle it).
+     */
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V value, ByteComparable.Version version)
+    {
+        if (isByteOrderComparable)
+        {
+            // When a type is byte-ordered on its own, we only need to escape it, so that we can include it in
+            // multi-component types and make the encoding weakly-prefix-free.
+            return ByteSource.of(accessor, value, version);
+        }
+        else
+            // default is only good for byte-comparables
+            throw new UnsupportedOperationException(getClass().getSimpleName() + " does not implement asComparableBytes");
+    }
+
+    public final ByteSource asComparableBytes(ByteBuffer byteBuffer, ByteComparable.Version version)
+    {
+        return asComparableBytes(ByteBufferAccessor.instance, byteBuffer, version);
+    }
+
+    /**
+     * Translates the given byte-ordered representation to the common, non-byte-ordered binary representation of a
+     * payload for this abstract type (the latter, common binary representation is what we mostly work with in the
+     * storage engine internals). If the given bytes don't correspond to the encoding of some payload value for this
+     * abstract type, an {@link IllegalArgumentException} may be thrown.
+     *
+     * @param accessor value accessor used to construct the value.
+     * @param comparableBytes A byte-ordered representation (presumably of a payload for this abstract type).
+     * @param version The byte-comparable version used to construct the representation.
+     * @return A of a payload for this abstract type, corresponding to the given byte-ordered representation,
+     *         constructed using the supplied value accessor.
+     *
+     * @see #asComparableBytes
+     */
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        if (isByteOrderComparable)
+            return accessor.valueOf(ByteSourceInverse.getUnescapedBytes(comparableBytes));
+        else
+            throw new UnsupportedOperationException(getClass().getSimpleName() + " does not implement fromComparableBytes");
+    }
+
+    public final ByteBuffer fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return fromComparableBytes(ByteBufferAccessor.instance, comparableBytes, version);
+    }
+
     /**
      * This must be overriden by subclasses if necessary so that for any
      * AbstractType, this == TypeParser.parse(toString()).
diff --git a/src/java/org/apache/cassandra/db/marshal/BooleanType.java b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
index 4ef5f95b0b..d144f4ee4d 100644
--- a/src/java/org/apache/cassandra/db/marshal/BooleanType.java
+++ b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
@@ -26,14 +26,11 @@ import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.BooleanSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 
 public class BooleanType extends AbstractType<Boolean>
 {
-    private static final Logger logger = LoggerFactory.getLogger(BooleanType.class);
-
     public static final BooleanType instance = new BooleanType();
 
     BooleanType() {super(ComparisonType.CUSTOM);} // singleton
@@ -54,6 +51,26 @@ public class BooleanType extends AbstractType<Boolean>
         return v1 - v2;
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        if (accessor.isEmpty(data))
+            return null;
+        byte b = accessor.toByte(data);
+        if (b != 0)
+            b = 1;
+        return ByteSource.oneByte(b);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        if (comparableBytes == null)
+            return accessor.empty();
+        int b = comparableBytes.next();
+        return accessor.valueOf(b == 1);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
 
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteArrayAccessor.java b/src/java/org/apache/cassandra/db/marshal/ByteArrayAccessor.java
index df24a627a4..d7108992da 100644
--- a/src/java/org/apache/cassandra/db/marshal/ByteArrayAccessor.java
+++ b/src/java/org/apache/cassandra/db/marshal/ByteArrayAccessor.java
@@ -248,6 +248,13 @@ public class ByteArrayAccessor implements ValueAccessor<byte[]>
         return Ballot.deserialize(value);
     }
 
+    @Override
+    public int putByte(byte[] dst, int offset, byte value)
+    {
+        dst[offset] = value;
+        return TypeSizes.BYTE_SIZE;
+    }
+
     @Override
     public int putShort(byte[] dst, int offset, short value)
     {
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java b/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java
index ea9bf11383..9b477aeeea 100644
--- a/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java
+++ b/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java
@@ -18,6 +18,7 @@
 
 package org.apache.cassandra.db.marshal;
 
+import org.apache.cassandra.db.AbstractArrayClusteringPrefix;
 import org.apache.cassandra.db.ArrayClustering;
 import org.apache.cassandra.db.ArrayClusteringBound;
 import org.apache.cassandra.db.ArrayClusteringBoundary;
@@ -33,7 +34,7 @@ import org.apache.cassandra.schema.TableMetadata;
 
 class ByteArrayObjectFactory implements ValueAccessor.ObjectFactory<byte[]>
 {
-    private static final Clustering<byte[]> EMPTY_CLUSTERING = new ArrayClustering()
+    private static final Clustering<byte[]> EMPTY_CLUSTERING = new ArrayClustering(AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY)
     {
         public String toString(TableMetadata metadata)
         {
@@ -41,14 +42,37 @@ class ByteArrayObjectFactory implements ValueAccessor.ObjectFactory<byte[]>
         }
     };
 
+    public static final Clustering<byte[]> STATIC_CLUSTERING = new ArrayClustering(AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY)
+    {
+        @Override
+        public Kind kind()
+        {
+            return Kind.STATIC_CLUSTERING;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "STATIC";
+        }
+
+        @Override
+        public String toString(TableMetadata metadata)
+        {
+            return toString();
+        }
+    };
+
     static final ValueAccessor.ObjectFactory<byte[]> instance = new ByteArrayObjectFactory();
 
     private ByteArrayObjectFactory() {}
 
     /** The smallest start bound, i.e. the one that starts before any row. */
-    private static final ArrayClusteringBound BOTTOM_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, new byte[0][]);
+    private static final ArrayClusteringBound BOTTOM_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND,
+                                                                                      AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY);
     /** The biggest end bound, i.e. the one that ends after any row. */
-    private static final ArrayClusteringBound TOP_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, new byte[0][]);
+    private static final ArrayClusteringBound TOP_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND,
+                                                                                   AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY);
 
     public Cell<byte[]> cell(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, byte[] value, CellPath path)
     {
@@ -65,6 +89,11 @@ class ByteArrayObjectFactory implements ValueAccessor.ObjectFactory<byte[]>
         return EMPTY_CLUSTERING;
     }
 
+    public Clustering<byte[]> staticClustering()
+    {
+        return STATIC_CLUSTERING;
+    }
+
     public ClusteringBound<byte[]> bound(ClusteringPrefix.Kind kind, byte[]... values)
     {
         return new ArrayClusteringBound(kind, values);
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java b/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java
index 40a3bf4b34..0712930c3a 100644
--- a/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java
+++ b/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java
@@ -252,6 +252,13 @@ public class ByteBufferAccessor implements ValueAccessor<ByteBuffer>
         return Ballot.deserialize(value);
     }
 
+    @Override
+    public int putByte(ByteBuffer dst, int offset, byte value)
+    {
+        dst.put(dst.position() + offset, value);
+        return TypeSizes.BYTE_SIZE;
+    }
+
     @Override
     public int putShort(ByteBuffer dst, int offset, short value)
     {
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java b/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java
index 00f4646341..0ac3db9265 100644
--- a/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java
+++ b/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.db.AbstractBufferClusteringPrefix;
 import org.apache.cassandra.db.BufferClustering;
 import org.apache.cassandra.db.BufferClusteringBound;
 import org.apache.cassandra.db.BufferClusteringBoundary;
@@ -31,24 +32,15 @@ import org.apache.cassandra.db.rows.BufferCell;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.schema.TableMetadata;
 
 class ByteBufferObjectFactory implements ValueAccessor.ObjectFactory<ByteBuffer>
 {
-    /** Empty clustering for tables having no clustering columns. */
-    private static final Clustering<ByteBuffer> EMPTY_CLUSTERING = new BufferClustering()
-    {
-        @Override
-        public String toString(TableMetadata metadata)
-        {
-            return "EMPTY";
-        }
-    };
-
     /** The smallest start bound, i.e. the one that starts before any row. */
-    private static final BufferClusteringBound BOTTOM_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, new ByteBuffer[0]);
+    private static final BufferClusteringBound BOTTOM_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND,
+                                                                                        AbstractBufferClusteringPrefix.EMPTY_VALUES_ARRAY);
     /** The biggest end bound, i.e. the one that ends after any row. */
-    private static final BufferClusteringBound TOP_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, new ByteBuffer[0]);
+    private static final BufferClusteringBound TOP_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND,
+                                                                                     AbstractBufferClusteringPrefix.EMPTY_VALUES_ARRAY);
 
     static final ValueAccessor.ObjectFactory<ByteBuffer> instance = new ByteBufferObjectFactory();
 
@@ -66,7 +58,12 @@ class ByteBufferObjectFactory implements ValueAccessor.ObjectFactory<ByteBuffer>
 
     public Clustering<ByteBuffer> clustering()
     {
-        return EMPTY_CLUSTERING;
+        return Clustering.EMPTY;
+    }
+
+    public Clustering<ByteBuffer> staticClustering()
+    {
+        return Clustering.STATIC_CLUSTERING;
     }
 
     public ClusteringBound<ByteBuffer> bound(ClusteringPrefix.Kind kind, ByteBuffer... values)
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteType.java b/src/java/org/apache/cassandra/db/marshal/ByteType.java
index f94f4bb01c..a910fbba11 100644
--- a/src/java/org/apache/cassandra/db/marshal/ByteType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ByteType.java
@@ -27,6 +27,10 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 public class ByteType extends NumberType<Byte>
 {
@@ -42,6 +46,19 @@ public class ByteType extends NumberType<Byte>
         return accessorL.getByte(left, 0) - accessorR.getByte(right, 0);
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, Version version)
+    {
+        // This type does not allow non-present values, but we do just to avoid future complexity.
+        return ByteSource.optionalSignedFixedLengthNumber(accessor, data);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return ByteSourceInverse.getOptionalSignedFixedLength(accessor, comparableBytes, 1);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
         // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
index c52cddc07f..5e9916e7e4 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Iterator;
 
@@ -27,6 +28,7 @@ import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.Lists;
 import org.apache.cassandra.cql3.Maps;
 import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -35,6 +37,9 @@ import org.apache.cassandra.serializers.CollectionSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 /**
  * The abstract validator that is the base for maps, sets and lists (both frozen and non-frozen).
@@ -245,6 +250,91 @@ public abstract class CollectionType<T> extends AbstractType<T>
         return this.toString(false);
     }
 
+    static <VL, VR> int compareListOrSet(AbstractType<?> elementsComparator, VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
+    {
+        // Note that this is only used if the collection is frozen
+        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
+
+        int sizeL = CollectionSerializer.readCollectionSize(left, accessorL, ProtocolVersion.V3);
+        int offsetL = CollectionSerializer.sizeOfCollectionSize(sizeL, ProtocolVersion.V3);
+        int sizeR = CollectionSerializer.readCollectionSize(right, accessorR, ProtocolVersion.V3);
+        int offsetR = TypeSizes.INT_SIZE;
+
+        for (int i = 0; i < Math.min(sizeL, sizeR); i++)
+        {
+            VL v1 = CollectionSerializer.readValue(left, accessorL, offsetL, ProtocolVersion.V3);
+            offsetL += CollectionSerializer.sizeOfValue(v1, accessorL, ProtocolVersion.V3);
+            VR v2 = CollectionSerializer.readValue(right, accessorR, offsetR, ProtocolVersion.V3);
+            offsetR += CollectionSerializer.sizeOfValue(v2, accessorR, ProtocolVersion.V3);
+            int cmp = elementsComparator.compare(v1, accessorL, v2, accessorR);
+            if (cmp != 0)
+                return cmp;
+        }
+
+        return Integer.compare(sizeL, sizeR);
+    }
+
+    static <V> ByteSource asComparableBytesListOrSet(AbstractType<?> elementsComparator,
+                                                     ValueAccessor<V> accessor,
+                                                     V data,
+                                                     ByteComparable.Version version)
+    {
+        if (accessor.isEmpty(data))
+            return null;
+
+        int offset = 0;
+        int size = CollectionSerializer.readCollectionSize(data, accessor, ProtocolVersion.V3);
+        offset += CollectionSerializer.sizeOfCollectionSize(size, ProtocolVersion.V3);
+        ByteSource[] srcs = new ByteSource[size];
+        for (int i = 0; i < size; ++i)
+        {
+            V v = CollectionSerializer.readValue(data, accessor, offset, ProtocolVersion.V3);
+            offset += CollectionSerializer.sizeOfValue(v, accessor, ProtocolVersion.V3);
+            srcs[i] = elementsComparator.asComparableBytes(accessor, v, version);
+        }
+        return ByteSource.withTerminatorMaybeLegacy(version, 0x00, srcs);
+    }
+
+    static <V> V fromComparableBytesListOrSet(ValueAccessor<V> accessor,
+                                              ByteSource.Peekable comparableBytes,
+                                              ByteComparable.Version version,
+                                              AbstractType<?> elementType)
+    {
+        if (comparableBytes == null)
+            return accessor.empty();
+        assert version != ByteComparable.Version.LEGACY; // legacy translation is not reversible
+
+        List<V> buffers = new ArrayList<>();
+        int separator = comparableBytes.next();
+        while (separator != ByteSource.TERMINATOR)
+        {
+            if (!ByteSourceInverse.nextComponentNull(separator))
+                buffers.add(elementType.fromComparableBytes(accessor, comparableBytes, version));
+            else
+                buffers.add(null);
+            separator = comparableBytes.next();
+        }
+        return CollectionSerializer.pack(buffers, accessor, buffers.size(), ProtocolVersion.V3);
+    }
+
+    public static String setOrListToJsonString(ByteBuffer buffer, AbstractType elementsType, ProtocolVersion protocolVersion)
+    {
+        ByteBuffer value = buffer.duplicate();
+        StringBuilder sb = new StringBuilder("[");
+        int size = CollectionSerializer.readCollectionSize(value, ByteBufferAccessor.instance, protocolVersion);
+        int offset = CollectionSerializer.sizeOfCollectionSize(size, protocolVersion);
+        for (int i = 0; i < size; i++)
+        {
+            if (i > 0)
+                sb.append(", ");
+            ByteBuffer element = CollectionSerializer.readValue(value, ByteBufferAccessor.instance, offset, protocolVersion);
+            offset += CollectionSerializer.sizeOfValue(element, ByteBufferAccessor.instance, protocolVersion);
+            sb.append(elementsType.toJSONString(element, protocolVersion));
+        }
+        return sb.append("]").toString();
+    }
+
     private static class CollectionPathSerializer implements CellPath.Serializer
     {
         public void serialize(CellPath path, DataOutputPlus out) throws IOException
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index bf5e914a9d..00cbeb5898 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -31,6 +32,9 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 import static com.google.common.collect.Iterables.any;
 import static com.google.common.collect.Iterables.transform;
@@ -165,6 +169,86 @@ public class CompositeType extends AbstractCompositeType
         return types.get(i);
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, Version version)
+    {
+        if (data == null || accessor.isEmpty(data))
+            return null;
+
+        ByteSource[] srcs = new ByteSource[types.size() * 2 + 1];
+        int length = accessor.size(data);
+
+        // statics go first
+        boolean isStatic = readIsStaticInternal(data, accessor);
+        int offset = startingOffsetInternal(isStatic);
+        srcs[0] = isStatic ? null : ByteSource.EMPTY;
+
+        int i = 0;
+        byte lastEoc = 0;
+        while (offset < length)
+        {
+            // Only the end-of-component byte of the last component of this composite can be non-zero, so the
+            // component before can't have a non-zero end-of-component byte.
+            assert lastEoc == 0 : lastEoc;
+
+            int componentLength = accessor.getUnsignedShort(data, offset);
+            offset += 2;
+            srcs[i * 2 + 1] = types.get(i).asComparableBytes(accessor, accessor.slice(data, offset, componentLength), version);
+            offset += componentLength;
+            lastEoc = accessor.getByte(data, offset);
+            offset += 1;
+            srcs[i * 2 + 2] = ByteSource.oneByte(lastEoc & 0xFF ^ 0x80); // end-of-component also takes part in comparison as signed byte
+            ++i;
+        }
+        // A composite may be leaving some values unspecified. If this is the case, make sure we terminate early
+        // so that translations created before an extra field was added match translations that have the field but don't
+        // specify a value for it.
+        if (i * 2 + 1 < srcs.length)
+            srcs = Arrays.copyOfRange(srcs, 0, i * 2 + 1);
+
+        return ByteSource.withTerminatorMaybeLegacy(version, ByteSource.END_OF_STREAM, srcs);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, Version version)
+    {
+        // For ByteComparable.Version.LEGACY the terminator byte is ByteSource.END_OF_STREAM. The latter means that it's
+        // indistinguishable from the END_OF_STREAM byte that gets returned _after_ the terminator byte has already
+        // been consumed, when the composite is part of a multi-component sequence. So if in such a scenario we consume
+        // the ByteSource.END_OF_STREAM terminator here, this will result in actually consuming the multi-component
+        // sequence separator after it and jumping directly into the bytes of the next component, when we try to
+        // consume the (already consumed) separator.
+        // Instead of trying to find a way around the situation, we can just take advantage of the fact that we don't
+        // need to decode from Version.LEGACY, assume that we never do that, and assert it here.
+        assert version != Version.LEGACY;
+
+        if (comparableBytes == null)
+            return accessor.empty();
+
+        int separator = comparableBytes.next();
+        boolean isStatic = ByteSourceInverse.nextComponentNull(separator);
+        int i = 0;
+        V[] buffers = accessor.createArray(types.size());
+        byte lastEoc = 0;
+
+        while ((separator = comparableBytes.next()) != ByteSource.TERMINATOR && i < types.size())
+        {
+            // Only the end-of-component byte of the last component of this composite can be non-zero, so the
+            // component before can't have a non-zero end-of-component byte.
+            assert lastEoc == 0 : lastEoc;
+
+            // Get the next type and decode its payload.
+            AbstractType<?> type = types.get(i);
+            V decoded = type.fromComparableBytes(accessor,
+                                                 ByteSourceInverse.nextComponentSource(comparableBytes, separator),
+                                                 version);
+            buffers[i++] = decoded;
+
+            lastEoc = ByteSourceInverse.getSignedByte(ByteSourceInverse.nextComponentSource(comparableBytes));
+        }
+        return build(accessor, isStatic, Arrays.copyOf(buffers, i), lastEoc);
+    }
+
     protected ParsedComparator parseComparator(int i, String part)
     {
         return new StaticParsedComparator(types.get(i), part);
@@ -370,6 +454,12 @@ public class CompositeType extends AbstractCompositeType
 
     @SafeVarargs
     public static <V> V build(ValueAccessor<V> accessor, boolean isStatic, V... values)
+    {
+        return build(accessor, isStatic, values, (byte) 0);
+    }
+
+    @VisibleForTesting
+    public static <V> V build(ValueAccessor<V> accessor, boolean isStatic, V[] values, byte lastEoc)
     {
         int totalLength = isStatic ? 2 : 0;
         for (V v : values)
@@ -380,11 +470,12 @@ public class CompositeType extends AbstractCompositeType
         if (isStatic)
             out.putShort((short)STATIC_MARKER);
 
-        for (V v : values)
+        for (int i = 0; i < values.length; ++i)
         {
+            V v = values[i];
             ByteBufferUtil.writeShortLength(out, accessor.size(v));
             accessor.write(v, out);
-            out.put((byte) 0);
+            out.put(i != values.length - 1 ? (byte) 0 : lastEoc);
         }
         out.flip();
         return accessor.valueOf(out);
diff --git a/src/java/org/apache/cassandra/db/marshal/DateType.java b/src/java/org/apache/cassandra/db/marshal/DateType.java
index 473cedf407..595106d3d1 100644
--- a/src/java/org/apache/cassandra/db/marshal/DateType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DateType.java
@@ -31,6 +31,9 @@ import org.apache.cassandra.serializers.TimestampSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 /**
  * This is the old version of TimestampType, but has been replaced as it wasn't comparing pre-epoch timestamps
@@ -50,6 +53,19 @@ public class DateType extends AbstractType<Date>
         return true;
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        // While BYTE_ORDER would still work for this type, making use of the fixed length is more efficient.
+        return ByteSource.optionalFixedLength(accessor, data);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return ByteSourceInverse.getOptionalFixedLength(accessor, comparableBytes, 8);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
       // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/DecimalType.java b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
index 5740fdcc0f..3e02dc9696 100644
--- a/src/java/org/apache/cassandra/db/marshal/DecimalType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
@@ -24,6 +24,8 @@ import java.math.RoundingMode;
 import java.nio.ByteBuffer;
 import java.util.Objects;
 
+import com.google.common.primitives.Ints;
+
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.Constants;
 import org.apache.cassandra.cql3.Term;
@@ -32,6 +34,8 @@ import org.apache.cassandra.serializers.DecimalSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 
 public class DecimalType extends NumberType<BigDecimal>
 {
@@ -41,6 +45,16 @@ public class DecimalType extends NumberType<BigDecimal>
     private static final int MAX_SCALE = 1000;
     private static final MathContext MAX_PRECISION = new MathContext(10000);
 
+    // Constants or escaping values needed to encode/decode variable-length floating point numbers (decimals) in our
+    // custom byte-ordered encoding scheme.
+    private static final int POSITIVE_DECIMAL_HEADER_MASK = 0x80;
+    private static final int NEGATIVE_DECIMAL_HEADER_MASK = 0x00;
+    private static final int DECIMAL_EXPONENT_LENGTH_HEADER_MASK = 0x40;
+    private static final byte DECIMAL_LAST_BYTE = (byte) 0x00;
+    private static final BigInteger HUNDRED = BigInteger.valueOf(100);
+
+    private static final ByteBuffer ZERO_BUFFER = instance.decompose(BigDecimal.ZERO);
+
     DecimalType() {super(ComparisonType.CUSTOM);} // singleton
 
     public boolean isEmptyValueMeaningless()
@@ -59,6 +73,196 @@ public class DecimalType extends NumberType<BigDecimal>
         return compareComposed(left, accessorL, right, accessorR, this);
     }
 
+    /**
+     * Constructs a byte-comparable representation.
+     * This is rather difficult and involves reconstructing the decimal.
+     *
+     * To compare, we need a normalized value, i.e. one with a sign, exponent and (0,1) mantissa. To avoid
+     * loss of precision, both exponent and mantissa need to be base-100.  We can't get this directly off the serialized
+     * bytes, as they have base-10 scale and base-256 unscaled part.
+     *
+     * We store:
+     *     - sign bit inverted * 0x80 + 0x40 + signed exponent length, where exponent is negated if value is negative
+     *     - zero or more exponent bytes (as given by length)
+     *     - 0x80 + first pair of decimal digits, negative if value is negative, rounded to -inf
+     *     - zero or more 0x80 + pair of decimal digits, always positive
+     *     - trailing 0x00
+     * Zero is special-cased as 0x80.
+     *
+     * Because the trailing 00 cannot be produced from a pair of decimal digits (positive or not), no value can be
+     * a prefix of another.
+     *
+     * Encoding examples:
+     *    1.1    as       c1 = 0x80 (positive number) + 0x40 + (positive exponent) 0x01 (exp length 1)
+     *                    01 = exponent 1 (100^1)
+     *                    81 = 0x80 + 01 (0.01)
+     *                    8a = 0x80 + 10 (....10)   0.0110e2
+     *                    00
+     *    -1     as       3f = 0x00 (negative number) + 0x40 - (negative exponent) 0x01 (exp length 1)
+     *                    ff = exponent -1. negative number, thus 100^1
+     *                    7f = 0x80 - 01 (-0.01)    -0.01e2
+     *                    00
+     *    -99.9  as       3f = 0x00 (negative number) + 0x40 - (negative exponent) 0x01 (exp length 1)
+     *                    ff = exponent -1. negative number, thus 100^1
+     *                    1c = 0x80 - 100 (-1.00)
+     *                    8a = 0x80 + 10  (+....10) -0.999e2
+     *                    00
+     *
+     */
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        BigDecimal value = compose(data, accessor);
+        if (value == null)
+            return null;
+        if (value.compareTo(BigDecimal.ZERO) == 0)  // Note: 0.equals(0.0) returns false!
+            return ByteSource.oneByte(POSITIVE_DECIMAL_HEADER_MASK);
+
+        long scale = (((long) value.scale()) - value.precision()) & ~1;
+        boolean negative = value.signum() < 0;
+        // Make a base-100 exponent (this will always fit in an int).
+        int exponent = Math.toIntExact(-scale >> 1);
+        // Flip the exponent sign for negative numbers, so that ones with larger magnitudes are propely treated as smaller.
+        final int modulatedExponent = negative ? -exponent : exponent;
+        // We should never have scale > Integer.MAX_VALUE, as we're always subtracting the non-negative precision of
+        // the encoded BigDecimal, and furthermore we're rounding to negative infinity.
+        assert scale <= Integer.MAX_VALUE;
+        // However, we may end up overflowing on the negative side.
+        if (scale < Integer.MIN_VALUE)
+        {
+            // As scaleByPowerOfTen needs an int scale, do the scaling in two steps.
+            int mv = Integer.MIN_VALUE;
+            value = value.scaleByPowerOfTen(mv);
+            scale -= mv;
+        }
+        final BigDecimal mantissa = value.scaleByPowerOfTen(Ints.checkedCast(scale)).stripTrailingZeros();
+        // We now have a smaller-than-one signed mantissa, and a signed and modulated base-100 exponent.
+        assert mantissa.abs().compareTo(BigDecimal.ONE) < 0;
+
+        return new ByteSource()
+        {
+            // Start with up to 5 bytes for sign + exponent.
+            int exponentBytesLeft = 5;
+            BigDecimal current = mantissa;
+
+            @Override
+            public int next()
+            {
+                if (exponentBytesLeft > 0)
+                {
+                    --exponentBytesLeft;
+                    if (exponentBytesLeft == 4)
+                    {
+                        // Skip leading zero bytes in the modulatedExponent.
+                        exponentBytesLeft -= Integer.numberOfLeadingZeros(Math.abs(modulatedExponent)) / 8;
+                        // Now prepare the leading byte which includes the sign of the number plus the sign and length of the modulatedExponent.
+                        int explen = DECIMAL_EXPONENT_LENGTH_HEADER_MASK + (modulatedExponent < 0 ? -exponentBytesLeft : exponentBytesLeft);
+                        return explen + (negative ? NEGATIVE_DECIMAL_HEADER_MASK : POSITIVE_DECIMAL_HEADER_MASK);
+                    }
+                    else
+                        return (modulatedExponent >> (exponentBytesLeft * 8)) & 0xFF;
+                }
+                else if (current == null)
+                {
+                    return END_OF_STREAM;
+                }
+                else if (current.compareTo(BigDecimal.ZERO) == 0)
+                {
+                    current = null;
+                    return 0x00;
+                }
+                else
+                {
+                    BigDecimal v = current.scaleByPowerOfTen(2);
+                    BigDecimal floor = v.setScale(0, RoundingMode.FLOOR);
+                    current = v.subtract(floor);
+                    return floor.byteValueExact() + 0x80;
+                }
+            }
+        };
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        if (comparableBytes == null)
+            return accessor.empty();
+
+        int headerBits = comparableBytes.next();
+        if (headerBits == POSITIVE_DECIMAL_HEADER_MASK)
+            return accessor.valueOf(ZERO_BUFFER);
+
+        // I. Extract the exponent.
+        // The sign of the decimal, and the sign and the length (in bytes) of the decimal exponent, are all encoded in
+        // the first byte.
+        // Get the sign of the decimal...
+        boolean isNegative = headerBits < POSITIVE_DECIMAL_HEADER_MASK;
+        headerBits -= isNegative ? NEGATIVE_DECIMAL_HEADER_MASK : POSITIVE_DECIMAL_HEADER_MASK;
+        headerBits -= DECIMAL_EXPONENT_LENGTH_HEADER_MASK;
+        // Get the sign and the length of the exponent (the latter is encoded as its negative if the sign of the
+        // exponent is negative)...
+        boolean isExponentNegative = headerBits < 0;
+        headerBits = isExponentNegative ? -headerBits : headerBits;
+        // Now consume the exponent bytes. If the exponent is negative and uses less than 4 bytes, the remaining bytes
+        // should be padded with 1s, in order for the constructed int to contain the correct (negative) exponent value.
+        // So, if the exponent is negative, we can just start with all bits set to 1 (i.e. we can start with -1).
+        int exponent = isExponentNegative ? -1 : 0;
+        for (int i = 0; i < headerBits; ++i)
+            exponent = (exponent << 8) | comparableBytes.next();
+        // The encoded exponent also contains the decimal sign, in order to correctly compare exponents in case of
+        // negative decimals (e.g. x * 10^y > x * 10^z if x < 0 && y < z). After the decimal sign is "removed", what's
+        // left is a base-100 exponent following BigDecimal's convention for the exponent sign.
+        exponent = isNegative ? -exponent : exponent;
+
+        // II. Extract the mantissa as a BigInteger value. It was encoded as a BigDecimal value between 0 and 1, in
+        // order to be used for comparison (after the sign of the decimal and the sign and the value of the exponent),
+        // but when decoding we don't need that property on the transient mantissa value.
+        BigInteger mantissa = BigInteger.ZERO;
+        int curr = comparableBytes.next();
+        while (curr != DECIMAL_LAST_BYTE)
+        {
+            // The mantissa value is constructed by a standard positional notation value calculation.
+            // The value of the next digit is the next most-significant mantissa byte as an unsigned integer,
+            // offset by a predetermined value (in this case, 0x80)...
+            int currModified = curr - 0x80;
+            // ...multiply the current value by the base (in this case, 100)...
+            mantissa = mantissa.multiply(HUNDRED);
+            // ...then add the next digit to the modified current value...
+            mantissa = mantissa.add(BigInteger.valueOf(currModified));
+            // ...and finally, adjust the base-100, BigDecimal format exponent accordingly.
+            --exponent;
+            curr = comparableBytes.next();
+        }
+
+        // III. Construct the final BigDecimal value, by combining the mantissa and the exponent, guarding against
+        // underflow or overflow when exponents are close to their boundary values.
+        long base10NonBigDecimalFormatExp = 2L * exponent;
+        // When expressing a sufficiently big decimal, BigDecimal's internal scale value will be negative with very
+        // big absolute value. To compute the encoded exponent, this internal scale has the number of digits of the
+        // unscaled value subtracted from it, after which it's divided by 2, rounding down to negative infinity
+        // (before accounting for the decimal sign). When decoding, this exponent is converted to a base-10 exponent in
+        // non-BigDecimal format, which means that it can very well overflow Integer.MAX_VALUE.
+        // For example, see how <code>new BigDecimal(BigInteger.TEN, Integer.MIN_VALUE)</code> is encoded and decoded.
+        if (base10NonBigDecimalFormatExp > Integer.MAX_VALUE)
+        {
+            // If the base-10 exponent will result in an overflow, some of its powers of 10 need to be absorbed by the
+            // mantissa. How much exactly? As little as needed, in order to avoid complex BigInteger operations, which
+            // means exactly as much as to have a scale of -Integer.MAX_VALUE.
+            int exponentReduction = (int) (base10NonBigDecimalFormatExp - Integer.MAX_VALUE);
+            mantissa = mantissa.multiply(BigInteger.TEN.pow(exponentReduction));
+            base10NonBigDecimalFormatExp = Integer.MAX_VALUE;
+        }
+        assert base10NonBigDecimalFormatExp >= Integer.MIN_VALUE && base10NonBigDecimalFormatExp <= Integer.MAX_VALUE;
+        // Here we negate the exponent, as we are not using BigDecimal.scaleByPowerOfTen, where a positive number means
+        // "multiplying by a positive power of 10", but to BigDecimal's internal scale representation, where a positive
+        // number means "dividing by a positive power of 10".
+        byte[] mantissaBytes = mantissa.toByteArray();
+        V resultBuf = accessor.allocate(4 + mantissaBytes.length);
+        accessor.putInt(resultBuf, 0, (int) -base10NonBigDecimalFormatExp);
+        accessor.copyByteArrayTo(mantissaBytes, 0, resultBuf, 4, mantissaBytes.length);
+        return resultBuf;
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
         // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/DoubleType.java b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
index 570d420a75..56ae0131b3 100644
--- a/src/java/org/apache/cassandra/db/marshal/DoubleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
@@ -27,6 +27,9 @@ import org.apache.cassandra.serializers.DoubleSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 public class DoubleType extends NumberType<Double>
 {
@@ -50,6 +53,18 @@ public class DoubleType extends NumberType<Double>
         return compareComposed(left, accessorL, right, accessorR, this);
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        return ByteSource.optionalSignedFixedLengthFloat(accessor, data);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return ByteSourceInverse.getOptionalSignedFixedLengthFloat(accessor, comparableBytes, 8);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
       // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
index 5df3600995..e7a2360fa9 100644
--- a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
@@ -19,9 +19,16 @@ package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,6 +41,9 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 import static com.google.common.collect.Iterables.any;
 
@@ -60,7 +70,11 @@ public class DynamicCompositeType extends AbstractCompositeType
 {
     private static final Logger logger = LoggerFactory.getLogger(DynamicCompositeType.class);
 
+    private static final ByteSource[] EMPTY_BYTE_SOURCE_ARRAY = new ByteSource[0];
+    private static final String REVERSED_TYPE = ReversedType.class.getSimpleName();
+
     private final Map<Byte, AbstractType<?>> aliases;
+    private final Map<AbstractType<?>, Byte> inverseMapping;
 
     // interning instances
     private static final ConcurrentHashMap<Map<Byte, AbstractType<?>>, DynamicCompositeType> instances = new ConcurrentHashMap<>();
@@ -81,6 +95,9 @@ public class DynamicCompositeType extends AbstractCompositeType
     private DynamicCompositeType(Map<Byte, AbstractType<?>> aliases)
     {
         this.aliases = aliases;
+        this.inverseMapping = new HashMap<>();
+        for (Map.Entry<Byte, AbstractType<?>> en : aliases.entrySet())
+            this.inverseMapping.put(en.getValue(), en.getKey());
     }
 
     protected <V> boolean readIsStatic(V value, ValueAccessor<V> accessor)
@@ -197,6 +214,196 @@ public class DynamicCompositeType extends AbstractCompositeType
         }
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, Version version)
+    {
+        List<ByteSource> srcs = new ArrayList<>();
+        int length = accessor.size(data);
+
+        // statics go first
+        boolean isStatic = readIsStatic(data, accessor);
+        int offset = startingOffset(isStatic);
+        srcs.add(isStatic ? null : ByteSource.EMPTY);
+
+        byte lastEoc = 0;
+        int i = 0;
+        while (offset < length)
+        {
+            // Only the end-of-component byte of the last component of this composite can be non-zero, so the
+            // component before can't have a non-zero end-of-component byte.
+            assert lastEoc == 0 : lastEoc;
+
+            AbstractType<?> comp = getComparator(data, accessor, offset);
+            offset += getComparatorSize(i, data, accessor, offset);
+            // The comparable bytes for the component need to ensure comparisons consistent with
+            // AbstractCompositeType.compareCustom(ByteBuffer, ByteBuffer) and
+            // DynamicCompositeType.getComparator(int, ByteBuffer, ByteBuffer):
+            if (version == Version.LEGACY || !(comp instanceof ReversedType))
+            {
+                // ...most often that means just adding the short name of the type, followed by the full name of the type.
+                srcs.add(ByteSource.of(comp.getClass().getSimpleName(), version));
+                srcs.add(ByteSource.of(comp.getClass().getName(), version));
+            }
+            else
+            {
+                // ...however some times the component uses a complex type (currently the only supported complex type
+                // is ReversedType - we can't have elements that are of MapType, CompositeType, TupleType, etc.)...
+                ReversedType<?> reversedComp = (ReversedType<?>) comp;
+                // ...in this case, we need to add the short name of ReversedType before the short name of the base
+                // type, to ensure consistency with DynamicCompositeType.getComparator(int, ByteBuffer, ByteBuffer).
+                srcs.add(ByteSource.of(REVERSED_TYPE, version));
+                srcs.add(ByteSource.of(reversedComp.baseType.getClass().getSimpleName(), version));
+                srcs.add(ByteSource.of(reversedComp.baseType.getClass().getName(), version));
+            }
+            // Only then the payload of the component gets encoded.
+            int componentLength = accessor.getUnsignedShort(data, offset);
+            offset += 2;
+            srcs.add(comp.asComparableBytes(accessor, accessor.slice(data, offset, componentLength), version));
+            offset += componentLength;
+            // The end-of-component byte also takes part in the comparison, and therefore needs to be encoded.
+            lastEoc = accessor.getByte(data, offset);
+            offset += 1;
+            srcs.add(ByteSource.oneByte(version == Version.LEGACY ? lastEoc : lastEoc & 0xFF ^ 0x80));
+            ++i;
+        }
+
+        return ByteSource.withTerminatorMaybeLegacy(version, ByteSource.END_OF_STREAM, srcs.toArray(EMPTY_BYTE_SOURCE_ARRAY));
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, Version version)
+    {
+        // For ByteComparable.Version.LEGACY the terminator byte is ByteSource.END_OF_STREAM. Just like with
+        // CompositeType, this means that in multi-component sequences the terminator may be transformed to a regular
+        // component separator, but unlike CompositeType (where we have the expected number of types/components),
+        // this can make the end of the whole dynamic composite type indistinguishable from the end of a component
+        // somewhere in the middle of the dynamic composite type. Because of that, DynamicCompositeType elements
+        // cannot always be safely decoded using that encoding version.
+        // Even more so than with CompositeType, we just take advantage of the fact that we don't need to decode from
+        // Version.LEGACY, assume that we never do that, and assert it here.
+        assert version != Version.LEGACY;
+
+        if (comparableBytes == null)
+            return accessor.empty();
+
+        // The first byte is the isStatic flag which we don't need but must consume to continue past it.
+        comparableBytes.next();
+
+        List<AbstractType<?>> types = new ArrayList<>();
+        List<V> values = new ArrayList<>();
+        byte lastEoc = 0;
+
+        for (int separator = comparableBytes.next(); separator != ByteSource.TERMINATOR; separator = comparableBytes.next())
+        {
+            // Solely the end-of-component byte of the last component of this composite can be non-zero.
+            assert lastEoc == 0 : lastEoc;
+
+            boolean isReversed = false;
+            // Decode the next type's simple class name that is encoded before its fully qualified class name (in order
+            // for comparisons to work correctly).
+            String simpleClassName = ByteSourceInverse.getString(ByteSourceInverse.nextComponentSource(comparableBytes, separator));
+            if (REVERSED_TYPE.equals(simpleClassName))
+            {
+                // Special-handle if the type is reversed (and decode the actual base type simple class name).
+                isReversed = true;
+                simpleClassName = ByteSourceInverse.getString(ByteSourceInverse.nextComponentSource(comparableBytes));
+            }
+
+            // Decode the type's fully qualified class name and parse the actual type from it.
+            String fullClassName = ByteSourceInverse.getString(ByteSourceInverse.nextComponentSource(comparableBytes));
+            assert fullClassName.endsWith(simpleClassName);
+            if (isReversed)
+                fullClassName = REVERSED_TYPE + '(' + fullClassName + ')';
+            AbstractType<?> type = TypeParser.parse(fullClassName);
+            assert type != null;
+            types.add(type);
+
+            // Decode the payload from this type.
+            V value = type.fromComparableBytes(accessor, ByteSourceInverse.nextComponentSource(comparableBytes), version);
+            values.add(value);
+
+            // Also decode the corresponding end-of-component byte - the last one we decode will be taken into
+            // account when we deserialize the decoded data into an object.
+            lastEoc = ByteSourceInverse.getSignedByte(ByteSourceInverse.nextComponentSource(comparableBytes));
+        }
+        return build(accessor, types, inverseMapping, values, lastEoc);
+    }
+
+    public static ByteBuffer build(List<String> types, List<ByteBuffer> values)
+    {
+        return build(ByteBufferAccessor.instance,
+                     Lists.transform(types, TypeParser::parse),
+                     Collections.emptyMap(),
+                     values,
+                     (byte) 0);
+    }
+
+    @VisibleForTesting
+    public static <V> V build(ValueAccessor<V> accessor,
+                              List<AbstractType<?>> types,
+                              Map<AbstractType<?>, Byte> inverseMapping,
+                              List<V> values,
+                              byte lastEoc)
+    {
+        assert types.size() == values.size();
+
+        int numComponents = types.size();
+        // Compute the total number of bytes that we'll need to store the types and their payloads.
+        int totalLength = 0;
+        for (int i = 0; i < numComponents; ++i)
+        {
+            AbstractType<?> type = types.get(i);
+            Byte alias = inverseMapping.get(type);
+            int typeNameLength = alias == null ? type.toString().getBytes(StandardCharsets.UTF_8).length : 0;
+            // The type data will be stored by means of the type's fully qualified name, not by aliasing, so:
+            //   1. The type data header should be the fully qualified name length in bytes.
+            //   2. The length should be small enough so that it fits in 15 bits (2 bytes with the first bit zero).
+            assert typeNameLength <= 0x7FFF;
+            int valueLength = accessor.size(values.get(i));
+            // The value length should also expect its first bit to be 0, as the length should be stored as a signed
+            // 2-byte value (short).
+            assert valueLength <= 0x7FFF;
+            totalLength += 2 + typeNameLength + 2 + valueLength + 1;
+        }
+
+        V result = accessor.allocate(totalLength);
+        int offset = 0;
+        for (int i = 0; i < numComponents; ++i)
+        {
+            AbstractType<?> type = types.get(i);
+            Byte alias = inverseMapping.get(type);
+            if (alias == null)
+            {
+                // Write the type data (2-byte length header + the fully qualified type name in UTF-8).
+                byte[] typeNameBytes = type.toString().getBytes(StandardCharsets.UTF_8);
+                accessor.putShort(result,
+                                  offset,
+                                  (short) typeNameBytes.length); // this should work fine also if length >= 32768
+                offset += 2;
+                accessor.copyByteArrayTo(typeNameBytes, 0, result, offset, typeNameBytes.length);
+                offset += typeNameBytes.length;
+            }
+            else
+            {
+                accessor.putShort(result, offset, (short) (alias | 0x8000));
+                offset += 2;
+            }
+
+            // Write the type payload data (2-byte length header + the payload).
+            V value = values.get(i);
+            int bytesToCopy = accessor.size(value);
+            accessor.putShort(result, offset, (short) bytesToCopy);
+            offset += 2;
+            accessor.copyTo(value, 0, result, accessor, offset, bytesToCopy);
+            offset += bytesToCopy;
+
+            // Write the end-of-component byte.
+            accessor.putByte(result, offset, i != numComponents - 1 ? (byte) 0 : lastEoc);
+            offset += 1;
+        }
+        return result;
+    }
+
     protected ParsedComparator parseComparator(int i, String part)
     {
         return new DynamicParsedComparator(part);
diff --git a/src/java/org/apache/cassandra/db/marshal/EmptyType.java b/src/java/org/apache/cassandra/db/marshal/EmptyType.java
index 357b6e85ad..dcc57b7c4a 100644
--- a/src/java/org/apache/cassandra/db/marshal/EmptyType.java
+++ b/src/java/org/apache/cassandra/db/marshal/EmptyType.java
@@ -33,6 +33,8 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 import org.apache.cassandra.utils.NoSpamLogger;
 
 /**
@@ -68,6 +70,18 @@ public class EmptyType extends AbstractType<Void>
 
     private EmptyType() {super(ComparisonType.CUSTOM);} // singleton
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        return null;
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return accessor.empty();
+    }
+
     public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
         return 0;
diff --git a/src/java/org/apache/cassandra/db/marshal/FloatType.java b/src/java/org/apache/cassandra/db/marshal/FloatType.java
index 35abee0f98..2adb127d41 100644
--- a/src/java/org/apache/cassandra/db/marshal/FloatType.java
+++ b/src/java/org/apache/cassandra/db/marshal/FloatType.java
@@ -27,6 +27,9 @@ import org.apache.cassandra.serializers.FloatSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 
 public class FloatType extends NumberType<Float>
@@ -51,6 +54,18 @@ public class FloatType extends NumberType<Float>
         return compareComposed(left, accessorL, right, accessorR, this);
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        return ByteSource.optionalSignedFixedLengthFloat(accessor, data);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return ByteSourceInverse.getOptionalSignedFixedLengthFloat(accessor, comparableBytes, 4);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
       // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/Int32Type.java b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
index 98f4c83cf6..6dee26e224 100644
--- a/src/java/org/apache/cassandra/db/marshal/Int32Type.java
+++ b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
@@ -28,6 +28,9 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 public class Int32Type extends NumberType<Integer>
 {
@@ -55,6 +58,18 @@ public class Int32Type extends NumberType<Integer>
         return ValueAccessor.compare(left, accessorL, right, accessorR);
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        return ByteSource.optionalSignedFixedLengthNumber(accessor, data);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return ByteSourceInverse.getOptionalSignedFixedLength(accessor, comparableBytes, 4);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
         // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/IntegerType.java b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
index 4c913d50af..b52bda8900 100644
--- a/src/java/org/apache/cassandra/db/marshal/IntegerType.java
+++ b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
@@ -30,11 +30,23 @@ import org.apache.cassandra.serializers.IntegerSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 public final class IntegerType extends NumberType<BigInteger>
 {
     public static final IntegerType instance = new IntegerType();
 
+    // Constants or escaping values needed to encode/decode variable-length integers in our custom byte-ordered
+    // encoding scheme.
+    private static final int POSITIVE_VARINT_HEADER = 0x80;
+    private static final int NEGATIVE_VARINT_LENGTH_HEADER = 0x00;
+    private static final int POSITIVE_VARINT_LENGTH_HEADER = 0xFF;
+    private static final byte BIG_INTEGER_NEGATIVE_LEADING_ZERO = (byte) 0xFF;
+    private static final byte BIG_INTEGER_POSITIVE_LEADING_ZERO = (byte) 0x00;
+    public static final int FULL_FORM_THRESHOLD = 7;
+
     private static <V> int findMostSignificantByte(V value, ValueAccessor<V> accessor)
     {
         int len = accessor.size(value) - 1;
@@ -131,6 +143,301 @@ public final class IntegerType extends NumberType<BigInteger>
         return 0;
     }
 
+    /**
+     * Constructs a byte-comparable representation of the number.
+     *
+     * In the current format we represent it:
+     *    directly as varint, if the length is 6 or smaller (the encoding has non-00/FF first byte)
+     *    <signbyte><length as unsigned integer - 7><7 or more bytes>, otherwise
+     * where <signbyte> is 00 for negative numbers and FF for positive ones, and the length's bytes are inverted if
+     * the number is negative (so that longer length sorts smaller).
+     *
+     * Because we present the sign separately, we don't need to include 0x00 prefix for positive integers whose first
+     * byte is >= 0x80 or 0xFF prefix for negative integers whose first byte is < 0x80. Note that we do this before
+     * taking the length for the purposes of choosing between varint and full-form encoding.
+     *
+     * The representations are prefix-free, because the choice between varint and full-form encoding is determined by
+     * the first byte where varints are properly ordered between full-form negative and full-form positive, varint
+     * encoding is prefix-free, and full-form representations of different length always have length bytes that differ.
+     *
+     * Examples:
+     *    -1            as 7F
+     *    0             as 80
+     *    1             as 81
+     *    127           as C07F
+     *    255           as C0FF
+     *    2^32-1        as F8FFFFFFFF
+     *    2^32          as F900000000
+     *    2^56-1        as FEFFFFFFFFFFFFFF
+     *    2^56          as FF000100000000000000
+     *
+     * See {@link #asComparableBytesLegacy} for description of the legacy format.
+     */
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        final int limit = accessor.size(data);
+        if (limit == 0)
+            return null;
+
+        // skip any leading sign-only byte(s)
+        int p = 0;
+        final byte signbyte = accessor.getByte(data, p);
+        if (signbyte == BIG_INTEGER_NEGATIVE_LEADING_ZERO || signbyte == BIG_INTEGER_POSITIVE_LEADING_ZERO)
+        {
+            while (p + 1 < limit)
+            {
+                if (accessor.getByte(data, ++p) != signbyte)
+                    break;
+            }
+        }
+
+        if (version != ByteComparable.Version.LEGACY)
+            return (limit - p < FULL_FORM_THRESHOLD)
+                   ? encodeAsVarInt(accessor, data, limit)
+                   : asComparableBytesCurrent(accessor, data, p, limit, (signbyte >> 7) & 0xFF);
+        else
+            return asComparableBytesLegacy(accessor, data, p, limit, signbyte);
+    }
+
+    /**
+     * Encode the BigInteger stored in the given buffer as a variable-length signed integer.
+     * The length of the number is given in the limit argument, and must be <= 8.
+     */
+    private <V> ByteSource encodeAsVarInt(ValueAccessor<V> accessor, V data, int limit)
+    {
+        long v;
+        switch (limit)
+        {
+            case 1:
+                v = accessor.getByte(data, 0);
+                break;
+            case 2:
+                v = accessor.getShort(data, 0);
+                break;
+            case 3:
+                v = (accessor.getShort(data, 0) << 8) | (accessor.getByte(data, 2) & 0xFF);
+                break;
+            case 4:
+                v = accessor.getInt(data, 0);
+                break;
+            case 5:
+                v = ((long) accessor.getInt(data, 0) << 8) | (accessor.getByte(data, 4) & 0xFF);
+                break;
+            case 6:
+                v = ((long) accessor.getInt(data, 0) << 16) | (accessor.getShort(data, 4) & 0xFFFF);
+                break;
+            case 7:
+                v = ((long) accessor.getInt(data, 0) << 24) | ((accessor.getShort(data, 4) & 0xFFFF) << 8) | (accessor.getByte(data, 6) & 0xFF);
+                break;
+            case 8:
+                // This is not reachable within the encoding; added for completeness.
+                v = accessor.getLong(data, 0);
+                break;
+            default:
+                throw new AssertionError();
+        }
+        return ByteSource.variableLengthInteger(v);
+    }
+
+    /**
+     * Constructs a full-form byte-comparable representation of the number in the current format.
+     *
+     * This contains:
+     *    <signbyte><length as unsigned integer - 7><7 or more bytes>, otherwise
+     * where <signbyte> is 00 for negative numbers and FF for positive ones, and the length's bytes are inverted if
+     * the number is negative (so that longer length sorts smaller).
+     *
+     * Because we present the sign separately, we don't need to include 0x00 prefix for positive integers whose first
+     * byte is >= 0x80 or 0xFF prefix for negative integers whose first byte is < 0x80.
+     *
+     * The representations are prefix-free, because representations of different length always have length bytes that
+     * differ.
+     */
+    private <V> ByteSource asComparableBytesCurrent(ValueAccessor<V> accessor, V data, int startpos, int limit, int signbyte)
+    {
+        // start with sign as a byte, then variable-length-encoded length, then bytes (stripped leading sign)
+        return new ByteSource()
+        {
+            int pos = -2;
+            ByteSource lengthEncoding = new VariableLengthUnsignedInteger(limit - startpos - FULL_FORM_THRESHOLD);
+
+            @Override
+            public int next()
+            {
+                if (pos == -2)
+                {
+                    ++pos;
+                    return signbyte ^ 0xFF; // 00 for negative/FF for positive (01-FE for direct varint encoding)
+                }
+                else if (pos == -1)
+                {
+                    int nextByte = lengthEncoding.next();
+                    if (nextByte != END_OF_STREAM)
+                        return nextByte ^ signbyte;
+                    pos = startpos;
+                }
+
+                if (pos == limit)
+                    return END_OF_STREAM;
+
+                return accessor.getByte(data, pos++) & 0xFF;
+            }
+        };
+    }
+
+    /**
+     * Constructs a byte-comparable representation of the number in the legacy format.
+     * We represent it as
+     *    <zero or more length_bytes where length = 128> <length_byte> <first_significant_byte> <zero or more bytes>
+     * where a length_byte is:
+     *    - 0x80 + (length - 1) for positive numbers (so that longer length sorts bigger)
+     *    - 0x7F - (length - 1) for negative numbers (so that longer length sorts smaller)
+     *
+     * Because we include the sign in the length byte:
+     * - unlike fixed-length ints, we don't need to sign-invert the first significant byte,
+     * - unlike BigInteger, we don't need to include 0x00 prefix for positive integers whose first byte is >= 0x80
+     *   or 0xFF prefix for negative integers whose first byte is < 0x80.
+     *
+     * The representations are prefix-free, because representations of different length always have length bytes that
+     * differ.
+     *
+     * Examples:
+     *    0             as 8000
+     *    1             as 8001
+     *    127           as 807F
+     *    255           as 80FF
+     *    2^31-1        as 837FFFFFFF
+     *    2^31          as 8380000000
+     *    2^32          as 840100000000
+     */
+    private <V> ByteSource asComparableBytesLegacy(ValueAccessor<V> accessor, V data, int startpos, int limit, int signbyte)
+    {
+        return new ByteSource()
+        {
+            int pos = startpos;
+            int sizeToReport = limit - startpos;
+            boolean sizeReported = false;
+
+            public int next()
+            {
+                if (!sizeReported)
+                {
+                    if (sizeToReport >= 128)
+                    {
+                        sizeToReport -= 128;
+                        return signbyte >= 0
+                               ? POSITIVE_VARINT_LENGTH_HEADER
+                               : NEGATIVE_VARINT_LENGTH_HEADER;
+                    }
+                    else
+                    {
+                        sizeReported = true;
+                        return signbyte >= 0
+                               ? POSITIVE_VARINT_HEADER + (sizeToReport - 1)
+                               : POSITIVE_VARINT_HEADER - sizeToReport;
+                    }
+                }
+
+                if (pos == limit)
+                    return END_OF_STREAM;
+
+                return accessor.getByte(data, pos++) & 0xFF;
+            }
+        };
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        assert version != ByteComparable.Version.LEGACY;
+        if (comparableBytes == null)
+            return accessor.empty();
+
+        // Consume the first byte to determine whether the encoded number is positive and
+        // start iterating through the length header bytes and collecting the number of value bytes.
+        int sign = comparableBytes.peek() ^ 0xFF;   // FF if negative, 00 if positive
+        if (sign != 0xFF && sign != 0x00)
+            return extractVarIntBytes(accessor, ByteSourceInverse.getVariableLengthInteger(comparableBytes));
+
+        // consume the sign byte
+        comparableBytes.next();
+
+        // Read the length (inverted if the number is negative)
+        int valueBytes = Math.toIntExact(ByteSourceInverse.getVariableLengthUnsignedIntegerXoring(comparableBytes, sign) + FULL_FORM_THRESHOLD);
+        // Get the bytes.
+        return extractBytes(accessor, comparableBytes, sign, valueBytes);
+    }
+
+    private <V> V extractVarIntBytes(ValueAccessor<V> accessor, long value)
+    {
+        int length = (64 - Long.numberOfLeadingZeros(value ^ (value >> 63)) + 8) / 8;   // number of bytes needed: 7 bits -> one byte, 8 bits -> 2 bytes
+        V buf = accessor.allocate(length);
+        switch (length)
+        {
+            case 1:
+                accessor.putByte(buf, 0, (byte) value);
+                break;
+            case 2:
+                accessor.putShort(buf, 0, (short) value);
+                break;
+            case 3:
+                accessor.putShort(buf, 0, (short) (value >> 8));
+                accessor.putByte(buf, 2, (byte) value);
+                break;
+            case 4:
+                accessor.putInt(buf, 0, (int) value);
+                break;
+            case 5:
+                accessor.putInt(buf, 0, (int) (value >> 8));
+                accessor.putByte(buf, 4, (byte) value);
+                break;
+            case 6:
+                accessor.putInt(buf, 0, (int) (value >> 16));
+                accessor.putShort(buf, 4, (short) value);
+                break;
+            case 7:
+                accessor.putInt(buf, 0, (int) (value >> 24));
+                accessor.putShort(buf, 4, (short) (value >> 8));
+                accessor.putByte(buf, 6, (byte) value);
+                break;
+            case 8:
+                // This is not reachable within the encoding; added for completeness.
+                accessor.putLong(buf, 0, value);
+                break;
+            default:
+                throw new AssertionError();
+        }
+        return buf;
+    }
+
+    private <V> V extractBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, int sign, int valueBytes)
+    {
+        int writtenBytes = 0;
+        V buf;
+        // Add "leading zero" if needed (i.e. in case the leading byte of a positive number corresponds to a negative
+        // value, or in case the leading byte of a negative number corresponds to a non-negative value).
+        // Size the array containing all the value bytes accordingly.
+        int curr = comparableBytes.next();
+        if ((curr & 0x80) != (sign & 0x80))
+        {
+            ++valueBytes;
+            buf = accessor.allocate(valueBytes);
+            accessor.putByte(buf, writtenBytes++, (byte) sign);
+        }
+        else
+            buf = accessor.allocate(valueBytes);
+        // Don't forget to add the first consumed value byte after determining whether leading zero should be added
+        // and sizing the value bytes array.
+        accessor.putByte(buf, writtenBytes++, (byte) curr);
+
+        // Consume exactly the number of expected value bytes.
+        while (writtenBytes < valueBytes)
+            accessor.putByte(buf, writtenBytes++, (byte) comparableBytes.next());
+
+        return buf;
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
         // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
index 6dd41616f0..81ec9d9a56 100644
--- a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
@@ -26,6 +26,9 @@ import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.UUIDSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 public class LexicalUUIDType extends AbstractType<UUID>
 {
@@ -48,6 +51,46 @@ public class LexicalUUIDType extends AbstractType<UUID>
         return accessorL.toUUID(left).compareTo(accessorR.toUUID(right));
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        if (data == null || accessor.isEmpty(data))
+            return null;
+
+        // fixed-length (hence prefix-free) representation, but
+        // we have to sign-flip the highest bytes of the two longs
+        return new ByteSource()
+        {
+            int bufpos = 0;
+
+            public int next()
+            {
+                if (bufpos >= accessor.size(data))
+                    return END_OF_STREAM;
+                int v = accessor.getByte(data, bufpos) & 0xFF;
+                if (bufpos == 0 || bufpos == 8)
+                    v ^= 0x80;
+                ++bufpos;
+                return v;
+            }
+        };
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        // Optional-style encoding of empty values as null sources
+        if (comparableBytes == null)
+            return accessor.empty();
+
+        long hiBits = ByteSourceInverse.getSignedLong(comparableBytes);
+        long loBits = ByteSourceInverse.getSignedLong(comparableBytes);
+
+        // Lexical UUIDs are stored as just two signed longs. The decoding of these longs flips their sign bit back, so
+        // they can directly be used for constructing the original UUID.
+        return UUIDType.makeUuidBytes(accessor, hiBits, loBits);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
         // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/ListType.java b/src/java/org/apache/cassandra/db/marshal/ListType.java
index 281f7ee4cb..f795def3a7 100644
--- a/src/java/org/apache/cassandra/db/marshal/ListType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ListType.java
@@ -18,21 +18,24 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.cassandra.cql3.Json;
 import org.apache.cassandra.cql3.Lists;
 import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.serializers.CollectionSerializer;
 import org.apache.cassandra.serializers.ListSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.TimeUUID;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 
 public class ListType<T> extends CollectionType<List<T>>
 {
@@ -171,29 +174,16 @@ public class ListType<T> extends CollectionType<List<T>>
         return compareListOrSet(elements, left, accessorL, right, accessorR);
     }
 
-    static <VL, VR> int compareListOrSet(AbstractType<?> elementsComparator, VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, Version version)
     {
-        // Note that this is only used if the collection is frozen
-        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
-            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
-
-        int sizeL = CollectionSerializer.readCollectionSize(left, accessorL, ProtocolVersion.V3);
-        int offsetL = CollectionSerializer.sizeOfCollectionSize(sizeL, ProtocolVersion.V3);
-        int sizeR = CollectionSerializer.readCollectionSize(right, accessorR, ProtocolVersion.V3);
-        int offsetR = TypeSizes.INT_SIZE;
-
-        for (int i = 0; i < Math.min(sizeL, sizeR); i++)
-        {
-            VL v1 = CollectionSerializer.readValue(left, accessorL, offsetL, ProtocolVersion.V3);
-            offsetL += CollectionSerializer.sizeOfValue(v1, accessorL, ProtocolVersion.V3);
-            VR v2 = CollectionSerializer.readValue(right, accessorR, offsetR, ProtocolVersion.V3);
-            offsetR += CollectionSerializer.sizeOfValue(v2, accessorR, ProtocolVersion.V3);
-            int cmp = elementsComparator.compare(v1, accessorL, v2, accessorR);
-            if (cmp != 0)
-                return cmp;
-        }
+        return asComparableBytesListOrSet(getElementsType(), accessor, data, version);
+    }
 
-        return sizeL == sizeR ? 0 : (sizeL < sizeR ? -1 : 1);
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, Version version)
+    {
+        return fromComparableBytesListOrSet(accessor, comparableBytes, version, getElementsType());
     }
 
     @Override
@@ -242,23 +232,6 @@ public class ListType<T> extends CollectionType<List<T>>
         return new Lists.DelayedValue(terms);
     }
 
-    public static String setOrListToJsonString(ByteBuffer buffer, AbstractType elementsType, ProtocolVersion protocolVersion)
-    {
-        ByteBuffer value = buffer.duplicate();
-        StringBuilder sb = new StringBuilder("[");
-        int size = CollectionSerializer.readCollectionSize(value, protocolVersion);
-        int offset = CollectionSerializer.sizeOfCollectionSize(size, protocolVersion);
-        for (int i = 0; i < size; i++)
-        {
-            if (i > 0)
-                sb.append(", ");
-            ByteBuffer element = CollectionSerializer.readValue(value, ByteBufferAccessor.instance, offset, protocolVersion);
-            offset += CollectionSerializer.sizeOfValue(element, ByteBufferAccessor.instance, protocolVersion);
-            sb.append(elementsType.toJSONString(element, protocolVersion));
-        }
-        return sb.append("]").toString();
-    }
-
     public ByteBuffer getSliceFromSerialized(ByteBuffer collection, ByteBuffer from, ByteBuffer to)
     {
         // We don't support slicing on lists so we don't need that function
diff --git a/src/java/org/apache/cassandra/db/marshal/LongType.java b/src/java/org/apache/cassandra/db/marshal/LongType.java
index ad539f70de..6bf5e9e669 100644
--- a/src/java/org/apache/cassandra/db/marshal/LongType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LongType.java
@@ -28,6 +28,9 @@ import org.apache.cassandra.serializers.LongSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 public class LongType extends NumberType<Long>
 {
@@ -57,6 +60,28 @@ public class LongType extends NumberType<Long>
         return ValueAccessor.compare(left, accessorL, right, accessorR);
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        if (accessor.isEmpty(data))
+            return null;
+        if (version == ByteComparable.Version.LEGACY)
+            return ByteSource.signedFixedLengthNumber(accessor, data);
+        else
+            return ByteSource.variableLengthInteger(accessor.getLong(data, 0));
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        if (comparableBytes == null)
+            return accessor.empty();
+        if (version == ByteComparable.Version.LEGACY)
+            return ByteSourceInverse.getSignedFixedLength(accessor, comparableBytes, 8);
+        else
+            return accessor.valueOf(ByteSourceInverse.getVariableLengthInteger(comparableBytes));
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
         // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/MapType.java b/src/java/org/apache/cassandra/db/marshal/MapType.java
index 9473e29136..be74ff1626 100644
--- a/src/java/org/apache/cassandra/db/marshal/MapType.java
+++ b/src/java/org/apache/cassandra/db/marshal/MapType.java
@@ -28,9 +28,13 @@ import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.CollectionSerializer;
-import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.MapSerializer;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 import org.apache.cassandra.utils.Pair;
 
 public class MapType<K, V> extends CollectionType<Map<K, V>>
@@ -215,7 +219,71 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
                 return cmp;
         }
 
-        return sizeL == sizeR ? 0 : (sizeL < sizeR ? -1 : 1);
+        return Integer.compare(sizeL, sizeR);
+    }
+
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, Version version)
+    {
+        return asComparableBytesMap(getKeysType(), getValuesType(), accessor, data, version);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, Version version)
+    {
+        return fromComparableBytesMap(accessor, comparableBytes, version, getKeysType(), getValuesType());
+    }
+
+    static <V> ByteSource asComparableBytesMap(AbstractType<?> keysComparator,
+                                               AbstractType<?> valuesComparator,
+                                               ValueAccessor<V> accessor,
+                                               V data,
+                                               Version version)
+    {
+        if (accessor.isEmpty(data))
+            return null;
+
+        ProtocolVersion protocolVersion = ProtocolVersion.V3;
+        int offset = 0;
+        int size = CollectionSerializer.readCollectionSize(data, accessor, protocolVersion);
+        offset += CollectionSerializer.sizeOfCollectionSize(size, protocolVersion);
+        ByteSource[] srcs = new ByteSource[size * 2];
+        for (int i = 0; i < size; ++i)
+        {
+            V k = CollectionSerializer.readValue(data, accessor, offset, protocolVersion);
+            offset += CollectionSerializer.sizeOfValue(k, accessor, protocolVersion);
+            srcs[i * 2 + 0] = keysComparator.asComparableBytes(accessor, k, version);
+            V v = CollectionSerializer.readValue(data, accessor, offset, protocolVersion);
+            offset += CollectionSerializer.sizeOfValue(v, accessor, protocolVersion);
+            srcs[i * 2 + 1] = valuesComparator.asComparableBytes(accessor, v, version);
+        }
+        return ByteSource.withTerminatorMaybeLegacy(version, 0x00, srcs);
+    }
+
+    static <V> V fromComparableBytesMap(ValueAccessor<V> accessor,
+                                        ByteSource.Peekable comparableBytes,
+                                        Version version,
+                                        AbstractType<?> keysComparator,
+                                        AbstractType<?> valuesComparator)
+    {
+        if (comparableBytes == null)
+            return accessor.empty();
+        assert version != ByteComparable.Version.LEGACY; // legacy translation is not reversible
+
+        List<V> buffers = new ArrayList<>();
+        int separator = comparableBytes.next();
+        while (separator != ByteSource.TERMINATOR)
+        {
+            buffers.add(ByteSourceInverse.nextComponentNull(separator)
+                        ? null
+                        : keysComparator.fromComparableBytes(accessor, comparableBytes, version));
+            separator = comparableBytes.next();
+            buffers.add(ByteSourceInverse.nextComponentNull(separator)
+                        ? null
+                        : valuesComparator.fromComparableBytes(accessor, comparableBytes, version));
+            separator = comparableBytes.next();
+        }
+        return CollectionSerializer.pack(buffers, accessor,buffers.size() / 2, ProtocolVersion.V3);
     }
 
     @Override
@@ -286,7 +354,7 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
     {
         ByteBuffer value = buffer.duplicate();
         StringBuilder sb = new StringBuilder("{");
-        int size = CollectionSerializer.readCollectionSize(value, protocolVersion);
+        int size = CollectionSerializer.readCollectionSize(value, ByteBufferAccessor.instance, protocolVersion);
         int offset = CollectionSerializer.sizeOfCollectionSize(size, protocolVersion);
         for (int i = 0; i < size; i++)
         {
diff --git a/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java b/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java
index 89241b416b..02c28e7513 100644
--- a/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java
+++ b/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java
@@ -22,11 +22,15 @@ import java.util.Iterator;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 import org.apache.cassandra.utils.FBUtilities;
 
 /** for sorting columns representing row keys in the row ordering as determined by a partitioner.
@@ -93,6 +97,33 @@ public class PartitionerDefinedOrder extends AbstractType<ByteBuffer>
         return PartitionPosition.ForKey.get(accessorL.toBuffer(left), partitioner).compareTo(PartitionPosition.ForKey.get(accessorR.toBuffer(right), partitioner));
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, Version version)
+    {
+        // Partitioners work with ByteBuffers only.
+        ByteBuffer buf = ByteBufferAccessor.instance.convert(data, accessor);
+        if (version != Version.LEGACY)
+        {
+            // For ByteComparable.Version.OSS42 and above we encode an empty key with a null byte source. This
+            // way we avoid the need to special-handle a sentinel value when we decode the byte source for such a key
+            // (e.g. for ByteComparable.Version.Legacy we use the minimum key bound of the partitioner's minimum token as
+            // a sentinel value, and that results in the need to go twice through the byte source that is being
+            // decoded).
+            return buf.hasRemaining() ? partitioner.decorateKey(buf).asComparableBytes(version) : null;
+        }
+        return PartitionPosition.ForKey.get(buf, partitioner).asComparableBytes(version);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        assert version != Version.LEGACY;
+        if (comparableBytes == null)
+            return accessor.empty();
+        byte[] keyBytes = DecoratedKey.keyFromByteSource(comparableBytes, version, partitioner);
+        return accessor.valueOf(keyBytes);
+    }
+
     @Override
     public void validate(ByteBuffer bytes) throws MarshalException
     {
diff --git a/src/java/org/apache/cassandra/db/marshal/ReversedType.java b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
index ceea84a39f..eac800aec4 100644
--- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
@@ -28,6 +28,8 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 
 public class ReversedType<T> extends AbstractType<T>
 {
@@ -63,6 +65,32 @@ public class ReversedType<T> extends AbstractType<T>
         return baseType.isEmptyValueMeaningless();
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        ByteSource src = baseType.asComparableBytes(accessor, data, version);
+        if (src == null)    // Note: this will only compare correctly if used within a sequence
+            return null;
+        // Invert all bytes.
+        // The comparison requirements for the original type ensure that this encoding will compare correctly with
+        // respect to the reversed comparator function (and, specifically, prefixes of escaped byte-ordered types will
+        // compare as larger). Additionally, the weak prefix-freedom requirement ensures this encoding will also be
+        // weakly prefix-free.
+        return () ->
+        {
+            int v = src.next();
+            if (v == ByteSource.END_OF_STREAM)
+                return v;
+            return v ^ 0xFF;
+        };
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return baseType.fromComparableBytes(accessor, ReversedPeekableByteSource.of(comparableBytes), version);
+    }
+
     public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
         return baseType.compare(right, accessorR, left, accessorL);
@@ -156,4 +184,38 @@ public class ReversedType<T> extends AbstractType<T>
     {
         return getClass().getName() + "(" + baseType + ")";
     }
+
+    private static final class ReversedPeekableByteSource extends ByteSource.Peekable
+    {
+        private final ByteSource.Peekable original;
+
+        static ByteSource.Peekable of(ByteSource.Peekable original)
+        {
+            return original != null ? new ReversedPeekableByteSource(original) : null;
+        }
+
+        private ReversedPeekableByteSource(ByteSource.Peekable original)
+        {
+            super(null);
+            this.original = original;
+        }
+
+        @Override
+        public int next()
+        {
+            int v = original.next();
+            if (v != END_OF_STREAM)
+                return v ^ 0xFF;
+            return END_OF_STREAM;
+        }
+
+        @Override
+        public int peek()
+        {
+            int v = original.peek();
+            if (v != END_OF_STREAM)
+                return v ^ 0xFF;
+            return END_OF_STREAM;
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/marshal/SetType.java b/src/java/org/apache/cassandra/db/marshal/SetType.java
index e5bdadab25..67699ac3da 100644
--- a/src/java/org/apache/cassandra/db/marshal/SetType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SetType.java
@@ -30,6 +30,8 @@ import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.SetSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 
 public class SetType<T> extends CollectionType<Set<T>>
 {
@@ -154,7 +156,19 @@ public class SetType<T> extends CollectionType<Set<T>>
 
     public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        return ListType.compareListOrSet(elements, left, accessorL, right, accessorR);
+        return compareListOrSet(elements, left, accessorL, right, accessorR);
+    }
+
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        return asComparableBytesListOrSet(getElementsType(), accessor, data, version);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return fromComparableBytesListOrSet(accessor, comparableBytes, version, getElementsType());
     }
 
     public SetSerializer<T> getSerializer()
@@ -210,6 +224,6 @@ public class SetType<T> extends CollectionType<Set<T>>
     @Override
     public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion)
     {
-        return ListType.setOrListToJsonString(buffer, elements, protocolVersion);
+        return setOrListToJsonString(buffer, elements, protocolVersion);
     }
 }
diff --git a/src/java/org/apache/cassandra/db/marshal/ShortType.java b/src/java/org/apache/cassandra/db/marshal/ShortType.java
index 03dcf5d314..013fa95949 100644
--- a/src/java/org/apache/cassandra/db/marshal/ShortType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ShortType.java
@@ -28,6 +28,9 @@ import org.apache.cassandra.serializers.ShortSerializer;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 public class ShortType extends NumberType<Short>
 {
@@ -46,6 +49,19 @@ public class ShortType extends NumberType<Short>
         return ValueAccessor.compare(left, accessorL, right, accessorR);
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        // This type does not allow non-present values, but we do just to avoid future complexity.
+        return ByteSource.optionalSignedFixedLengthNumber(accessor, data);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return ByteSourceInverse.getOptionalSignedFixedLength(accessor, comparableBytes, 2);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
         // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
index 8f1d677f03..a0de2c2089 100644
--- a/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
@@ -28,6 +28,10 @@ import org.apache.cassandra.serializers.SimpleDateSerializer;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 
@@ -37,6 +41,20 @@ public class SimpleDateType extends TemporalType<Integer>
 
     SimpleDateType() {super(ComparisonType.BYTE_ORDER);} // singleton
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, Version version)
+    {
+        // While BYTE_ORDER would still work for this type, making use of the fixed length is more efficient.
+        // This type does not allow non-present values, but we do just to avoid future complexity.
+        return ByteSource.optionalFixedLength(accessor, data);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return ByteSourceInverse.getOptionalFixedLength(accessor, comparableBytes, 4);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
         return ByteBufferUtil.bytes(SimpleDateSerializer.dateStringToDays(source));
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeType.java b/src/java/org/apache/cassandra/db/marshal/TimeType.java
index fd8fac4745..f029b8bb94 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeType.java
@@ -28,6 +28,10 @@ import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 /**
  * Nanosecond resolution time values
@@ -42,6 +46,20 @@ public class TimeType extends TemporalType<Long>
         return decompose(TimeSerializer.timeStringToLong(source));
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, Version version)
+    {
+        // While BYTE_ORDER would still work for this type, making use of the fixed length is more efficient.
+        // This type does not allow non-present values, but we do just to avoid future complexity.
+        return ByteSource.optionalFixedLength(accessor, data);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return ByteSourceInverse.getOptionalFixedLength(accessor, comparableBytes, 8);
+    }
+
     @Override
     public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
     {
diff --git a/src/java/org/apache/cassandra/db/marshal/TimestampType.java b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
index ccf1da3e20..5bca7b1f56 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimestampType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
@@ -32,6 +32,9 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TimestampSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+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.cassandra.cql3.statements.RequestValidations.invalidRequest;
 
@@ -60,6 +63,18 @@ public class TimestampType extends TemporalType<Date>
         return LongType.compareLongs(left, accessorL, right, accessorR);
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        return ByteSource.optionalSignedFixedLengthNumber(accessor, data);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        return ByteSourceInverse.getOptionalSignedFixedLength(accessor, comparableBytes, 8);
+    }
+
     public ByteBuffer fromString(String source) throws MarshalException
     {
       // Return an empty ByteBuffer for an empty string.
diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java
index cc08487658..c203770bc7 100644
--- a/src/java/org/apache/cassandra/db/marshal/TupleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java
@@ -30,11 +30,12 @@ import com.google.common.collect.Lists;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.*;
 import org.apache.cassandra.transport.ProtocolVersion;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 
 import static com.google.common.collect.Iterables.any;
 import static com.google.common.collect.Iterables.transform;
@@ -200,47 +201,136 @@ public class TupleType extends AbstractType<ByteBuffer>
         return true;
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        switch (version)
+        {
+            case LEGACY:
+                return asComparableBytesLegacy(accessor, data);
+            case OSS42:
+                return asComparableBytesNew(accessor, data, version);
+            default:
+                throw new AssertionError();
+        }
+    }
+
+    private <V> ByteSource asComparableBytesLegacy(ValueAccessor<V> accessor, V data)
+    {
+        if (accessor.isEmpty(data))
+            return null;
+
+        V[] bufs = split(accessor, data);  // this may be shorter than types.size -- other srcs remain null in that case
+        ByteSource[] srcs = new ByteSource[types.size()];
+        for (int i = 0; i < bufs.length; ++i)
+            srcs[i] = bufs[i] != null ? types.get(i).asComparableBytes(accessor, bufs[i], ByteComparable.Version.LEGACY) : null;
+
+        // We always have a fixed number of sources, with the trailing ones possibly being nulls.
+        // This can only result in a prefix if the last type in the tuple allows prefixes. Since that type is required
+        // to be weakly prefix-free, so is the tuple.
+        return ByteSource.withTerminatorLegacy(ByteSource.END_OF_STREAM, srcs);
+    }
+
+    private <V> ByteSource asComparableBytesNew(ValueAccessor<V> accessor, V data, ByteComparable.Version version)
+    {
+        if (accessor.isEmpty(data))
+            return null;
+
+        V[] bufs = split(accessor, data);
+        int lengthWithoutTrailingNulls = 0;
+        for (int i = 0; i < bufs.length; ++i)
+            if (bufs[i] != null)
+                lengthWithoutTrailingNulls = i + 1;
+
+        ByteSource[] srcs = new ByteSource[lengthWithoutTrailingNulls];
+        for (int i = 0; i < lengthWithoutTrailingNulls; ++i)
+            srcs[i] = bufs[i] != null ? types.get(i).asComparableBytes(accessor, bufs[i], version) : null;
+
+        // Because we stop early when there are trailing nulls, there needs to be an explicit terminator to make the
+        // type prefix-free.
+        return ByteSource.withTerminator(ByteSource.TERMINATOR, srcs);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        assert version == ByteComparable.Version.OSS42; // Reverse translation is not supported for the legacy version.
+        if (comparableBytes == null)
+            return accessor.empty();
+
+        V[] componentBuffers = accessor.createArray(types.size());
+        for (int i = 0; i < types.size(); ++i)
+        {
+            if (comparableBytes.peek() == ByteSource.TERMINATOR)
+                break;  // the rest of the fields remain null
+            AbstractType<?> componentType = types.get(i);
+            ByteSource.Peekable component = ByteSourceInverse.nextComponentSource(comparableBytes);
+            if (component != null)
+                componentBuffers[i] = componentType.fromComparableBytes(accessor, component, version);
+            else
+                componentBuffers[i] = null;
+        }
+        // consume terminator
+        int terminator = comparableBytes.next();
+        assert terminator == ByteSource.TERMINATOR : String.format("Expected TERMINATOR (0x%2x) after %d components",
+                                                                   ByteSource.TERMINATOR,
+                                                                   types.size());
+        return buildValue(accessor, componentBuffers);
+    }
+
     /**
      * Split a tuple value into its component values.
      */
-    public ByteBuffer[] split(ByteBuffer value)
+    public <V> V[] split(ValueAccessor<V> accessor, V value)
     {
-        return split(value, size(), this);
+        return split(accessor, value, size(), this);
     }
 
     /**
      * Split a tuple value into its component values.
      */
-    public static ByteBuffer[] split(ByteBuffer value, int numberOfElements, TupleType type)
+    public static <V> V[] split(ValueAccessor<V> accessor, V value, int numberOfElements, TupleType type)
     {
-        ByteBuffer[] components = new ByteBuffer[numberOfElements];
-        ByteBuffer input = value.duplicate();
+        V[] components = accessor.createArray(numberOfElements);
+        int length = accessor.size(value);
+        int position = 0;
         for (int i = 0; i < numberOfElements; i++)
         {
-            if (!input.hasRemaining())
+            if (position == length)
                 return Arrays.copyOfRange(components, 0, i);
 
-            int size = input.getInt();
-
-            if (input.remaining() < size)
+            if (position + 4 > length)
                 throw new MarshalException(String.format("Not enough bytes to read %dth component", i));
 
+            int size = accessor.getInt(value, position);
+            position += 4;
+
             // size < 0 means null value
-            components[i] = size < 0 ? null : ByteBufferUtil.readBytes(input, size);
+            if (size >= 0)
+            {
+                if (position + size > length)
+                    throw new MarshalException(String.format("Not enough bytes to read %dth component", i));
+
+                components[i] = accessor.slice(value, position, size);
+                position += size;
+            }
+            else
+                components[i] = null;
         }
 
         // error out if we got more values in the tuple/UDT than we expected
-        if (input.hasRemaining())
+        if (position < length)
         {
-            throw new InvalidRequestException(String.format(
-            "Expected %s %s for %s column, but got more",
-            numberOfElements, numberOfElements == 1 ? "value" : "values", type.asCQL3Type()));
+            throw new MarshalException(String.format("Expected %s %s for %s column, but got more",
+                                                     numberOfElements, numberOfElements == 1 ? "value" : "values",
+                                                     type.asCQL3Type()));
         }
 
         return components;
     }
 
-    public static <V> V buildValue(ValueAccessor<V> accessor, V[] components)
+    @SafeVarargs
+    public static <V> V buildValue(ValueAccessor<V> accessor, V... components)
     {
         int totalLength = 0;
         for (V component : components)
@@ -264,7 +354,7 @@ public class TupleType extends AbstractType<ByteBuffer>
         return result;
     }
 
-    public static ByteBuffer buildValue(ByteBuffer[] components)
+    public static ByteBuffer buildValue(ByteBuffer... components)
     {
         return buildValue(ByteBufferAccessor.instance, components);
     }
diff --git a/src/java/org/apache/cassandra/db/marshal/UUIDType.java b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
index 55ce59dae7..9ec8063fae 100644
--- a/src/java/org/apache/cassandra/db/marshal/UUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
@@ -30,6 +30,9 @@ import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.UUIDSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 import org.apache.cassandra.utils.UUIDGen;
 
 /**
@@ -96,9 +99,71 @@ public class UUIDType extends AbstractType<UUID>
                 return c;
         }
 
+        // Amusingly (or not so much), although UUIDType freely takes time UUIDs (UUIDs with version 1), it compares
+        // them differently than TimeUUIDType. This is evident in the least significant bytes comparison (the code
+        // below for UUIDType), where UUIDType treats them as unsigned bytes, while TimeUUIDType compares the bytes
+        // signed. See CASSANDRA-8730 for details around this discrepancy.
         return UnsignedLongs.compare(accessorL.getLong(left, 8), accessorR.getLong(right, 8));
     }
 
+    @Override
+    public <V> ByteSource asComparableBytes(ValueAccessor<V> accessor, V data, ByteComparable.Version v)
+    {
+        if (accessor.isEmpty(data))
+            return null;
+
+        long msb = accessor.getLong(data, 0);
+        long version = ((msb >>> 12) & 0xf);
+        ByteBuffer swizzled = ByteBuffer.allocate(16);
+
+        if (version == 1)
+            swizzled.putLong(0, TimeUUIDType.reorderTimestampBytes(msb));
+        else
+            swizzled.putLong(0, (version << 60) | ((msb >>> 4) & 0x0FFFFFFFFFFFF000L) | (msb & 0xFFFL));
+
+        swizzled.putLong(8, accessor.getLong(data, 8));
+
+        // fixed-length thus prefix-free
+        return ByteSource.fixedLength(swizzled);
+    }
+
+    @Override
+    public <V> V fromComparableBytes(ValueAccessor<V> accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+    {
+        // Optional-style encoding of empty values as null sources
+        if (comparableBytes == null)
+            return accessor.empty();
+
+        // The UUID bits are stored as an unsigned fixed-length 128-bit integer.
+        long hiBits = ByteSourceInverse.getUnsignedFixedLengthAsLong(comparableBytes, 8);
+        long loBits = ByteSourceInverse.getUnsignedFixedLengthAsLong(comparableBytes, 8);
+
+        long uuidVersion = hiBits >>> 60 & 0xF;
+        if (uuidVersion == 1)
+        {
+            // If the version bits are set to 1, this is a time-based UUID, and its high bits are significantly more
+            // shuffled than in other UUIDs. Revert the shuffle.
+            hiBits = TimeUUIDType.reorderBackTimestampBytes(hiBits);
+        }
+        else
+        {
+            // For non-time UUIDs, the only thing that's needed is to put the version bits back where they were originally.
+            hiBits = hiBits << 4 & 0xFFFFFFFFFFFF0000L
+                     | uuidVersion << 12
+                     | hiBits & 0x0000000000000FFFL;
+        }
+
+        return makeUuidBytes(accessor, hiBits, loBits);
+    }
+
+    static <V> V makeUuidBytes(ValueAccessor<V> accessor, long high, long low)
+    {
+        V buffer = accessor.allocate(16);
+        accessor.putLong(buffer, 0, high);
+        accessor.putLong(buffer, 8, low);
+        return buffer;
+    }
+
     @Override
     public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
     {
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index 29afad9583..24c05e2559 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -258,7 +258,7 @@ public class UserType extends TupleType implements SchemaElement
     @Override
     public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion)
     {
-        ByteBuffer[] buffers = split(buffer);
+        ByteBuffer[] buffers = split(ByteBufferAccessor.instance, buffer);
         StringBuilder sb = new StringBuilder("{");
         for (int i = 0; i < types.size(); i++)
         {
diff --git a/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java b/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java
index a51836e65a..d454c5e188 100644
--- a/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java
+++ b/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java
@@ -68,6 +68,7 @@ public interface ValueAccessor<V>
         Cell<V> cell(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, V value, CellPath path);
         Clustering<V> clustering(V... values);
         Clustering<V> clustering();
+        Clustering<V> staticClustering();
         ClusteringBound<V> bound(ClusteringPrefix.Kind kind, V... values);
         ClusteringBound<V> bound(ClusteringPrefix.Kind kind);
         ClusteringBoundary<V> boundary(ClusteringPrefix.Kind kind, V... values);
@@ -105,7 +106,6 @@ public interface ValueAccessor<V>
         {
             return boundary(reversed ? INCL_END_EXCL_START_BOUNDARY : EXCL_END_INCL_START_BOUNDARY, boundValues);
         }
-
     }
     /**
      * @return the size of the given value
@@ -330,6 +330,12 @@ public interface ValueAccessor<V>
     /** returns a TimeUUID from offset 0 */
     Ballot toBallot(V value);
 
+    /**
+     * writes the byte value {@param value} to {@param dst} at offset {@param offset}
+     * @return the number of bytes written to {@param value}
+     */
+    int putByte(V dst, int offset, byte value);
+
     /**
      * writes the short value {@param value} to {@param dst} at offset {@param offset}
      * @return the number of bytes written to {@param value}
diff --git a/src/java/org/apache/cassandra/db/rows/EncodingStats.java b/src/java/org/apache/cassandra/db/rows/EncodingStats.java
index 37dd34e921..518285d671 100644
--- a/src/java/org/apache/cassandra/db/rows/EncodingStats.java
+++ b/src/java/org/apache/cassandra/db/rows/EncodingStats.java
@@ -67,7 +67,7 @@ public class EncodingStats implements IMeasurableMemory
 
     // We should use this sparingly obviously
     public static final EncodingStats NO_STATS = new EncodingStats(TIMESTAMP_EPOCH, DELETION_TIME_EPOCH, TTL_EPOCH);
-    public static long HEAP_SIZE = ObjectSizes.measure(NO_STATS);
+    public static final long HEAP_SIZE = ObjectSizes.measure(NO_STATS);
 
     public static final Serializer serializer = new Serializer();
 
diff --git a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
index 3a5db52bd7..2b0e2a2861 100644
--- a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
@@ -26,6 +26,9 @@ import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Hex;
 import org.apache.cassandra.utils.ObjectSizes;
@@ -101,6 +104,12 @@ public class ByteOrderedPartitioner implements IPartitioner
             return Arrays.equals(token, other.token);
         }
 
+        @Override
+        public ByteSource asComparableBytes(ByteComparable.Version version)
+        {
+            return ByteSource.of(token, version);
+        }
+
         @Override
         public IPartitioner getPartitioner()
         {
@@ -222,6 +231,11 @@ public class ByteOrderedPartitioner implements IPartitioner
 
     private final Token.TokenFactory tokenFactory = new Token.TokenFactory()
     {
+        public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+        {
+            return new BytesToken(ByteSourceInverse.getUnescapedBytes(comparableBytes));
+        }
+
         public ByteBuffer toByteArray(Token token)
         {
             BytesToken bytesToken = (BytesToken) token;
diff --git a/src/java/org/apache/cassandra/dht/LocalPartitioner.java b/src/java/org/apache/cassandra/dht/LocalPartitioner.java
index c7c6df0d73..df976701aa 100644
--- a/src/java/org/apache/cassandra/dht/LocalPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/LocalPartitioner.java
@@ -26,7 +26,10 @@ import java.util.Random;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.CachedHashDecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.HeapAllocator;
 
@@ -83,6 +86,12 @@ public class LocalPartitioner implements IPartitioner
 
     private final Token.TokenFactory tokenFactory = new Token.TokenFactory()
     {
+        public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+        {
+            ByteBuffer tokenData = comparator.fromComparableBytes(ByteBufferAccessor.instance, comparableBytes, version);
+            return new LocalToken(tokenData);
+        }
+
         public ByteBuffer toByteArray(Token token)
         {
             return ((LocalToken)token).token;
@@ -174,6 +183,12 @@ public class LocalPartitioner implements IPartitioner
             return token.equals(other.token);
         }
 
+        @Override
+        public ByteSource asComparableBytes(ByteComparable.Version version)
+        {
+            return comparator.asComparableBytes(ByteBufferAccessor.instance, token, version);
+        }
+
         @Override
         public IPartitioner getPartitioner()
         {
diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
index e2daac412c..015610fb53 100644
--- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
+++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
@@ -33,6 +33,9 @@ import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 import org.apache.cassandra.utils.MurmurHash;
 import org.apache.cassandra.utils.ObjectSizes;
 
@@ -176,6 +179,12 @@ public class Murmur3Partitioner implements IPartitioner
             return Long.compare(token, ((LongToken) o).token);
         }
 
+        @Override
+        public ByteSource asComparableBytes(ByteComparable.Version version)
+        {
+            return ByteSource.of(token);
+        }
+
         @Override
         public IPartitioner getPartitioner()
         {
@@ -326,6 +335,12 @@ public class Murmur3Partitioner implements IPartitioner
 
     private final Token.TokenFactory tokenFactory = new Token.TokenFactory()
     {
+        public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+        {
+            long tokenData = ByteSourceInverse.getSignedLong(comparableBytes);
+            return new LongToken(tokenData);
+        }
+
         public ByteBuffer toByteArray(Token token)
         {
             LongToken longToken = (LongToken) token;
diff --git a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
index 16c5db17a4..2d4def95d0 100644
--- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
@@ -33,6 +33,9 @@ import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.Pair;
@@ -128,6 +131,11 @@ public class OrderPreservingPartitioner implements IPartitioner
 
     private final Token.TokenFactory tokenFactory = new Token.TokenFactory()
     {
+        public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+        {
+            return new StringToken(ByteSourceInverse.getString(comparableBytes));
+        }
+
         public ByteBuffer toByteArray(Token token)
         {
             StringToken stringToken = (StringToken) token;
@@ -194,6 +202,12 @@ public class OrderPreservingPartitioner implements IPartitioner
         {
             return EMPTY_SIZE + ObjectSizes.sizeOf(token);
         }
+
+        @Override
+        public ByteSource asComparableBytes(ByteComparable.Version version)
+        {
+            return ByteSource.of(token, version);
+        }
     }
 
     public StringToken getToken(ByteBuffer key)
diff --git a/src/java/org/apache/cassandra/dht/RandomPartitioner.java b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
index 241b7850fd..d02cfd58ad 100644
--- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
@@ -27,6 +27,8 @@ import java.util.*;
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.db.CachedHashDecoratedKey;
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -34,6 +36,8 @@ import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.db.marshal.PartitionerDefinedOrder;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.GuidGenerator;
 import org.apache.cassandra.utils.ObjectSizes;
@@ -158,6 +162,11 @@ public class RandomPartitioner implements IPartitioner
 
     private final Token.TokenFactory tokenFactory = new Token.TokenFactory()
     {
+        public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+        {
+            return fromByteArray(IntegerType.instance.fromComparableBytes(ByteBufferAccessor.instance, comparableBytes, version));
+        }
+
         public ByteBuffer toByteArray(Token token)
         {
             BigIntegerToken bigIntegerToken = (BigIntegerToken) token;
@@ -244,6 +253,12 @@ public class RandomPartitioner implements IPartitioner
             this(new BigInteger(token));
         }
 
+        @Override
+        public ByteSource asComparableBytes(ByteComparable.Version version)
+        {
+            return IntegerType.instance.asComparableBytes(ByteArrayAccessor.instance, token.toByteArray(), version);
+        }
+
         @Override
         public IPartitioner getPartitioner()
         {
diff --git a/src/java/org/apache/cassandra/dht/Token.java b/src/java/org/apache/cassandra/dht/Token.java
index d8e82f82c5..3543dabc0e 100644
--- a/src/java/org/apache/cassandra/dht/Token.java
+++ b/src/java/org/apache/cassandra/dht/Token.java
@@ -26,6 +26,8 @@ import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 
 public abstract class Token implements RingPosition<Token>, Serializable
 {
@@ -37,8 +39,31 @@ public abstract class Token implements RingPosition<Token>, Serializable
     {
         public abstract ByteBuffer toByteArray(Token token);
         public abstract Token fromByteArray(ByteBuffer bytes);
+
+        /**
+         * Produce a byte-comparable representation of the token.
+         * See {@link Token#asComparableBytes}
+         */
+        public ByteSource asComparableBytes(Token token, ByteComparable.Version version)
+        {
+            return token.asComparableBytes(version);
+        }
+
+        /**
+         * Translates the given byte-comparable representation to a token instance. If the given bytes don't correspond
+         * to the encoding of an instance of the expected token type, an {@link IllegalArgumentException} may be thrown.
+         *
+         * @param comparableBytes A byte-comparable representation (presumably of a token of some expected token type).
+         * @return A new {@link Token} instance, corresponding to the given byte-ordered representation. If we were
+         * to call {@link #asComparableBytes(ByteComparable.Version)} on the returned object, we should get a
+         * {@link ByteSource} equal to the input one as a result.
+         * @throws IllegalArgumentException if the bytes do not encode a valid token.
+         */
+        public abstract Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version);
+
         public abstract String toString(Token token); // serialize as string, not necessarily human-readable
         public abstract Token fromString(String string); // deserialize
+
         public abstract void validate(String token) throws ConfigurationException;
 
         public void serialize(Token token, DataOutputPlus out) throws IOException
@@ -99,6 +124,20 @@ public abstract class Token implements RingPosition<Token>, Serializable
     abstract public long getHeapSize();
     abstract public Object getTokenValue();
 
+    /**
+     * Produce a weakly prefix-free byte-comparable representation of the token, i.e. such a sequence of bytes that any
+     * pair x, y of valid tokens of this type and any bytes b1, b2 between 0x10 and 0xEF,
+     * (+ stands for concatenation)
+     *   compare(x, y) == compareLexicographicallyUnsigned(asByteComparable(x)+b1, asByteComparable(y)+b2)
+     * (i.e. the values compare like the original type, and an added 0x10-0xEF byte at the end does not change that) and:
+     *   asByteComparable(x)+b1 is not a prefix of asByteComparable(y)      (weakly prefix free)
+     * (i.e. a valid representation of a value may be a prefix of another valid representation of a value only if the
+     * following byte in the latter is smaller than 0x10 or larger than 0xEF). These properties are trivially true if
+     * the encoding compares correctly and is prefix free, but also permits a little more freedom that enables somewhat
+     * more efficient encoding of arbitrary-length byte-comparable blobs.
+     */
+    abstract public ByteSource asComparableBytes(ByteComparable.Version version);
+
     /**
      * Returns a measure for the token space covered between this token and next.
      * Used by the token allocation algorithm (see CASSANDRA-7032).
@@ -128,7 +167,7 @@ public abstract class Token implements RingPosition<Token>, Serializable
 
     /*
      * A token corresponds to the range of all the keys having this token.
-     * A token is thus no comparable directly to a key. But to be able to select
+     * A token is thus not comparable directly to a key. But to be able to select
      * keys given tokens, we introduce two "fake" keys for each token T:
      *   - lowerBoundKey: a "fake" key representing the lower bound T represents.
      *                    In other words, lowerBoundKey is the smallest key that
@@ -190,6 +229,20 @@ public abstract class Token implements RingPosition<Token>, Serializable
                 return ((pos instanceof KeyBound) && !((KeyBound)pos).isMinimumBound) ? 0 : 1;
         }
 
+        @Override
+        public ByteSource asComparableBytes(Version version)
+        {
+            int terminator = isMinimumBound ? ByteSource.LT_NEXT_COMPONENT : ByteSource.GT_NEXT_COMPONENT;
+            return ByteSource.withTerminator(terminator, token.asComparableBytes(version));
+        }
+
+        @Override
+        public ByteComparable asComparableBound(boolean before)
+        {
+            // This class is already a bound thus nothing needs to be changed from its representation
+            return this;
+        }
+
         public IPartitioner getPartitioner()
         {
             return getToken().getPartitioner();
diff --git a/src/java/org/apache/cassandra/serializers/BooleanSerializer.java b/src/java/org/apache/cassandra/serializers/BooleanSerializer.java
index d372a2ad77..403e6b75b0 100644
--- a/src/java/org/apache/cassandra/serializers/BooleanSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/BooleanSerializer.java
@@ -24,8 +24,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class BooleanSerializer extends TypeSerializer<Boolean>
 {
-    private static final ByteBuffer TRUE = ByteBuffer.wrap(new byte[] {1});
-    private static final ByteBuffer FALSE = ByteBuffer.wrap(new byte[] {0});
+    public static final ByteBuffer TRUE = ByteBuffer.wrap(new byte[] {1});
+    public static final ByteBuffer FALSE = ByteBuffer.wrap(new byte[] {0});
 
     public static final BooleanSerializer instance = new BooleanSerializer();
 
diff --git a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
index eb2991b8d7..204261d46f 100644
--- a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
@@ -91,11 +91,6 @@ public abstract class CollectionSerializer<T> extends TypeSerializer<T>
         output.putInt(elements);
     }
 
-    public static int readCollectionSize(ByteBuffer input, ProtocolVersion version)
-    {
-        return readCollectionSize(input, ByteBufferAccessor.instance, version);
-    }
-
     public static <V> int readCollectionSize(V value, ValueAccessor<V> accessor, ProtocolVersion version)
     {
         return accessor.toInt(value);
diff --git a/src/java/org/apache/cassandra/serializers/MapSerializer.java b/src/java/org/apache/cassandra/serializers/MapSerializer.java
index 13468fc0aa..400a8e7cc5 100644
--- a/src/java/org/apache/cassandra/serializers/MapSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/MapSerializer.java
@@ -148,7 +148,7 @@ public class MapSerializer<K, V> extends CollectionSerializer<Map<K, V>>
         try
         {
             ByteBuffer input = collection.duplicate();
-            int n = readCollectionSize(input, ProtocolVersion.V3);
+            int n = readCollectionSize(input, ByteBufferAccessor.instance, ProtocolVersion.V3);
             int offset = sizeOfCollectionSize(n, ProtocolVersion.V3);
             for (int i = 0; i < n; i++)
             {
@@ -184,7 +184,7 @@ public class MapSerializer<K, V> extends CollectionSerializer<Map<K, V>>
         try
         {
             ByteBuffer input = collection.duplicate();
-            int n = readCollectionSize(input, ProtocolVersion.V3);
+            int n = readCollectionSize(input, ByteBufferAccessor.instance, ProtocolVersion.V3);
             input.position(input.position() + sizeOfCollectionSize(n, ProtocolVersion.V3));
             int startPos = input.position();
             int count = 0;
diff --git a/src/java/org/apache/cassandra/serializers/SetSerializer.java b/src/java/org/apache/cassandra/serializers/SetSerializer.java
index 6be919ff55..54b8496096 100644
--- a/src/java/org/apache/cassandra/serializers/SetSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/SetSerializer.java
@@ -156,7 +156,7 @@ public class SetSerializer<T> extends CollectionSerializer<Set<T>>
     {
         try
         {
-            int n = readCollectionSize(input, ProtocolVersion.V3);
+            int n = readCollectionSize(input, ByteBufferAccessor.instance, ProtocolVersion.V3);
             int offset = sizeOfCollectionSize(n, ProtocolVersion.V3);
 
             for (int i = 0; i < n; i++)
@@ -192,7 +192,7 @@ public class SetSerializer<T> extends CollectionSerializer<Set<T>>
         try
         {
             ByteBuffer input = collection.duplicate();
-            int n = readCollectionSize(input, ProtocolVersion.V3);
+            int n = readCollectionSize(input, ByteBufferAccessor.instance, ProtocolVersion.V3);
             input.position(input.position() + sizeOfCollectionSize(n, ProtocolVersion.V3));
             int startPos = input.position();
             int count = 0;
diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosRepairHistory.java b/src/java/org/apache/cassandra/service/paxos/PaxosRepairHistory.java
index a88e83136e..5e9fad1308 100644
--- a/src/java/org/apache/cassandra/service/paxos/PaxosRepairHistory.java
+++ b/src/java/org/apache/cassandra/service/paxos/PaxosRepairHistory.java
@@ -29,6 +29,7 @@ import com.google.common.collect.ImmutableList;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.db.marshal.TupleType;
@@ -181,7 +182,7 @@ public class PaxosRepairHistory
         Ballot[] ballotLowBounds = new Ballot[tuples.size()];
         for (int i = 0 ; i < tuples.size() ; ++i)
         {
-            ByteBuffer[] split = TYPE.split(tuples.get(i));
+            ByteBuffer[] split = TYPE.split(ByteBufferAccessor.instance, tuples.get(i));
             if (i < tokenInclusiveUpperBounds.length)
                 tokenInclusiveUpperBounds[i] = TOKEN_FACTORY.fromByteArray(split[0]);
             ballotLowBounds[i] = Ballot.deserialize(split[1]);
diff --git a/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java b/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java
new file mode 100644
index 0000000000..4bccb40c4c
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils.bytecomparable;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Interface indicating a value can be represented/identified by a comparable {@link ByteSource}.
+ *
+ * All Cassandra types that can be used as part of a primary key have a corresponding byte-comparable translation,
+ * detailed in ByteComparable.md. Byte-comparable representations are used in some memtable as well as primary and
+ * secondary index implementations.
+ */
+public interface ByteComparable
+{
+    /**
+     * Returns a source that generates the byte-comparable representation of the value byte by byte.
+     */
+    ByteSource asComparableBytes(Version version);
+
+    enum Version
+    {
+        LEGACY, // Encoding used in legacy sstable format; forward (value to byte-comparable) translation only
+        OSS42,  // CASSANDRA 4.2 encoding
+    }
+
+    ByteComparable EMPTY = (Version version) -> ByteSource.EMPTY;
+
+    /**
+     * Construct a human-readable string from the byte-comparable representation. Used for debugging.
+     */
+    default String byteComparableAsString(Version version)
+    {
+        StringBuilder builder = new StringBuilder();
+        ByteSource stream = asComparableBytes(version);
+        if (stream == null)
+            return "null";
+        for (int b = stream.next(); b != ByteSource.END_OF_STREAM; b = stream.next())
+            builder.append(Integer.toHexString((b >> 4) & 0xF)).append(Integer.toHexString(b & 0xF));
+        return builder.toString();
+    }
+
+    // Simple factories used for testing
+
+    static ByteComparable of(String s)
+    {
+        return v -> ByteSource.of(s, v);
+    }
+
+    static ByteComparable of(long value)
+    {
+        return v -> ByteSource.of(value);
+    }
+
+    static ByteComparable of(int value)
+    {
+        return v -> ByteSource.of(value);
+    }
+
+    static ByteComparable fixedLength(ByteBuffer bytes)
+    {
+        return v -> ByteSource.fixedLength(bytes);
+    }
+
+    static ByteComparable fixedLength(byte[] bytes)
+    {
+        return v -> ByteSource.fixedLength(bytes);
+    }
+
+    /**
+     * Returns a separator for two byte sources, i.e. something that is definitely > prevMax, and <= currMin, assuming
+     * prevMax < currMin.
+     * This returns the shortest prefix of currMin that is greater than prevMax.
+     */
+    static ByteComparable separatorPrefix(ByteComparable prevMax, ByteComparable currMin)
+    {
+        return version -> ByteSource.separatorPrefix(prevMax.asComparableBytes(version), currMin.asComparableBytes(version));
+    }
+
+    /**
+     * Returns a separator for two byte comparable, i.e. something that is definitely > prevMax, and <= currMin, assuming
+     * prevMax < currMin.
+     * This is a stream of length 1 longer than the common prefix of the two streams, with last byte one higher than the
+     * prevMax stream.
+     */
+    static ByteComparable separatorGt(ByteComparable prevMax, ByteComparable currMin)
+    {
+        return version -> ByteSource.separatorGt(prevMax.asComparableBytes(version), currMin.asComparableBytes(version));
+    }
+
+    static ByteComparable cut(ByteComparable src, int cutoff)
+    {
+        return version -> ByteSource.cut(src.asComparableBytes(version), cutoff);
+    }
+
+    /**
+     * Return the length of a byte comparable, not including the terminator byte.
+     */
+    static int length(ByteComparable src, Version version)
+    {
+        int l = 0;
+        ByteSource s = src.asComparableBytes(version);
+        while (s.next() != ByteSource.END_OF_STREAM)
+            ++l;
+        return l;
+    }
+
+    /**
+     * Compare two byte-comparable values by their byte-comparable representation. Used for tests.
+     *
+     * @return the result of the lexicographic unsigned byte comparison of the byte-comparable representations of the
+     *         two arguments
+     */
+    static int compare(ByteComparable bytes1, ByteComparable bytes2, Version version)
+    {
+        ByteSource s1 = bytes1.asComparableBytes(version);
+        ByteSource s2 = bytes2.asComparableBytes(version);
+
+        if (s1 == null || s2 == null)
+            return Boolean.compare(s1 != null, s2 != null);
+
+        while (true)
+        {
+            int b1 = s1.next();
+            int b2 = s2.next();
+            int cmp = Integer.compare(b1, b2);
+            if (cmp != 0)
+                return cmp;
+            if (b1 == ByteSource.END_OF_STREAM)
+                return 0;
+        }
+    }
+
+    /**
+     * Returns the length of the minimum prefix that differentiates the two given byte-comparable representations.
+     */
+    static int diffPoint(ByteComparable bytes1, ByteComparable bytes2, Version version)
+    {
+        ByteSource s1 = bytes1.asComparableBytes(version);
+        ByteSource s2 = bytes2.asComparableBytes(version);
+        int pos = 1;
+        int b;
+        while ((b = s1.next()) == s2.next() && b != ByteSource.END_OF_STREAM)
+            ++pos;
+        return pos;
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.md b/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.md
new file mode 100644
index 0000000000..f360635381
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/bytecomparable/ByteComparable.md
@@ -0,0 +1,693 @@
+<!---
+ 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.
+-->
+
+# Byte-comparable translation of types (ByteComparable/ByteSource)
+
+## Problem / Motivation
+
+Cassandra has a very heavy reliance on comparisons — they are used throughout read and write paths, coordination,
+compaction, etc. to be able to order and merge results. It also supports a range of types which often require the 
+compared object to be completely in memory to order correctly, which in turn has necessitated interfaces where 
+comparisons can only be applied if the compared objects are completely loaded.
+
+This has some rather painful implications on the performance of the database, both in terms of the time it takes to load,
+compare and garbage collect, as well as in terms of the space required to hold complete keys in on-disk indices and
+deserialized versions in in-memory data structures. In addition to this, the reliance on comparisons forces Cassandra to
+use only comparison-based structures, which aren’t the most efficient.
+
+There is no way to escape the need to compare and order objects in Cassandra, but the machinery for doing this can be
+done much more smartly if we impose some simple structure in the objects we deal with — byte ordering.
+
+The term “byte order” as used in this document refers to the property of being ordered via lexicographic compare on the
+unsigned values of the byte contents. Some of the types in Cassandra already have this property (e.g. strings, blobs),
+but other most heavily used ones (e.g. integers, uuids) don’t.
+
+When byte order is universally available for the types used for keys, several key advantages can be put to use:
+- Comparisons can be done using a single simple method, core machinery doesn’t need to know anything about types.
+- Prefix differences are enough to define order; unique prefixes can be used instead of complete keys.
+- Tries can be used to store, query and iterate over ranges of keys, providing fast lookup and prefix compression.
+- Merging can be performed by merging tries, significantly reducing the number of necessary comparisons.
+
+## Ordering the types
+
+As we want to keep all existing functionality in Cassandra, we need to be able to deal with existing
+non-byte-order-comparable types. This requires some form of conversion of each value to a sequence of bytes that can be 
+byte-order compared (also called "byte-comparable"), as well as the inverse conversion from byte-comparable to value.
+
+As one of the main advantages of byte order is the ability to decide comparisons early, without having to read the whole
+of the input sequence, byte-ordered interpretations of values are represented as sources of bytes with unknown length, 
+using the interface `ByteSource`. The interface declares one method, `next()` which produces the next byte of the
+stream, or `ByteSource.END_OF_STREAM` if the stream is exhausted.
+
+`END_OF_STREAM` is chosen as `-1` (`(int) -1`, which is outside the range of possible byte values), to make comparing 
+two byte sources as trivial (and thus fast) as possible.
+  
+To be able to completely abstract type information away from the storage machinery, we also flatten complex types into
+single byte sequences. To do this, we add separator bytes in front, between components, and at the end and do some 
+encoding of variable-length sequences.
+
+The other interface provided by this package `ByteComparable`, is an entity whose byte-ordered interpretation can be
+requested. The interface is implemented by `DecoratedKey`, and can be requested for clustering keys and bounds using
+`ClusteringComparator.asByteComparable`. The inverse translation is provided by 
+`Buffer/NativeDecoratedKey.fromByteComparable` and `ClusteringComparator.clustering/bound/boundaryFromByteComparable`.
+
+The (rather technical) paragraphs below detail the encoding we have chosen for the various types. For simplicity we
+only discuss the bidirectional `OSS42` version of the translation. The implementations in code of the various mappings
+are in the releavant `AbstractType` subclass.
+
+### Desired properties
+
+Generally, we desire the following two properties from the byte-ordered translations of values we use in the database:
+- Comparison equivalence (1):  
+    <math xmlns="http://www.w3.org/1998/Math/MathML">
+      <semantics>
+        <mstyle displaystyle="true">
+          <mo>&#x2200;</mo>
+          <mi>x</mi>
+          <mo>,</mo>
+          <mi>y</mi>
+          <mo>&#x2208;</mo>
+          <mi>T</mi>
+          <mo>,</mo>
+          <mrow>
+            <mtext>compareBytesUnsigned</mtext>
+          </mrow>
+          <mrow>
+            <mo>(</mo>
+            <mi>T</mi>
+            <mo>.</mo>
+            <mrow>
+              <mtext>byteOrdered</mtext>
+            </mrow>
+            <mrow>
+              <mo>(</mo>
+              <mi>x</mi>
+              <mo>)</mo>
+            </mrow>
+            <mo>,</mo>
+            <mi>T</mi>
+            <mo>.</mo>
+            <mrow>
+              <mtext>byteOrdered</mtext>
+            </mrow>
+            <mrow>
+              <mo>(</mo>
+              <mi>y</mi>
+              <mo>)</mo>
+            </mrow>
+            <mo>)</mo>
+          </mrow>
+          <mo>=</mo>
+          <mi>T</mi>
+          <mo>.</mo>
+          <mrow>
+            <mtext>compare</mtext>
+          </mrow>
+          <mrow>
+            <mo>(</mo>
+            <mi>x</mi>
+            <mo>,</mo>
+            <mi>y</mi>
+            <mo>)</mo>
+          </mrow>
+        </mstyle>
+        <!-- <annotation encoding="text/x-asciimath">forall x,y in T, "compareBytesUnsigned"(T."byteOrdered"(x), T."byteOrdered"(y))=T."compare"(x, y)</annotation> -->
+      </semantics>
+    </math>
+- Prefix-freedom (2):  
+    <math xmlns="http://www.w3.org/1998/Math/MathML">
+      <semantics>
+        <mstyle displaystyle="true">
+          <mo>&#x2200;</mo>
+          <mi>x</mi>
+          <mo>,</mo>
+          <mi>y</mi>
+          <mo>&#x2208;</mo>
+          <mi>T</mi>
+          <mo>,</mo>
+          <mi>T</mi>
+          <mo>.</mo>
+          <mrow>
+            <mtext>byteOrdered</mtext>
+          </mrow>
+          <mrow>
+            <mo>(</mo>
+            <mi>x</mi>
+            <mo>)</mo>
+          </mrow>
+          <mrow>
+            <mspace width="1ex" />
+            <mtext> is not a prefix of </mtext>
+            <mspace width="1ex" />
+          </mrow>
+          <mi>T</mi>
+          <mo>.</mo>
+          <mrow>
+            <mtext>byteOrdered</mtext>
+          </mrow>
+          <mrow>
+            <mo>(</mo>
+            <mi>y</mi>
+            <mo>)</mo>
+          </mrow>
+        </mstyle>
+        <!-- <annotation encoding="text/x-asciimath">forall x,y in T, T."byteOrdered"(x) " is not a prefix of " T."byteOrdered"(y)</annotation> -->
+      </semantics>
+    </math>
+   
+The former is the essential requirement, and the latter allows construction of encodings of sequences of multiple
+values, as well as a little more efficiency in the data structures.
+
+To more efficiently encode byte-ordered blobs, however, we use a slightly tweaked version of the above requirements:
+
+- Comparison equivalence (3):  
+    <math xmlns="http://www.w3.org/1998/Math/MathML">
+      <semantics>
+        <mstyle displaystyle="true">
+          <mo>&#x2200;</mo>
+          <mi>x</mi>
+          <mo>,</mo>
+          <mi>y</mi>
+          <mo>&#x2208;</mo>
+          <mi>T</mi>
+          <mo>,</mo>
+          <mo>&#x2200;</mo>
+          <msub>
+            <mi>b</mi>
+            <mn>1</mn>
+          </msub>
+          <mo>,</mo>
+          <msub>
+            <mi>b</mi>
+            <mn>2</mn>
+          </msub>
+          <mo>&#x2208;</mo>
+          <mrow>
+            <mo>[</mo>
+            <mn>0x10</mn>
+            <mo>-</mo>
+            <mn>0xEF</mn>
+            <mo>]</mo>
+          </mrow>
+          <mo>,</mo>
+            <mtext><br/></mtext>
+          <mrow>
+            <mtext>compareBytesUnsigned</mtext>
+          </mrow>
+          <mrow>
+            <mo>(</mo>
+            <mi>T</mi>
+            <mo>.</mo>
+            <mrow>
+              <mtext>byteOrdered</mtext>
+            </mrow>
+            <mrow>
+              <mo>(</mo>
+              <mi>x</mi>
+              <mo>)</mo>
+            </mrow>
+            <mo>+</mo>
+            <msub>
+              <mi>b</mi>
+              <mn>1</mn>
+            </msub>
+            <mo>,</mo>
+            <mi>T</mi>
+            <mo>.</mo>
+            <mrow>
+              <mtext>byteOrdered</mtext>
+            </mrow>
+            <mrow>
+              <mo>(</mo>
+              <mi>y</mi>
+              <mo>)</mo>
+            </mrow>
+            <mo>+</mo>
+            <msub>
+              <mi>b</mi>
+              <mn>2</mn>
+            </msub>
+            <mo>)</mo>
+          </mrow>
+          <mo>=</mo>
+          <mi>T</mi>
+          <mo>.</mo>
+          <mrow>
+            <mtext>compare</mtext>
+          </mrow>
+          <mrow>
+            <mo>(</mo>
+            <mi>x</mi>
+            <mo>,</mo>
+            <mi>y</mi>
+            <mo>)</mo>
+          </mrow>
+        </mstyle>
+        <!-- <annotation encoding="text/x-asciimath">forall x,y in T, forall b_1, b_2 in [0x10-0xEF],
+    "compareBytesUnsigned"(T."byteOrdered"(x)+b_1, T."byteOrdered"(y)+b_2)=T."compare"(x, y)</annotation> -->
+      </semantics>
+    </math>
+- Weak prefix-freedom (4):  
+    <math xmlns="http://www.w3.org/1998/Math/MathML">
+      <semantics>
+        <mstyle displaystyle="true">
+          <mo>&#x2200;</mo>
+          <mi>x</mi>
+          <mo>,</mo>
+          <mi>y</mi>
+          <mo>&#x2208;</mo>
+          <mi>T</mi>
+          <mo>,</mo>
+          <mo>&#x2200;</mo>
+          <mi>b</mi>
+          <mo>&#x2208;</mo>
+          <mrow>
+            <mo>[</mo>
+            <mn>0x10</mn>
+            <mo>-</mo>
+            <mn>0xEF</mn>
+            <mo>]</mo>
+          </mrow>
+          <mo>,</mo>
+            <mtext><br/></mtext>
+          <mrow>
+            <mo>(</mo>
+            <mi>T</mi>
+            <mo>.</mo>
+            <mrow>
+              <mtext>byteOrdered</mtext>
+            </mrow>
+            <mrow>
+              <mo>(</mo>
+              <mi>x</mi>
+              <mo>)</mo>
+            </mrow>
+            <mo>+</mo>
+            <mi>b</mi>
+            <mo>)</mo>
+          </mrow>
+          <mrow>
+            <mspace width="1ex" />
+            <mtext> is not a prefix of </mtext>
+            <mspace width="1ex" />
+          </mrow>
+          <mi>T</mi>
+          <mo>.</mo>
+          <mrow>
+            <mtext>byteOrdered</mtext>
+          </mrow>
+          <mrow>
+            <mo>(</mo>
+            <mi>y</mi>
+            <mo>)</mo>
+          </mrow>
+        </mstyle>
+        <!-- <annotation encoding="text/x-asciimath">forall x,y in T, forall b in [0x10-0xEF],
+    (T."byteOrdered"(x)+b) " is not a prefix of " T."byteOrdered"(y)</annotation> -->
+      </semantics>
+    </math>
+
+These versions allow the addition of a separator byte after each value, and guarantee that the combination with 
+separator fulfills the original requirements. (3) is somewhat stronger than (1) but is necessarily true if (2) is also 
+in force, while (4) trivially follows from (2).
+
+## Fixed length unsigned integers (Murmur token, date/time)
+
+This is the trivial case, as we can simply use the input bytes in big-endian order. The comparison result is the same, 
+and fixed length values are trivially prefix free, i.e. (1) and (2) are satisfied, and thus (3) and (4) follow from the
+observation above.
+
+## Fixed-length signed integers (byte, short, int, legacy bigint)
+
+As above, but we need to invert the sign bit of the number to put negative numbers before positives. This maps 
+`MIN_VALUE` to `0x00`..., `-1` to `0x7F…`, `0` to `0x80…`, and `MAX_VALUE` to `0xFF…`; comparing the resulting number 
+as an unsigned integer has the same effect as comparing the source signed.
+
+Examples:
+
+| Type and value | bytes                   |encodes as|
+|----------------|-------------------------|----------|
+| int 1          | 00 00 00 01             |             80 00 00 01
+| short -1       | FF FF                   |             7F FF
+| byte 0         | 00                      |             80
+| byte -2        | FE                      |             7E
+| int MAX_VALUE  | 7F FF FF FF             |             FF FF FF FF
+| long MIN_VALUE | 80 00 00 00 00 00 00 00 | 00 00 00 00 00 00 00 00
+
+## Variable-length encoding of integers (current bigint)
+
+Another way to encode integers that may save significant amounts of space when smaller numbers are often in use, but
+still permits large values to be efficiently encoded, is to use an encoding scheme similar to UTF-8.
+
+For unsigned numbers this can be done by starting the number with as many 1s in most significant bits as there are 
+additional bytes in the encoding, followed by a 0, and the bits of the number. Numbers between 0 and 127 are encoded
+in one byte, and each additional byte adds 7 more bits. Values that use all 8 bytes do not need a 9th bit of 0 and can
+thus fit 9 bytes. Because longer numbers have more 1s in their MSBs, they compare 
+higher than shorter ones (and we always use the shortest representation). Because the length is specified through these
+initial bits, no value can be a prefix of another.
+
+| Value            | bytes                   |encodes as|
+|------------------|-------------------------|----------|
+| 0                | 00 00 00 00 00 00 00 00 |             00
+| 1                | 00 00 00 00 00 00 00 01 |             01
+| 127 (2^7-1)      | 00 00 00 00 00 00 00 7F |             7F
+| 128 (2^7)        | 00 00 00 00 00 00 00 80 |             80 80
+| 16383 (2^14 - 1) | 00 00 00 00 00 00 3F FF |             BF FF
+| 16384 (2^14)     | 00 00 00 00 00 00 40 00 |             C0 40 00
+| 2^31 - 1         | 00 00 00 00 7F FF FF FF |         F0 7F FF FF FF
+| 2^31             | 00 00 00 00 80 00 00 00 |         F0 80 00 00 00
+| 2^56 - 1         | 00 FF FF FF FF FF FF FF | FE FF FF FF FF FF FF FF
+| 2^56             | 01 00 00 00 00 00 00 00 | FF 01 00 00 00 00 00 00 00
+| 2^64- 1          | FF FF FF FF FF FF FF FF | FF FF FF FF FF FF FF FF FF
+
+
+To encode signed numbers, we must start with the sign bit, and must also ensure that longer negative numbers sort 
+smaller than shorter ones. The first bit of the encoding is the inverted sign (i.e. 1 for positive, 0 for negative),
+followed by the length encoded as a sequence of bits that matches the inverted sign, followed by a bit that differs 
+(like above, not necessary for 9-byte encodings) and the bits of the number's two's complement.
+
+| Value             | bytes                    |encodes as|
+|-------------------|--------------------------|----------|
+| 1                 | 00 00 00 00 00 00 00 01  |             01
+| -1                | FF FF FF FF FF FF FF FF  |             7F
+| 0                 | 00 00 00 00 00 00 00 00  |             80
+| 63                | 00 00 00 00 00 00 00 3F  |             BF
+| -64               | FF FF FF FF FF FF FF C0  |             40
+| 64                | 00 00 00 00 00 00 00 40  |             C0 40
+| -65               | FF FF FF FF FF FF FF BF  |             3F BF
+| 8191              | 00 00 00 00 00 00 1F FF  | DF FF
+| 8192              | 00 00 00 00 00 00 20 00  | E0 20 00
+| Integer.MAX_VALUE | 00 00 00 00 7F FF FF FF  |             F8 7F FF FF FF
+| Long.MIN_VALUE    | 80 00 00 00 00 00 00 00  | 00 00 00 00 00 00 00 00 00
+
+
+## Fixed-size floating-point numbers (float, double)
+
+IEEE-754 was designed with byte-by-byte comparisons in mind, and provides an important guarantee about the bytes of a
+floating point number:  
+* If x and y are of the same sign, bytes(x) ≥ bytes(y) ⇔ |x| ≥ |y|.
+
+Thus, to be able to order floating point numbers as unsigned integers, we can:
+* Flip the sign bit so negatives are smaller than positive numbers.
+* If the number was negative, also flip all the other bits so larger magnitudes become smaller integers.
+
+This matches exactly the behaviour of `Double.compare`, which doesn’t fully agree with numerical comparisons (see spec) 
+in order to define a natural order over the floating point numbers.
+
+Examples:
+
+|Type and value|bytes|encodes as|
+|---|---|---|
+|float +1.0|            3F 80 00 00|               BF 80 00 00|
+|float +0.0|            00 00 00 00|               80 00 00 00|
+|float -0.0|            80 00 00 00|               7F FF FF FF|
+|float -1.0|            BF 80 00 00|               40 7F FF FF|
+|double +1.0|           3F F0 00 00 00 00 00 00|   BF F0 00 00 00 00 00 00|
+|double +Inf|           7F F0 00 00 00 00 00 00|   FF F0 00 00 00 00 00 00|
+|double -Inf|           FF F0 00 00 00 00 00 00|   00 0F FF FF FF FF FF FF|
+|double NaN|            7F F8 00 00 00 00 00 00|   FF F8 00 00 00 00 00 00|
+
+## UUIDs
+UUIDs are fixed-length unsigned integers, where the UUID version/type is compared first, and where bits need to be 
+reordered for the time UUIDs. To create a byte-ordered representation, we reorder the bytes: pull the version digit 
+first, then the rest of the digits, using the special time order if the version is equal to one.
+
+Examples:
+
+|Type and value|bytes|encodes as|
+|---|---|---|
+|Random (v4)|    cc520882-9507-44fb-8fc9-b349ecdee658 |    4cc52088295074fb8fc9b349ecdee658
+|Time (v1)  |    2a92d750-d8dc-11e6-a2de-cf8ecd4cf053 |    11e6d8dc2a92d750a2decf8ecd4cf053
+
+## Multi-component sequences (Partition or Clustering keys, tuples), bounds and nulls
+
+As mentioned above, we encode sequences by adding separator bytes in front, between components, and a terminator at the
+end. The values we chose for the separator and terminator are `0x40` and `0x38`, and they serve several purposes:
+- Permits partially specified bounds, with strict/exclusive or non-strict/inclusive semantics. This is done by finishing
+  a bound with a terminator value that is smaller/greater than the separator and terminator. We can use `0x20` for `<`/`≥`
+  and `0x60` for `≤`/`>`.
+- Permits encoding of `null` and `empty` values. We use `0x3E` as the separator for nulls and `0x3F` for empty, 
+  followed by no value bytes. This is always smaller than a sequence with non-null value for this component, but not 
+  smaller than a sequence that ends in this component.
+- Helps identify the ending of variable-length components (see below).
+
+Examples:
+
+|Types and values|bytes|encodes as|
+|---|---|---|
+|(short 1, float 1.0)    |    00 01, 3F 80 00 00    |   40·80 01·40·BF 80 00 00·38
+|(short -1, null)        |    FF FF, —              |   40·7F FF·3E·38
+|≥ (short 0, float -Inf) |    00 00, FF 80 00 00, >=|   40·80 00·40·00 7F FF FF·20
+|< (short MIN)           |    80 00, <=             |   40·00 00·20
+|\> (null)               |                          |   3E·60
+|BOTTOM                  |                          |   20
+|TOP                     |                          |   60
+
+(The middle dot · doesn't exist in the encoding, it’s just a visualisation of the boundaries in the examples.)
+
+Since:
+- all separators in use are within `0x10`-`0xEF`, and
+- we use the same separator for internal components, with the exception of nulls which we encode with a smaller 
+  separator
+- the sequence has a fixed number of components or we use a different trailing value whenever it can be shorter
+
+the properties (3) and (4) guarantee that the byte comparison of the encoding goes in the same direction as the
+lexicographical comparison of the sequence. In combination with the third point above, (4) also ensures that no encoding 
+is a prefix of another. Since we have (1) and (2), (3) and (4) are also satisfied.
+
+Note that this means that the encodings of all partition and clustering keys used in the database will be prefix-free.
+
+## Variable-length byte comparables (ASCII, UTF-8 strings, blobs, InetAddress)
+
+In isolation, these can be compared directly without reinterpretation. However, once we place these inside a flattened
+sequence of values we need to clearly define the boundaries between values while maintaining order. To do this we use an
+end-of-value marker; since shorter values must be smaller than longer, this marker must be 0 and we need to find a way 
+to encode/escape actual 0s in the input sequence.
+
+The method we chose for this is the following:
+- If the input does not end on `00`, a `00` byte is appended at the end.
+- If the input contains a `00` byte, it is encoded as `00 FF`.
+- If the input contains a sequence of *n* `00` bytes, they are encoded as `00` `FE` (*n*-1 times) `FF`  
+  (so that we don’t double the size of `00` blobs).
+- If the input ends in `00`, the last `FF` is changed to `FE`  
+  (to ensure it’s smaller than the same value with `00` appended).
+
+Examples:
+
+|bytes/sequence|encodes as|
+|---|----|
+|22 00                |        22 00 FE
+|22 00 00 33          |        22 00 FE FF 33 00
+|22 00 11             |        22 00 FF 11 00
+|(blob 22, short 0)   |        40·22 00·40·80 00·40
+| ≥ (blob 22 00)      |        40·22 00 FE·20
+| ≤ (blob 22 00 00)   |        40·22 00 FE FE·60
+
+Within the encoding, a `00` byte can only be followed by a `FE` or `FF` byte, and hence if an encoding is a prefix of 
+another, the latter has to have a `FE` or `FF` as the next byte, which ensures both (4) (adding `10`-`EF` to the former 
+makes it no longer a prefix of the latter) and (3) (adding `10`-`EF` to the former makes it smaller than the latter; in
+this case the original value of the former is a prefix of the original value of the latter).
+
+## Variable-length integers (varint, RandomPartitioner token), legacy encoding
+
+If integers of unbounded length are guaranteed to start with a non-zero digit, to compare them we can first use a signed
+length, as numbers with longer representations have higher magnitudes. Only if the lengths match we need to compare the
+sequence of digits, which now has a known length.
+
+(Note: The meaning of “digit” here is not the same as “decimal digit”. We operate with numbers stored as bytes, thus it
+makes most sense to treat the numbers as encoded in base-256, where each digit is a byte.)
+
+This translates to the following encoding of varints:
+- Strip any leading zeros. Note that for negative numbers, `BigInteger` encodes leading 0 as `0xFF`.
+- If the length is 128 or greater, lead with a byte of `0xFF` (positive) or `0x00` (negative) for every 128 until there
+  are less than 128 left.
+- Encode the sign and (remaining) length of the number as a byte:
+  - `0x80 + (length - 1)` for positive numbers (so that greater magnitude is higher);
+  - `0x7F - (length - 1)` for negative numbers (so that greater magnitude is lower, and all negatives are lower than
+    positives).
+- Paste the bytes of the number, 2’s complement encoded for negative numbers (`BigInteger` already applies the 2’s
+  complement).
+
+Since when comparing two numbers we either have a difference in the length prefix, or the lengths are the same if we 
+need to compare the content bytes, there is no risk that a longer number can be confused with a shorter combined in a
+multi-component sequence. In other words, no value can be a prefix of another, thus we have (1) and (2) and thus (3) and (4)
+as well.
+
+Examples:
+
+|   value | bytes            |encodes as|
+|--------:|------------------|---|
+|       0 | 00               | 80·00
+|       1 | 01               | 80·01
+|      -1 | FF               | 7F·FF
+|     255 | 00 FF            | 80·FF
+|    -256 | FF 00            | 7F·00
+|     256 | 01 00            | 81·01 00
+|    2^16 | 01 00 00         | 82·01 00 00
+|   -2^32 | FF 00 00 00 00   | 7C·00 00 00 00
+|  2^1024 | 01 00(128 times) | FF 80·01 00(128 times)
+| -2^2048 | FF 00(256 times) | 00 00 80·00(256 times)
+
+(Middle dot · shows the transition point between length and digits.)
+
+## Variable-length integers, current encoding
+
+Because variable-length integers are also often used to store smaller range integers, it makes sense to also apply
+the variable-length integer encoding. Thus, the current varint scheme chooses to:
+- Strip any leading zeros. Note that for negative numbers, `BigInteger` encodes leading 0 as `0xFF`.
+- Map numbers directly to their [variable-length integer encoding](#variable-length-encoding-of-integers-current-bigint),
+  if they have 6 bytes or less.
+- Otherwise, encode as:
+  - a sign byte (00 for negative numbers, FF for positive, distinct from the leading byte of the variable-length 
+    encoding above)
+  - a variable-length encoded number of bytes adjusted by -7 (so that the smallest length this encoding uses maps to 
+    0), inverted for negative numbers (so that greater length compares smaller)
+  - the bytes of the number, two's complement encoded.
+We never use a longer encoding (e.g. using the second method if variable-length suffices or with added 00 leading 
+bytes) if a shorter one suffices.
+
+By the same reasoning as above, and the fact that the sign byte cannot be confused with a variable-length encoding 
+first byte, no value can be a prefix of another. As the sign byte compares smaller for negative (respectively bigger 
+for positive numbers) than any variable-length encoded integer, the comparison order is maintained when one number 
+uses variable-length encoding, and the other doesn't. Longer numbers compare smaller when negative (because of the 
+inverted length bytes), and bigger when positive.
+
+Examples:
+
+|    value | bytes                   |encodes as|
+|---------:|-------------------------|---|
+|        0 | 00                      | 80
+|        1 | 01                      | 81
+|       -1 | FF                      | 7F
+|      255 | 00 FF                   | C0 FF
+|     -256 | FF 00                   | 3F 00
+|      256 | 01 00                   | C1 00
+|     2^16 | 01 00 00                | E1 00 00
+|    -2^32 | FF 00 00 00 00          | 07 00 00 00 00
+|   2^56-1 | 00 FF FF FF FF FF FF FF | FE FF FF FF FF FF FF FF
+|    -2^56 | FF 00 00 00 00 00 00 00 | 01 00 00 00 00 00 00 00
+|     2^56 | 01 00 00 00 00 00 00 00 | FF·00·01 00 00 00 00 00 00 00
+| -2^56-1  | FE FF FF FF FF FF FF FF | 00·FF·FE FF FF FF FF FF FF FF
+|   2^1024 | 01 00(128 times)        | FF·7A·01 00(128 times)
+|  -2^2048 | FF 00(256 times)        | 00·7F 06·00(256 times)
+
+(Middle dot · shows the transition point between length and digits.)
+
+## Variable-length floating-point decimals (decimal)
+
+Variable-length floats are more complicated, but we can treat them similarly to IEEE-754 floating point numbers, by
+normalizing them by splitting them into sign, mantissa and signed exponent such that the mantissa is a number below 1 
+with a non-zero leading digit. We can then compare sign, exponent and mantissa in sequence (where the comparison of
+exponent and mantissa are with reversed meaning if the sign is negative) and that gives us the decimal ordering.
+
+A bit of extra care must be exercised when encoding decimals. Since fractions like `0.1` cannot be perfectly encoded in
+binary, decimals (and mantissas) cannot be encoded in binary or base-256 correctly. A decimal base must be used; since 
+we deal with bytes, it makes most sense to make things a little more efficient by using base-100. Floating-point 
+encoding and the comparison idea from the previous paragraph work in any number base.
+
+`BigDecimal` presents a further challenge, as it encodes decimals using a mixture of bases: numbers have a binary-
+encoded integer part and a decimal power-of-ten scale. The bytes produced by a `BigDecimal` are thus not suitable for 
+direct conversion to byte comparable and we must first instantiate the bytes as a `BigDecimal`, and then apply the 
+class’s methods to operate on it as a number.
+
+We then use the following encoding:
+- If the number is 0, the encoding is a single `0x80` byte.
+- Convert the input to signed mantissa and signed exponent in base-100. If the value is negative, invert the sign of the
+  exponent to form the "modulated exponent".
+- Output a byte encoding:
+  - the sign of the number encoded as `0x80` if positive and `0x00` if negative,
+  - the exponent length (stripping leading 0s) in bytes as `0x40 + modulated_exponent_length`, where the length is given
+    with the sign of the modulated exponent.
+- Output `exponent_length` bytes of modulated exponent, 2’s complement encoded so that negative values are correctly 
+  ordered.
+- Output `0x80 + leading signed byte of mantissa`, which is obtained by multiplying the mantissa by 100 and rounding to
+  -∞. The rounding is done so that the remainder of the mantissa becomes positive, and thus every new byte adds some 
+  value to it, making shorter sequences lower in value.
+- Update the mantissa to be the remainder after the rounding above. The result is guaranteed to be 0 or greater.
+- While the mantissa is non-zero, output `0x80 + leading byte` as above and update the mantissa to be the remainder.
+- Output `0x00`.
+
+As a description of how this produces the correct ordering, consider the result of comparison in the first differing 
+byte:
+- Difference in the first byte can be caused by:
+  - Difference in sign of the number or being zero, which yields the correct ordering because
+    - Negative numbers start with `0x3c` - `0x44`
+    - Zero starts with `0x80`
+    - Positive numbers start with `0xbc` - `0xc4`
+  - Difference in sign of the exponent modulated with the sign of the number. In a positive number negative exponents 
+    mean smaller values, while in a negative number it’s the opposite, thus the modulation with the number’s sign 
+    ensures the correct ordering. 
+  - Difference in modulated length of the exponent: again, since we gave the length a sign that is formed from both 
+    the sign of the exponent and the sign of the number, smaller numbers mean smaller exponent in the positive number 
+    case, and bigger exponent in the negative number case. In either case this provides the correct ordering.
+- Difference in one of the bytes of the modulated exponent (whose length and sign are now equal for both compared
+  numbers):
+  - Smaller byte means a smaller modulated exponent. In the positive case this means a smaller exponent, thus a smaller 
+    number. In the negative case this means the exponent is bigger, the absolute value of the number as well, and thus 
+    the number is smaller.
+- It is not possible for the difference to mix one number’s exponent with another’s mantissa (as such numbers would have
+  different leading bytes).
+- Difference in a mantissa byte present in both inputs:
+  - Smaller byte means smaller signed mantissa and hence smaller number when the exponents are equal.
+- One mantissa ending before another:
+  - This will result in the shorter being treated as smaller (since the trailing byte is `00`).
+  - Since all mantissas have at least one byte, this can’t happen in the leading mantissa byte.
+  - Thus the other number’s bytes from here on are not negative, and at least one of them must be non-zero, which means 
+    its mantissa is bigger and thus it encodes a bigger number.
+    
+Examples:
+
+|value|mexp|mantissa|mantissa in bytes|encodes as|
+|---:|---:|---|---|---|
+|1.1        | 1    | 0.0110 |.  01 10  |    C1·01·81 8A·00
+|1          | 1    | 0.01   |.  01     |    C1·01·81·00
+|0.01       | 0    | 0.01   |.  01     |    C0·81·00
+|0          |      |        |          |    80
+|-0.01      | 0    | -0.01  |. -01     |    40·81·00
+|-1         | -1   | -0.01  |. -01     |    3F·FF·7F·00
+|-1.1       | -1   | -0.0110|. -02 90  |    3F·FF·7E DA·00
+|-98.9      | -1   | -0.9890|. -99 10  |    3F·FF·1D 8A·00
+|-99        | -1   | -0.99  |. -99     |    3F·FF·1D·00
+|-99.9      | -1   | -0.9990|.-100 10  |    3F·FF·1C 8A·00
+|-8.1e2000  | -1001| -0.0810|. -09 90  |    3E·FC 17·77 DA·00
+|-8.1e-2000 | 999  | -0.0810|. -09 90  |    42·03 E7·77 DA·00
+|8.1e-2000  | -999 | 0.0810 |.  08 10  |    BE·FC 19·88 8A·00
+|8.1e2000   | 1001 | 0.0810 |.  08 10  |    C2·03 E9·88 8A·00
+(mexp stands for “modulated exponent”, i.e. exponent * sign)
+
+The values are prefix-free, because no exponent’s encoding can be a prefix of another, and the mantissas can never have
+a `00` byte at any place other than the last byte, and thus all (1)-(4) are satisfied.
+
+## Nulls and empty encodings
+
+Some types in Cassandra (e.g. numbers) admit null values that are represented as empty byte buffers. This is 
+distinct from null byte buffers, which can also appear in some cases. Particularly, null values in clustering 
+columns, when allowed by the type, are interpreted as empty byte buffers, encoded with the empty separator `0x3F`. 
+Unspecified clustering columns (at the end of a clustering specification), possible with `COMPACT STORAGE` or secondary 
+indexes, use the null separator `0x3E`.
+
+## Reversed types
+
+Reversing a type is straightforward: flip all bits of the encoded byte sequence. Since the source type encoding must
+satisfy (3) and (4), the flipped bits also do for the reversed comparator. (It is also true that if the source type 
+satisfies (1)-(2), the reversed will satisfy these too.)
+
+In a sequence we also must correct the empty encoding for a reversed type (since it must be greater than all values).
+Instead of `0x3F` we use `0x41` as the separator byte. Null encodings are not modified, as nulls compare smaller even
+in reversed types.
+
diff --git a/src/java/org/apache/cassandra/utils/bytecomparable/ByteSource.java b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSource.java
new file mode 100644
index 0000000000..be4cec0eae
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSource.java
@@ -0,0 +1,853 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.utils.bytecomparable;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.cassandra.db.marshal.ValueAccessor;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * A stream of bytes, used for byte-order-comparable representations of data, and utilities to convert various values
+ * to their byte-ordered translation.
+ * See ByteComparable.md for details about the encoding scheme.
+ */
+public interface ByteSource
+{
+    /** Consume the next byte, unsigned. Must be between 0 and 255, or END_OF_STREAM if there are no more bytes. */
+    int next();
+
+    /** Value returned if at the end of the stream. */
+    int END_OF_STREAM = -1;
+
+    ByteSource EMPTY = () -> END_OF_STREAM;
+
+    /**
+     * Escape value. Used, among other things, to mark the end of subcomponents (so that shorter compares before anything longer).
+     * Actual zeros in input need to be escaped if this is in use (see {@link AbstractEscaper}).
+     */
+    int ESCAPE = 0x00;
+
+    // Zeros are encoded as a sequence of ESCAPE, 0 or more of ESCAPED_0_CONT, ESCAPED_0_DONE so zeroed spaces only grow by 1 byte
+    int ESCAPED_0_CONT = 0xFE;
+    int ESCAPED_0_DONE = 0xFF;
+
+    // All separators must be within these bounds
+    int MIN_SEPARATOR = 0x10;
+    int MAX_SEPARATOR = 0xEF;
+
+    // Next component marker.
+    int NEXT_COMPONENT = 0x40;
+    // Marker used to present null values represented by empty buffers (e.g. by Int32Type)
+    int NEXT_COMPONENT_EMPTY = 0x3F;
+    int NEXT_COMPONENT_EMPTY_REVERSED = 0x41;
+    // Marker for null components in tuples, maps, sets and clustering keys.
+    int NEXT_COMPONENT_NULL = 0x3E;
+
+    // Section for next component markers which is not allowed for use
+    int MIN_NEXT_COMPONENT = 0x3C;
+    int MAX_NEXT_COMPONENT = 0x44;
+
+    // Default terminator byte in sequences. Smaller than NEXT_COMPONENT_NULL, but larger than LT_NEXT_COMPONENT to
+    // ensure lexicographic compares go in the correct direction
+    int TERMINATOR = 0x38;
+    // These are special endings, for exclusive/inclusive bounds (i.e. smaller than anything with more components,
+    // bigger than anything with more components)
+    int LT_NEXT_COMPONENT = 0x20;
+    int GT_NEXT_COMPONENT = 0x60;
+
+    // Special value for components that should be excluded from the normal min/max span. (static rows)
+    int EXCLUDED = 0x18;
+
+    /**
+     * Encodes byte-accessible data as a byte-comparable source that has 0s escaped and finishes in an escaped
+     * state.
+     * This provides a weakly-prefix-free byte-comparable version of the content to use in sequences.
+     * (See {@link AbstractEscaper} for a detailed explanation.)
+     */
+    static <V> ByteSource of(ValueAccessor<V> accessor, V data, Version version)
+    {
+        return new AccessorEscaper<>(accessor, data, version);
+    }
+
+    /**
+     * Encodes a byte buffer as a byte-comparable source that has 0s escaped and finishes in an escape.
+     * This provides a weakly-prefix-free byte-comparable version of the content to use in sequences.
+     * (See ByteSource.BufferEscaper/Multi for explanation.)
+     */
+    static ByteSource of(ByteBuffer buf, Version version)
+    {
+        return new BufferEscaper(buf, version);
+    }
+
+    /**
+     * Encodes a byte array as a byte-comparable source that has 0s escaped and finishes in an escape.
+     * This provides a prefix-free byte-comparable version of the content to use in sequences.
+     * (See ByteSource.BufferEscaper/Multi for explanation.)
+     */
+    static ByteSource of(byte[] buf, Version version)
+    {
+        return new ArrayEscaper(buf, version);
+    }
+
+    /**
+     * Encodes a memory range as a byte-comparable source that has 0s escaped and finishes in an escape.
+     * This provides a weakly-prefix-free byte-comparable version of the content to use in sequences.
+     * (See ByteSource.BufferEscaper/Multi for explanation.)
+     */
+    static ByteSource ofMemory(long address, int length, ByteComparable.Version version)
+    {
+        return new MemoryEscaper(address, length, version);
+    }
+
+    /**
+     * Combines a chain of sources, turning their weak-prefix-free byte-comparable representation into the combination's
+     * prefix-free byte-comparable representation, with the included terminator character.
+     * For correctness, the terminator must be within MIN-MAX_SEPARATOR and outside the range reserved for
+     * NEXT_COMPONENT markers.
+     * Typically TERMINATOR, or LT/GT_NEXT_COMPONENT if used for partially specified bounds.
+     */
+    static ByteSource withTerminator(int terminator, ByteSource... srcs)
+    {
+        assert terminator >= MIN_SEPARATOR && terminator <= MAX_SEPARATOR;
+        assert terminator < MIN_NEXT_COMPONENT || terminator > MAX_NEXT_COMPONENT;
+        return new Multi(srcs, terminator);
+    }
+
+    /**
+     * As above, but permits any separator. The legacy format wasn't using weak prefix freedom and has some
+     * non-reversible transformations.
+     */
+    static ByteSource withTerminatorLegacy(int terminator, ByteSource... srcs)
+    {
+        return new Multi(srcs, terminator);
+    }
+
+    static ByteSource withTerminatorMaybeLegacy(Version version, int legacyTerminator, ByteSource... srcs)
+    {
+        return version == Version.LEGACY ? withTerminatorLegacy(legacyTerminator, srcs)
+                                         : withTerminator(TERMINATOR, srcs);
+    }
+
+    static ByteSource of(String s, Version version)
+    {
+        return new ArrayEscaper(s.getBytes(StandardCharsets.UTF_8), version);
+    }
+
+    static ByteSource of(long value)
+    {
+        return new Number(value ^ (1L<<63), 8);
+    }
+
+    static ByteSource of(int value)
+    {
+        return new Number(value ^ (1L<<31), 4);
+    }
+
+    /**
+     * Produce a source for a signed fixed-length number, also translating empty to null.
+     * The first byte has its sign bit inverted, and the rest are passed unchanged.
+     * Presumes that the length of the buffer is always either 0 or constant for the type, which permits decoding and
+     * ensures the representation is prefix-free.
+     */
+    static <V> ByteSource optionalSignedFixedLengthNumber(ValueAccessor<V> accessor, V data)
+    {
+        return !accessor.isEmpty(data) ? signedFixedLengthNumber(accessor, data) : null;
+    }
+
+    /**
+     * Produce a source for a signed fixed-length number.
+     * The first byte has its sign bit inverted, and the rest are passed unchanged.
+     * Presumes that the length of the buffer is always constant for the type.
+     */
+    static <V> ByteSource signedFixedLengthNumber(ValueAccessor<V> accessor, V data)
+    {
+        return new SignedFixedLengthNumber<>(accessor, data);
+    }
+
+    /**
+     * Produce a source for a signed fixed-length floating-point number, also translating empty to null.
+     * If sign bit is on, returns negated bytes. If not, add the sign bit value.
+     * (Sign of IEEE floats is the highest bit, the rest can be compared in magnitude by byte comparison.)
+     * Presumes that the length of the buffer is always either 0 or constant for the type, which permits decoding and
+     * ensures the representation is prefix-free.
+     */
+    static <V> ByteSource optionalSignedFixedLengthFloat(ValueAccessor<V> accessor, V data)
+    {
+        return !accessor.isEmpty(data) ? signedFixedLengthFloat(accessor, data) : null;
+    }
+
+    /**
+     * Produce a source for a signed fixed-length floating-point number.
+     * If sign bit is on, returns negated bytes. If not, add the sign bit value.
+     * (Sign of IEEE floats is the highest bit, the rest can be compared in magnitude by byte comparison.)
+     * Presumes that the length of the buffer is always constant for the type.
+     */
+    static <V> ByteSource signedFixedLengthFloat(ValueAccessor<V> accessor, V data)
+    {
+        return new SignedFixedLengthFloat<>(accessor, data);
+    }
+
+    /**
+     * Produce a source for a signed integer, stored using variable length encoding.
+     * The representation uses between 1 and 9 bytes, is prefix-free and compares
+     * correctly.
+     */
+    static ByteSource variableLengthInteger(long value)
+    {
+        return new VariableLengthInteger(value);
+    }
+
+    /**
+     * Returns a separator for two byte sources, i.e. something that is definitely > prevMax, and <= currMin, assuming
+     * prevMax < currMin.
+     * This returns the shortest prefix of currMin that is greater than prevMax.
+     */
+    public static ByteSource separatorPrefix(ByteSource prevMax, ByteSource currMin)
+    {
+        return new Separator(prevMax, currMin, true);
+    }
+
+    /**
+     * Returns a separator for two byte sources, i.e. something that is definitely > prevMax, and <= currMin, assuming
+     * prevMax < currMin.
+     * This is a source of length 1 longer than the common prefix of the two sources, with last byte one higher than the
+     * prevMax source.
+     */
+    public static ByteSource separatorGt(ByteSource prevMax, ByteSource currMin)
+    {
+        return new Separator(prevMax, currMin, false);
+    }
+
+    public static ByteSource oneByte(int i)
+    {
+        assert i >= 0 && i <= 0xFF : "Argument must be a valid unsigned byte.";
+        return new ByteSource()
+        {
+            boolean consumed = false;
+
+            @Override
+            public int next()
+            {
+                if (consumed)
+                    return END_OF_STREAM;
+                consumed = true;
+                return i;
+            }
+        };
+    }
+
+    public static ByteSource cut(ByteSource src, int cutoff)
+    {
+        return new ByteSource()
+        {
+            int pos = 0;
+
+            @Override
+            public int next()
+            {
+                return pos++ < cutoff ? src.next() : END_OF_STREAM;
+            }
+        };
+    }
+
+    /**
+     * Wrap a ByteSource in a length-fixing facade.
+     *
+     * If the length of {@code src} is less than {@code cutoff}, then pad it on the right with {@code padding} until
+     * the overall length equals {@code cutoff}.  If the length of {@code src} is greater than {@code cutoff}, then
+     * truncate {@code src} to that size.  Effectively a noop if {@code src} happens to have length {@code cutoff}.
+     *
+     * @param src the input source to wrap
+     * @param cutoff the size of the source returned
+     * @param padding a padding byte (an int subject to a 0xFF mask)
+     */
+    public static ByteSource cutOrRightPad(ByteSource src, int cutoff, int padding)
+    {
+        return new ByteSource()
+        {
+            int pos = 0;
+
+            @Override
+            public int next()
+            {
+                if (pos++ >= cutoff)
+                {
+                    return END_OF_STREAM;
+                }
+                int next = src.next();
+                return next == END_OF_STREAM ? padding : next;
+            }
+        };
+    }
+
+
+    /**
+     * Variable-length encoding. Escapes 0s as ESCAPE + zero or more ESCAPED_0_CONT + ESCAPED_0_DONE.
+     * If the source ends in 0, we use ESCAPED_0_CONT to make sure that the encoding remains smaller than that source
+     * with a further 0 at the end.
+     * Finishes in an escaped state (either with ESCAPE or ESCAPED_0_CONT), which in {@link Multi} is followed by
+     * a component separator between 0x10 and 0xFE.
+     *
+     * E.g. "A\0\0B" translates to 4100FEFF4200
+     *      "A\0B\0"               4100FF4200FE (+00 for {@link Version#LEGACY})
+     *      "A\0"                  4100FE       (+00 for {@link Version#LEGACY})
+     *      "AB"                   414200
+     *
+     * If in a single byte source, the bytes could be simply passed unchanged, but this would not allow us to
+     * combine components. This translation preserves order, and since the encoding for 0 is higher than the separator
+     * also makes sure shorter components are treated as smaller.
+     *
+     * The encoding is not prefix-free, since e.g. the encoding of "A" (4100) is a prefix of the encoding of "A\0"
+     * (4100FE), but the byte following the prefix is guaranteed to be FE or FF, which makes the encoding weakly
+     * prefix-free. Additionally, any such prefix sequence will compare smaller than the value to which it is a prefix,
+     * because any permitted separator byte will be smaller than the byte following the prefix.
+     */
+    abstract static class AbstractEscaper implements ByteSource
+    {
+        private final Version version;
+        private int bufpos;
+        private boolean escaped;
+
+        AbstractEscaper(int position, Version version)
+        {
+            this.bufpos = position;
+            this.version = version;
+        }
+
+        @Override
+        public final int next()
+        {
+            if (bufpos >= limit())
+            {
+                if (bufpos > limit())
+                    return END_OF_STREAM;
+
+                ++bufpos;
+                if (escaped)
+                {
+                    escaped = false;
+                    if (version == Version.LEGACY)
+                        --bufpos; // place an ESCAPE at the end of sequence ending in ESCAPE
+                    return ESCAPED_0_CONT;
+                }
+                return ESCAPE;
+            }
+
+            int index = bufpos++;
+            int b = get(index) & 0xFF;
+            if (!escaped)
+            {
+                if (b == ESCAPE)
+                    escaped = true;
+                return b;
+            }
+            else
+            {
+                if (b == ESCAPE)
+                    return ESCAPED_0_CONT;
+                --bufpos;
+                escaped = false;
+                return ESCAPED_0_DONE;
+            }
+        }
+
+        protected abstract byte get(int index);
+
+        protected abstract int limit();
+    }
+
+    static class AccessorEscaper<V> extends AbstractEscaper
+    {
+        private final V data;
+        private final ValueAccessor<V> accessor;
+
+        private AccessorEscaper(ValueAccessor<V> accessor, V data, Version version)
+        {
+            super(0, version);
+            this.accessor = accessor;
+            this.data = data;
+        }
+
+        protected int limit()
+        {
+            return accessor.size(data);
+        }
+
+        protected byte get(int index)
+        {
+            return accessor.getByte(data, index);
+        }
+    }
+
+    static class BufferEscaper extends AbstractEscaper
+    {
+        private final ByteBuffer buf;
+
+        private BufferEscaper(ByteBuffer buf, Version version)
+        {
+            super(buf.position(), version);
+            this.buf = buf;
+        }
+
+        protected int limit()
+        {
+            return buf.limit();
+        }
+
+        protected byte get(int index)
+        {
+            return buf.get(index);
+        }
+    }
+
+    static class ArrayEscaper extends AbstractEscaper
+    {
+        private final byte[] buf;
+
+        private ArrayEscaper(byte[] buf, Version version)
+        {
+            super(0, version);
+            this.buf = buf;
+        }
+
+        @Override
+        protected byte get(int index)
+        {
+            return buf[index];
+        }
+
+        @Override
+        protected int limit()
+        {
+            return buf.length;
+        }
+    }
+
+    static class MemoryEscaper extends AbstractEscaper
+    {
+        private final long address;
+        private final int length;
+
+        MemoryEscaper(long address, int length, ByteComparable.Version version)
+        {
+            super(0, version);
+            this.address = address;
+            this.length = length;
+        }
+
+        protected byte get(int index)
+        {
+            return MemoryUtil.getByte(address + index);
+        }
+
+        protected int limit()
+        {
+            return length;
+        }
+    }
+
+    /**
+     * Fixed length signed number encoding. Inverts first bit (so that neg < pos), then just posts all bytes from the
+     * buffer. Assumes buffer is of correct length.
+     */
+    static class SignedFixedLengthNumber<V> implements ByteSource
+    {
+        private final ValueAccessor<V> accessor;
+        private final V data;
+        private int bufpos;
+
+        public SignedFixedLengthNumber(ValueAccessor<V> accessor, V data)
+        {
+            this.accessor = accessor;
+            this.data = data;
+            this.bufpos = 0;
+        }
+
+        @Override
+        public int next()
+        {
+            if (bufpos >= accessor.size(data))
+                return END_OF_STREAM;
+            int v = accessor.getByte(data, bufpos) & 0xFF;
+            if (bufpos == 0)
+                v ^= 0x80;
+            ++bufpos;
+            return v;
+        }
+    }
+
+    /**
+     * Variable-length encoding for unsigned integers.
+     * The encoding is similar to UTF-8 encoding.
+     * Numbers between 0 and 127 are encoded in one byte, using 0 in the most significant bit.
+     * Larger values have 1s in as many of the most significant bits as the number of additional bytes
+     * in the representation, followed by a 0. This ensures that longer numbers compare larger than shorter
+     * ones. Since we never use a longer representation than necessary, this implies numbers compare correctly.
+     * As the number of bytes is specified in the bits of the first, no value is a prefix of another.
+     */
+    static class VariableLengthUnsignedInteger implements ByteSource
+    {
+        private final long value;
+        private int pos = -1;
+
+        public VariableLengthUnsignedInteger(long value)
+        {
+            this.value = value;
+        }
+
+        @Override
+        public int next()
+        {
+            if (pos == -1)
+            {
+                int bitsMinusOne = 63 - (Long.numberOfLeadingZeros(value | 1)); // 0 to 63 (the | 1 is to make sure 0 maps to 0 (1 bit))
+                int bytesMinusOne = bitsMinusOne / 7;
+                int mask = -256 >> bytesMinusOne;   // sequence of bytesMinusOne 1s in the most-significant bits
+                pos = bytesMinusOne * 8;
+                return (int) ((value >>> pos) | mask) & 0xFF;
+            }
+            pos -= 8;
+            if (pos < 0)
+                return END_OF_STREAM;
+            return (int) (value >>> pos) & 0xFF;
+        }
+    }
+
+    /**
+     * Variable-length encoding for signed integers.
+     * The encoding is based on the unsigned encoding above, where the first bit stored is the inverted sign,
+     * followed by as many matching bits as there are additional bytes in the encoding, followed by the two's
+     * complement of the number.
+     * Because of the inverted sign bit, negative numbers compare smaller than positives, and because the length
+     * bits match the sign, longer positive numbers compare greater and longer negative ones compare smaller.
+     *
+     * Examples:
+     *      0              encodes as           80
+     *      1              encodes as           81
+     *     -1              encodes as           7F
+     *     63              encodes as           BF
+     *     64              encodes as           C040
+     *    -64              encodes as           40
+     *    -65              encodes as           3FBF
+     *   2^20-1            encodes as           EFFFFF
+     *   2^20              encodes as           F0100000
+     *  -2^20              encodes as           100000
+     *   2^64-1            encodes as           FFFFFFFFFFFFFFFFFF
+     *  -2^64              encodes as           000000000000000000
+     *
+     * As the number of bytes is specified in bits 2-9, no value is a prefix of another.
+     */
+    static class VariableLengthInteger implements ByteSource
+    {
+        private final long value;
+        private int pos;
+
+        public VariableLengthInteger(long value)
+        {
+            long negativeMask = value >> 63;    // -1 for negative, 0 for positive
+            value ^= negativeMask;
+
+            int bits = 64 - Long.numberOfLeadingZeros(value | 1); // 1 to 63 (can't be 64 because we flip negative numbers)
+            int bytes = bits / 7 + 1;   // 0-6 bits 1 byte 7-13 2 bytes etc to 56-63 9 bytes
+            if (bytes >= 9)
+            {
+                value |= 0x8000000000000000L;   // 8th bit, which doesn't fit the first byte
+                pos = negativeMask < 0 ? 256 : -1; // out of 0-64 range integer such that & 0xFF is 0x00 for negative and 0xFF for positive
+            }
+            else
+            {
+                long mask = (-0x100 >> bytes) & 0xFF; // one in sign bit and as many more as there are extra bytes
+                pos = bytes * 8;
+                value = value | (mask << (pos - 8));
+            }
+
+            value ^= negativeMask;
+            this.value = value;
+        }
+
+        @Override
+        public int next()
+        {
+            if (pos <= 0 || pos > 64)
+            {
+                if (pos == 0)
+                    return END_OF_STREAM;
+                else
+                {
+                    // 8-byte value, returning first byte
+                    int result = pos & 0xFF; // 0x00 for negative numbers, 0xFF for positive
+                    pos = 64;
+                    return result;
+                }
+            }
+            pos -= 8;
+            return (int) (value >>> pos) & 0xFF;
+        }
+    }
+
+    static class Number implements ByteSource
+    {
+        private final long value;
+        private int pos;
+
+        public Number(long value, int length)
+        {
+            this.value = value;
+            this.pos = length;
+        }
+
+        @Override
+        public int next()
+        {
+            if (pos == 0)
+                return END_OF_STREAM;
+            return (int) ((value >> (--pos * 8)) & 0xFF);
+        }
+    }
+
+    /**
+     * Fixed length signed floating point number encoding. First bit is sign. If positive, add sign bit value to make
+     * greater than all negatives. If not, invert all content to make negatives with bigger magnitude smaller.
+     */
+    static class SignedFixedLengthFloat<V> implements ByteSource
+    {
+        private final ValueAccessor<V> accessor;
+        private final V data;
+        private int bufpos;
+        private boolean invert;
+
+        public SignedFixedLengthFloat(ValueAccessor<V> accessor, V data)
+        {
+            this.accessor = accessor;
+            this.data = data;
+            this.bufpos = 0;
+        }
+
+        @Override
+        public int next()
+        {
+            if (bufpos >= accessor.size(data))
+                return END_OF_STREAM;
+            int v = accessor.getByte(data, bufpos) & 0xFF;
+            if (bufpos == 0)
+            {
+                invert = v >= 0x80;
+                v |= 0x80;
+            }
+            if (invert)
+                v = v ^ 0xFF;
+            ++bufpos;
+            return v;
+        }
+    }
+
+    /**
+     * Combination of multiple byte sources. Adds {@link NEXT_COMPONENT} before sources, or {@link NEXT_COMPONENT_NULL} if next is null.
+     */
+    static class Multi implements ByteSource
+    {
+        private final ByteSource[] srcs;
+        private int srcnum = -1;
+        private final int sequenceTerminator;
+
+        Multi(ByteSource[] srcs, int sequenceTerminator)
+        {
+            this.srcs = srcs;
+            this.sequenceTerminator = sequenceTerminator;
+        }
+
+        @Override
+        public int next()
+        {
+            if (srcnum == srcs.length)
+                return END_OF_STREAM;
+
+            int b = END_OF_STREAM;
+            if (srcnum >= 0 && srcs[srcnum] != null)
+                b = srcs[srcnum].next();
+            if (b > END_OF_STREAM)
+                return b;
+
+            ++srcnum;
+            if (srcnum == srcs.length)
+                return sequenceTerminator;
+            if (srcs[srcnum] == null)
+                return NEXT_COMPONENT_NULL;
+            return NEXT_COMPONENT;
+        }
+    }
+
+    /**
+     * Construct the shortest common prefix of prevMax and currMin that separates those two byte streams.
+     * If {@code useCurr == true} the last byte of the returned stream comes from {@code currMin} and is the first
+     * byte which is greater than byte on the corresponding position of {@code prevMax}.
+     * Otherwise, the last byte of the returned stream comes from {@code prevMax} and is incremented by one, still
+     * guaranteeing that it is <= than the byte on the corresponding position of {@code currMin}.
+     */
+    static class Separator implements ByteSource
+    {
+        private final ByteSource prev;
+        private final ByteSource curr;
+        private boolean done = false;
+        private final boolean useCurr;
+
+        Separator(ByteSource prevMax, ByteSource currMin, boolean useCurr)
+        {
+            this.prev = prevMax;
+            this.curr = currMin;
+            this.useCurr = useCurr;
+        }
+
+        @Override
+        public int next()
+        {
+            if (done)
+                return END_OF_STREAM;
+            int p = prev.next();
+            int c = curr.next();
+            assert p <= c : prev + " not less than " + curr;
+            if (p == c)
+                return c;
+            done = true;
+            return useCurr ? c : p + 1;
+        }
+    }
+
+    static <V> ByteSource optionalFixedLength(ValueAccessor<V> accessor, V data)
+    {
+        return !accessor.isEmpty(data) ? fixedLength(accessor, data) : null;
+    }
+
+    /**
+     * A byte source of the given bytes without any encoding.
+     * The resulting source is only guaranteed to give correct comparison results and be prefix-free if the
+     * underlying type has a fixed length.
+     * In tests, this method is also used to generate non-escaped test cases.
+     */
+    public static <V> ByteSource fixedLength(ValueAccessor<V> accessor, V data)
+    {
+        return new ByteSource()
+        {
+            int pos = -1;
+
+            @Override
+            public int next()
+            {
+                return ++pos < accessor.size(data) ? accessor.getByte(data, pos) & 0xFF : END_OF_STREAM;
+            }
+        };
+    }
+
+    /**
+     * A byte source of the given bytes without any encoding.
+     * The resulting source is only guaranteed to give correct comparison results and be prefix-free if the
+     * underlying type has a fixed length.
+     * In tests, this method is also used to generate non-escaped test cases.
+     */
+    public static ByteSource fixedLength(ByteBuffer b)
+    {
+        return new ByteSource()
+        {
+            int pos = b.position() - 1;
+
+            @Override
+            public int next()
+            {
+                return ++pos < b.limit() ? b.get(pos) & 0xFF : END_OF_STREAM;
+            }
+        };
+    }
+
+    /**
+     * A byte source of the given bytes without any encoding.
+     * If used in a sequence, the resulting source is only guaranteed to give correct comparison results if the
+     * underlying type has a fixed length.
+     * In tests, this method is also used to generate non-escaped test cases.
+     */
+    public static ByteSource fixedLength(byte[] b)
+    {
+        return fixedLength(b, 0, b.length);
+    }
+
+    public static ByteSource fixedLength(byte[] b, int offset, int length)
+    {
+        checkArgument(offset >= 0 && offset <= b.length);
+        checkArgument(length >= 0 && offset + length <= b.length);
+
+        return new ByteSource()
+        {
+            int pos = offset - 1;
+
+            @Override
+            public int next()
+            {
+                return ++pos < offset + length ? b[pos] & 0xFF : END_OF_STREAM;
+            }
+        };
+    }
+
+    public class Peekable implements ByteSource
+    {
+        private static final int NONE = Integer.MIN_VALUE;
+
+        private final ByteSource wrapped;
+        private int peeked = NONE;
+
+        public Peekable(ByteSource wrapped)
+        {
+            this.wrapped = wrapped;
+        }
+
+        @Override
+        public int next()
+        {
+            if (peeked != NONE)
+            {
+                int val = peeked;
+                peeked = NONE;
+                return val;
+            }
+            else
+                return wrapped.next();
+        }
+
+        public int peek()
+        {
+            if (peeked == NONE)
+                peeked = wrapped.next();
+            return peeked;
+        }
+    }
+
+    public static Peekable peekable(ByteSource p)
+    {
+        // When given a null source, we're better off not wrapping it and just returning null. This way existing
+        // code that doesn't know about ByteSource.Peekable, but handles correctly null ByteSources won't be thrown
+        // off by a non-null instance that semantically should have been null.
+        if (p == null)
+            return null;
+        return (p instanceof Peekable)
+               ? (Peekable) p
+               : new Peekable(p);
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java
new file mode 100644
index 0000000000..16b66798c2
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java
@@ -0,0 +1,471 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.utils.bytecomparable;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.db.marshal.ValueAccessor;
+
+/**
+ * Contains inverse transformation utilities for {@link ByteSource}s.
+ *
+ * See ByteComparable.md for details about the encoding scheme.
+ */
+public final class ByteSourceInverse
+{
+    private static final int INITIAL_BUFFER_CAPACITY = 32;
+    private static final int BYTE_ALL_BITS = 0xFF;
+    private static final int BYTE_NO_BITS = 0x00;
+    private static final int BYTE_SIGN_BIT = 1 << 7;
+    private static final int SHORT_SIGN_BIT = 1 << 15;
+    private static final int INT_SIGN_BIT = 1 << 31;
+    private static final long LONG_SIGN_BIT = 1L << 63;
+
+    /**
+     * Consume the given number of bytes and produce a long from them, effectively treating the bytes as a big-endian
+     * unsigned encoding of the number.
+     */
+    public static long getUnsignedFixedLengthAsLong(ByteSource byteSource, int length)
+    {
+        Preconditions.checkNotNull(byteSource);
+        Preconditions.checkArgument(length >= 1 && length <= 8, "Between 1 and 8 bytes can be read at a time");
+
+        long result = 0;
+        for (int i = 0; i < length; ++i)
+            result = (result << 8) | getAndCheckByte(byteSource, i, length);  // note: this must use the unsigned byte value
+
+        return result;
+    }
+
+    /**
+     * Produce the bytes for an encoded signed fixed-length number.
+     * The first byte has its sign bit inverted, and the rest are passed unchanged.
+     */
+    public static <V> V getSignedFixedLength(ValueAccessor<V> accessor, ByteSource byteSource, int length)
+    {
+        Preconditions.checkNotNull(byteSource);
+        Preconditions.checkArgument(length >= 1, "At least 1 byte should be read");
+
+        V result = accessor.allocate(length);
+        // The first byte needs to have its sign flipped
+        accessor.putByte(result, 0, (byte) (getAndCheckByte(byteSource, 0, length) ^ BYTE_SIGN_BIT));
+        // and the rest can be retrieved unchanged.
+        for (int i = 1; i < length; ++i)
+            accessor.putByte(result, i, (byte) getAndCheckByte(byteSource, i, length));
+        return result;
+    }
+
+    /**
+     * Produce the bytes for an encoded signed fixed-length number, also translating null to empty buffer.
+     * The first byte has its sign bit inverted, and the rest are passed unchanged.
+     */
+    public static <V> V getOptionalSignedFixedLength(ValueAccessor<V> accessor, ByteSource byteSource, int length)
+    {
+        return byteSource == null ? accessor.empty() : getSignedFixedLength(accessor, byteSource, length);
+    }
+
+    /**
+     * Produce the bytes for an encoded signed fixed-length floating-point number.
+     * If sign bit is on, returns negated bytes. If not, clears the sign bit and passes the rest of the bytes unchanged.
+     */
+    public static <V> V getSignedFixedLengthFloat(ValueAccessor<V> accessor, ByteSource byteSource, int length)
+    {
+        Preconditions.checkNotNull(byteSource);
+        Preconditions.checkArgument(length >= 1, "At least 1 byte should be read");
+
+        V result = accessor.allocate(length);
+
+        int xor;
+        int first = getAndCheckByte(byteSource, 0, length);
+        if (first < 0x80)
+        {
+            // Negative number. Invert all bits.
+            xor = BYTE_ALL_BITS;
+            first ^= xor;
+        }
+        else
+        {
+            // Positive number. Invert only the sign bit.
+            xor = BYTE_NO_BITS;
+            first ^= BYTE_SIGN_BIT;
+        }
+        accessor.putByte(result, 0, (byte) first);
+
+        // xor is now applied to the rest of the bytes to flip their bits if necessary.
+        for (int i = 1; i < length; ++i)
+            accessor.putByte(result, i, (byte) (getAndCheckByte(byteSource, i, length) ^ xor));
+
+        return result;
+    }
+
+    /**
+     * Produce the bytes for an encoded signed fixed-length floating-point number, also translating null to an empty
+     * buffer.
+     * If sign bit is on, returns negated bytes. If not, clears the sign bit and passes the rest of the bytes unchanged.
+     */
+    public static <V> V getOptionalSignedFixedLengthFloat(ValueAccessor<V> accessor, ByteSource byteSource, int length)
+    {
+        return byteSource == null ? accessor.empty() : getSignedFixedLengthFloat(accessor, byteSource, length);
+    }
+
+    /**
+     * Consume the next length bytes from the source unchanged.
+     */
+    public static <V> V getFixedLength(ValueAccessor<V> accessor, ByteSource byteSource, int length)
+    {
+        Preconditions.checkNotNull(byteSource);
+        Preconditions.checkArgument(length >= 1, "At least 1 byte should be read");
+
+        V result = accessor.allocate(length);
+        for (int i = 0; i < length; ++i)
+            accessor.putByte(result, i, (byte) getAndCheckByte(byteSource, i, length));
+        return result;
+    }
+
+    /**
+     * Consume the next length bytes from the source unchanged, also translating null to an empty buffer.
+     */
+    public static <V> V getOptionalFixedLength(ValueAccessor<V> accessor, ByteSource byteSource, int length)
+    {
+        return byteSource == null ? accessor.empty() : getFixedLength(accessor, byteSource, length);
+    }
+
+    /**
+     * Consume the next {@code int} from the current position of the given {@link ByteSource}. The source position is
+     * modified accordingly (moved 4 bytes forward).
+     * <p>
+     * The source is not strictly required to represent just the encoding of an {@code int} value, so theoretically
+     * this API could be used for reading data in 4-byte strides. Nevertheless its usage is fairly limited because:
+     * <ol>
+     *     <li>...it presupposes signed fixed-length encoding for the encoding of the original value</li>
+     *     <li>...it decodes the data returned on each stride as an {@code int} (i.e. it inverts its leading bit)</li>
+     *     <li>...it doesn't provide any meaningful guarantees (with regard to throwing) in case there are not enough
+     *     bytes to read, in case a special escape value was not interpreted as such, etc.</li>
+     * </ol>
+     * </p>
+     *
+     * @param byteSource A non-null byte source, containing at least 4 bytes.
+     */
+    public static int getSignedInt(ByteSource byteSource)
+    {
+        return (int) getUnsignedFixedLengthAsLong(byteSource, 4) ^ INT_SIGN_BIT;
+    }
+
+    /**
+     * Consume the next {@code long} from the current position of the given {@link ByteSource}. The source position is
+     * modified accordingly (moved 8 bytes forward).
+     * <p>
+     * The source is not strictly required to represent just the encoding of a {@code long} value, so theoretically
+     * this API could be used for reading data in 8-byte strides. Nevertheless its usage is fairly limited because:
+     * <ol>
+     *     <li>...it presupposes signed fixed-length encoding for the encoding of the original value</li>
+     *     <li>...it decodes the data returned on each stride as a {@code long} (i.e. it inverts its leading bit)</li>
+     *     <li>...it doesn't provide any meaningful guarantees (with regard to throwing) in case there are not enough
+     *     bytes to read, in case a special escape value was not interpreted as such, etc.</li>
+     * </ol>
+     * </p>
+     *
+     * @param byteSource A non-null byte source, containing at least 8 bytes.
+     */
+    public static long getSignedLong(ByteSource byteSource)
+    {
+        return getUnsignedFixedLengthAsLong(byteSource, 8) ^ LONG_SIGN_BIT;
+    }
+
+    /**
+     * Converts the given {@link ByteSource} to a {@code byte}.
+     *
+     * @param byteSource A non-null byte source, containing at least 1 byte.
+     */
+    public static byte getSignedByte(ByteSource byteSource)
+    {
+        return (byte) (getAndCheckByte(Preconditions.checkNotNull(byteSource), 0, 1) ^ BYTE_SIGN_BIT);
+    }
+
+    /**
+     * Converts the given {@link ByteSource} to a {@code short}. All terms and conditions valid for
+     * {@link #getSignedInt(ByteSource)} and {@link #getSignedLong(ByteSource)} translate to this as well.
+     *
+     * @param byteSource A non-null byte source, containing at least 2 bytes.
+     *
+     * @see #getSignedInt(ByteSource)
+     * @see #getSignedLong(ByteSource)
+     */
+    public static short getSignedShort(ByteSource byteSource)
+    {
+        return (short) (getUnsignedFixedLengthAsLong(byteSource, 2) ^ SHORT_SIGN_BIT);
+    }
+
+    /**
+     * Decode a variable-length signed integer.
+     */
+    public static long getVariableLengthInteger(ByteSource byteSource)
+    {
+        int signAndMask = getAndCheckByte(byteSource);
+
+        long sum = 0;
+        int bytes;
+        // For every bit after the sign that matches the sign, read one more byte.
+        for (bytes = 0; bytes < 7 && sameByteSign(signAndMask << (bytes + 1), signAndMask); ++bytes)
+            sum = (sum << 8) | getAndCheckByte(byteSource);
+
+        // The eighth length bit is stored in the second byte.
+        if (bytes == 7 && sameByteSign((int) (sum >> 48), signAndMask))
+            return ((sum << 8) | getAndCheckByte(byteSource)) ^ LONG_SIGN_BIT;    // 9-byte encoding, use bytes 2-9 with inverted sign
+        else
+        {
+            sum |= (((long) signAndMask) << bytes * 8);     // add the rest of the bits
+            long signMask = -0x40L << bytes * 7;            // mask of the bits that should be replaced by the sign
+            long sign = (byte) (signAndMask ^ 0x80) >> 7;   // -1 if negative (0 leading bit), 0 otherwise
+            return sum & ~signMask | sign & signMask;
+        }
+    }
+
+    /**
+     * Decode a variable-length unsigned integer, passing all bytes read through XOR with the given xorWith parameter.
+     *
+     * Used in BigInteger encoding to read number length, where negative numbers have their length negated
+     * (i.e. xorWith = 0xFF) to ensure correct ordering.
+     */
+    public static long getVariableLengthUnsignedIntegerXoring(ByteSource byteSource, int xorWith)
+    {
+        int signAndMask = getAndCheckByte(byteSource) ^ xorWith;
+
+        long sum = 0;
+        int bytes;
+        // Read an extra byte while the next most significant bit is 1.
+        for (bytes = 0; bytes <= 7 && ((signAndMask << bytes) & 0x80) != 0; ++bytes)
+            sum = (sum << 8) | getAndCheckByte(byteSource) ^ xorWith;
+
+        // Strip the length bits from the leading byte.
+        signAndMask &= ~(-256 >> bytes);
+        return sum | (((long) signAndMask) << bytes * 8);     // Add the rest of the bits of the leading byte.
+    }
+
+    /** Returns true if the two parameters treated as bytes have the same sign. */
+    private static boolean sameByteSign(int a, int b)
+    {
+        return ((a ^ b) & 0x80) == 0;
+    }
+
+
+    private static int getAndCheckByte(ByteSource byteSource)
+    {
+        return getAndCheckByte(byteSource, -1, -1);
+    }
+
+    private static int getAndCheckByte(ByteSource byteSource, int pos, int length)
+    {
+        int data = byteSource.next();
+        if (data == ByteSource.END_OF_STREAM)
+            throw new IllegalArgumentException(
+                length > 0 ? String.format("Unexpected end of stream reached after %d bytes (expected >= %d)", pos, length)
+                           : "Unexpected end of stream");
+        assert data >= BYTE_NO_BITS && data <= BYTE_ALL_BITS
+            : "A ByteSource must produce unsigned bytes and end in END_OF_STREAM";
+        return data;
+    }
+
+    /**
+     * Reads a single variable-length byte sequence (blob, string, ...) encoded according to the scheme described
+     * in ByteComparable.md, decoding it back to its original, unescaped form.
+     *
+     * @param byteSource The source of the variable-length bytes sequence.
+     * @return A byte array containing the original, unescaped bytes of the given source. Unescaped here means
+     * not including any of the escape sequences of the encoding scheme used for variable-length byte sequences.
+     */
+    public static byte[] getUnescapedBytes(ByteSource.Peekable byteSource)
+    {
+        return byteSource == null ? null : readBytes(unescape(byteSource));
+    }
+
+    /**
+     * As above, but converts the result to a ByteSource.
+     */
+    public static ByteSource unescape(ByteSource.Peekable byteSource)
+    {
+        return new ByteSource() {
+            boolean escaped = false;
+
+            @Override
+            public int next()
+            {
+                if (!escaped)
+                {
+                    int data = byteSource.next(); // we consume this byte no matter what it is
+                    if (data > ByteSource.ESCAPE)
+                        return data;        // most used path leads here
+
+                    assert data != ByteSource.END_OF_STREAM : "Invalid escaped byte sequence";
+                    escaped = true;
+                }
+
+                int next = byteSource.peek();
+                switch (next)
+                {
+                    case END_OF_STREAM:
+                        // The end of a byte-comparable outside of a multi-component sequence. No matter what we have
+                        // seen or peeked before, we should stop now.
+                        byteSource.next();
+                        return END_OF_STREAM;
+                    case ESCAPED_0_DONE:
+                        // The end of 1 or more consecutive 0x00 value bytes.
+                        escaped = false;
+                        byteSource.next();
+                        return ESCAPE;
+                    case ESCAPED_0_CONT:
+                        // Escaped sequence continues
+                        byteSource.next();
+                        return ESCAPE;
+                    default:
+                        // An ESCAPE or ESCAPED_0_CONT won't be followed by either another ESCAPED_0_CONT, an
+                        // ESCAPED_0_DONE, or an END_OF_STREAM only when the byte-comparable is part of a multi-component
+                        // sequence and we have reached the end of the encoded byte-comparable. In this case, the byte
+                        // we have just peeked is the separator or terminator byte between or at the end of components
+                        // (which by contact must be 0x10 - 0xFE, which cannot conflict with our special bytes).
+                        assert next >= ByteSource.MIN_SEPARATOR && next <= ByteSource.MAX_SEPARATOR : next;
+                        // Unlike above, we don't consume this byte (the sequence decoding needs it).
+                        return END_OF_STREAM;
+                }
+            }
+        };
+    }
+
+    /**
+     * Reads the bytes of the given source into a byte array. Doesn't do any transformation on the bytes, just reads
+     * them until it reads an {@link ByteSource#END_OF_STREAM} byte, after which it returns an array of all the read
+     * bytes, <strong>excluding the {@link ByteSource#END_OF_STREAM}</strong>.
+     * <p>
+     * This method sizes a tentative internal buffer array at {@code initialBufferCapacity}.  However, if
+     * {@code byteSource} exceeds this size, the buffer array is recreated with doubled capacity as many times as
+     * necessary.  If, after {@code byteSource} is fully exhausted, the number of bytes read from it does not exactly
+     * match the current size of the tentative buffer array, then it is copied into another array sized to fit the
+     * number of bytes read; otherwise, it is returned without that final copy step.
+     *
+     * @param byteSource The source which bytes we're interested in.
+     * @param initialBufferCapacity The initial size of the internal buffer.
+     * @return A byte array containing exactly all the read bytes. In case of a {@code null} source, the returned byte
+     * array will be empty.
+     */
+    public static byte[] readBytes(ByteSource byteSource, final int initialBufferCapacity)
+    {
+        Preconditions.checkNotNull(byteSource);
+
+        int readBytes = 0;
+        byte[] buf = new byte[initialBufferCapacity];
+        int data;
+        while ((data = byteSource.next()) != ByteSource.END_OF_STREAM)
+        {
+            buf = ensureCapacity(buf, readBytes);
+            buf[readBytes++] = (byte) data;
+        }
+
+        if (readBytes != buf.length)
+        {
+            buf = Arrays.copyOf(buf, readBytes);
+        }
+        return buf;
+    }
+
+    /**
+     * Reads the bytes of the given source into a byte array. Doesn't do any transformation on the bytes, just reads
+     * them until it reads an {@link ByteSource#END_OF_STREAM} byte, after which it returns an array of all the read
+     * bytes, <strong>excluding the {@link ByteSource#END_OF_STREAM}</strong>.
+     * <p>
+     * This is equivalent to {@link #readBytes(ByteSource, int)} where the second actual parameter is
+     * {@linkplain #INITIAL_BUFFER_CAPACITY} ({@value INITIAL_BUFFER_CAPACITY}).
+     *
+     * @param byteSource The source which bytes we're interested in.
+     * @return A byte array containing exactly all the read bytes. In case of a {@code null} source, the returned byte
+     * array will be empty.
+     */
+    public static byte[] readBytes(ByteSource byteSource)
+    {
+        return readBytes(byteSource, INITIAL_BUFFER_CAPACITY);
+    }
+
+    /**
+     * 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.
+     */
+    private static byte[] ensureCapacity(byte[] buf, int dataLengthInBytes)
+    {
+        if (dataLengthInBytes == buf.length)
+            // We won't gain much with guarding against overflow. We'll overflow when dataLengthInBytes >= 1 << 30,
+            // and if we do guard, we'll be able to extend the capacity to Integer.MAX_VALUE (which is 1 << 31 - 1).
+            // Controlling the exception that will be thrown shouldn't matter that much, and  in practice, we almost
+            // surely won't be reading gigabytes of ByteSource data at once.
+            return Arrays.copyOf(buf, dataLengthInBytes * 2);
+        else
+            return buf;
+    }
+
+    /**
+     * Converts the given {@link ByteSource} to a UTF-8 {@link String}.
+     *
+     * @param byteSource The source we're interested in.
+     * @return A UTF-8 string corresponding to the given source.
+     */
+    public static String getString(ByteSource.Peekable byteSource)
+    {
+        if (byteSource == null)
+            return null;
+
+        byte[] data = getUnescapedBytes(byteSource);
+
+        return new String(data, StandardCharsets.UTF_8);
+    }
+
+    /*
+     * Multi-component sequence utilities.
+     */
+
+    /**
+     * A utility for consuming components from a peekable multi-component sequence.
+     * It uses the component separators, so the given sequence needs to have its last component fully consumed, in
+     * order for the next consumable byte to be a separator. Identifying the end of the component that will then be
+     * consumed is the responsibility of the consumer (the user of this method).
+     * @param source A peekable multi-component sequence, which next byte is a component separator.
+     * @return the given multi-component sequence if its next component is not null, or {@code null} if it is.
+     */
+    public static ByteSource.Peekable nextComponentSource(ByteSource.Peekable source)
+    {
+        return nextComponentSource(source, source.next());
+    }
+
+    /**
+     * A utility for consuming components from a peekable multi-component sequence, very similar to
+     * {@link #nextComponentSource(ByteSource.Peekable)} - the difference being that here the separator can be passed
+     * in case it had to be consumed beforehand.
+     */
+    public static ByteSource.Peekable nextComponentSource(ByteSource.Peekable source, int separator)
+    {
+        return nextComponentNull(separator)
+               ? null
+               : source;
+    }
+
+    public static boolean nextComponentNull(int separator)
+    {
+        return separator == ByteSource.NEXT_COMPONENT_NULL || separator == ByteSource.NEXT_COMPONENT_EMPTY
+               || separator == ByteSource.NEXT_COMPONENT_EMPTY_REVERSED;
+    }
+}
diff --git a/test/microbench/org/apache/cassandra/test/microbench/AbstractTypeByteSourceDecodingBench.java b/test/microbench/org/apache/cassandra/test/microbench/AbstractTypeByteSourceDecodingBench.java
new file mode 100644
index 0000000000..427265ec9e
--- /dev/null
+++ b/test/microbench/org/apache/cassandra/test/microbench/AbstractTypeByteSourceDecodingBench.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.test.microbench;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+
+import net.nicoulaj.compilecommand.annotations.Inline;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MICROSECONDS)
+@Warmup(iterations = 5, time = 1)
+@Measurement(iterations = 5, time = 2)
+@Fork(value = 1,jvmArgsAppend = { "-Xmx4G", "-Xms4G", "-Djmh.executor=CUSTOM", "-Djmh.executor.class=org.apache.cassandra.test.microbench.FastThreadExecutor"})
+@Threads(1)
+@State(Scope.Benchmark)
+public class AbstractTypeByteSourceDecodingBench
+{
+
+    private static final ByteComparable.Version LATEST = ByteComparable.Version.OSS42;
+
+    private static final Map<AbstractType, BiFunction<Random, Integer, ByteSource.Peekable>> PEEKABLE_GENERATOR_BY_TYPE = new HashMap<>();
+    static
+    {
+        PEEKABLE_GENERATOR_BY_TYPE.put(UTF8Type.instance, (prng, length) ->
+        {
+            byte[] randomBytes = new byte[length];
+            prng.nextBytes(randomBytes);
+            return ByteSource.peekable(ByteSource.of(new String(randomBytes, StandardCharsets.UTF_8), LATEST));
+        });
+        PEEKABLE_GENERATOR_BY_TYPE.put(BytesType.instance, (prng, length) ->
+        {
+            byte[] randomBytes = new byte[length];
+            prng.nextBytes(randomBytes);
+            return ByteSource.peekable(ByteSource.of(randomBytes, LATEST));
+        });
+        PEEKABLE_GENERATOR_BY_TYPE.put(IntegerType.instance, (prng, length) ->
+        {
+            BigInteger randomVarint = BigInteger.valueOf(prng.nextLong());
+            for (int i = 1; i < length / 8; ++i)
+                randomVarint = randomVarint.multiply(BigInteger.valueOf(prng.nextLong()));
+            return ByteSource.peekable(IntegerType.instance.asComparableBytes(IntegerType.instance.decompose(randomVarint), LATEST));
+        });
+        PEEKABLE_GENERATOR_BY_TYPE.put(DecimalType.instance, (prng, length) ->
+        {
+            BigInteger randomMantissa = BigInteger.valueOf(prng.nextLong());
+            for (int i = 1; i < length / 8; ++i)
+                randomMantissa = randomMantissa.multiply(BigInteger.valueOf(prng.nextLong()));
+            int randomScale = prng.nextInt(Integer.MAX_VALUE >> 1) + Integer.MAX_VALUE >> 1;
+            BigDecimal randomDecimal = new BigDecimal(randomMantissa, randomScale);
+            return ByteSource.peekable(DecimalType.instance.asComparableBytes(DecimalType.instance.decompose(randomDecimal), LATEST));
+        });
+    }
+
+    private Random prng = new Random();
+
+    @Param({"32", "128", "512"})
+    private int length;
+
+    @Param({"UTF8Type", "BytesType", "IntegerType", "DecimalType"})
+    private String abstractTypeName;
+
+    private AbstractType abstractType;
+    private BiFunction<Random, Integer, ByteSource.Peekable> peekableGenerator;
+
+    @Setup(Level.Trial)
+    public void setup()
+    {
+        abstractType = TypeParser.parse(abstractTypeName);
+        peekableGenerator = PEEKABLE_GENERATOR_BY_TYPE.get(abstractType);
+    }
+
+    @Inline
+    private ByteSource.Peekable randomPeekableBytes()
+    {
+        return peekableGenerator.apply(prng, length);
+    }
+
+    @Benchmark
+    public int baseline()
+    {
+        // Getting the source is not enough as its content is produced on next() calls.
+        ByteSource.Peekable source = randomPeekableBytes();
+        int count = 0;
+        while (source.next() != ByteSource.END_OF_STREAM)
+            ++count;
+        return count;
+    }
+
+    @Benchmark
+    public ByteBuffer fromComparableBytes()
+    {
+        ByteSource.Peekable peekableBytes = randomPeekableBytes();
+        return abstractType.fromComparableBytes(peekableBytes, ByteComparable.Version.OSS42);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 0459cb3b3b..b0b5964117 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -23,6 +23,7 @@ import java.io.Closeable;
 import java.io.EOFException;
 import java.io.IOError;
 import java.io.IOException;
+import java.math.BigInteger;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
@@ -859,6 +860,199 @@ public class Util
         assertEquals(expectedSSTableCount, fileCount);
     }
 
+    public static ByteBuffer generateMurmurCollision(ByteBuffer original, byte... bytesToAdd)
+    {
+        // Round size up to 16, and add another 16 bytes
+        ByteBuffer collision = ByteBuffer.allocate((original.remaining() + bytesToAdd.length + 31) & -16);
+        collision.put(original);    // we can use this as a copy of original with 0s appended at the end
+
+        original.flip();
+
+        long c1 = 0x87c37b91114253d5L;
+        long c2 = 0x4cf5ad432745937fL;
+
+        long h1 = 0;
+        long h2 = 0;
+
+        // Get hash of original
+        int index = 0;
+        final int length = original.limit();
+        while (index <= length - 16)
+        {
+            long k1 = Long.reverseBytes(collision.getLong(index + 0));
+            long k2 = Long.reverseBytes(collision.getLong(index + 8));
+
+            // 16 bytes
+            k1 *= c1;
+            k1 = rotl64(k1, 31);
+            k1 *= c2;
+            h1 ^= k1;
+            h1 = rotl64(h1, 27);
+            h1 += h2;
+            h1 = h1 * 5 + 0x52dce729;
+            k2 *= c2;
+            k2 = rotl64(k2, 33);
+            k2 *= c1;
+            h2 ^= k2;
+            h2 = rotl64(h2, 31);
+            h2 += h1;
+            h2 = h2 * 5 + 0x38495ab5;
+
+            index += 16;
+        }
+
+        long oh1 = h1;
+        long oh2 = h2;
+
+        // Process final unfilled chunk, but only adjust the original hash value
+        if (index < length)
+        {
+            long k1 = Long.reverseBytes(collision.getLong(index + 0));
+            long k2 = Long.reverseBytes(collision.getLong(index + 8));
+
+            // 16 bytes
+            k1 *= c1;
+            k1 = rotl64(k1, 31);
+            k1 *= c2;
+            oh1 ^= k1;
+
+            k2 *= c2;
+            k2 = rotl64(k2, 33);
+            k2 *= c1;
+            oh2 ^= k2;
+        }
+
+        // These are the hashes the original would provide, before final mixing
+        oh1 ^= original.capacity();
+        oh2 ^= original.capacity();
+
+        // Fill in the remaining bytes before the last 16 and get their hash
+        collision.put(bytesToAdd);
+        while ((collision.position() & 0x0f) != 0)
+            collision.put((byte) 0);
+
+        while (index < collision.position())
+        {
+            long k1 = Long.reverseBytes(collision.getLong(index + 0));
+            long k2 = Long.reverseBytes(collision.getLong(index + 8));
+
+            // 16 bytes
+            k1 *= c1;
+            k1 = rotl64(k1, 31);
+            k1 *= c2;
+            h1 ^= k1;
+            h1 = rotl64(h1, 27);
+            h1 += h2;
+            h1 = h1 * 5 + 0x52dce729;
+            k2 *= c2;
+            k2 = rotl64(k2, 33);
+            k2 *= c1;
+            h2 ^= k2;
+            h2 = rotl64(h2, 31);
+            h2 += h1;
+            h2 = h2 * 5 + 0x38495ab5;
+
+            index += 16;
+        }
+
+        // Working backwards, we must get this hash pair
+        long th1 = h1;
+        long th2 = h2;
+
+        // adjust ohx with length
+        h1 = oh1 ^ collision.capacity();
+        h2 = oh2 ^ collision.capacity();
+
+        // Get modulo-long inverses of the multipliers used in the computation
+        long i5i = inverse(5L);
+        long c1i = inverse(c1);
+        long c2i = inverse(c2);
+
+        // revert one step
+        h2 -= 0x38495ab5;
+        h2 *= i5i;
+        h2 -= h1;
+        h2 = rotl64(h2, 33);
+
+        h1 -= 0x52dce729;
+        h1 *= i5i;
+        h1 -= th2;  // use h2 before it's adjusted with k2
+        h1 = rotl64(h1, 37);
+
+        // extract the required modifiers and applies the inverse of their transformation
+        long k1 = h1 ^ th1;
+        k1 = c2i * k1;
+        k1 = rotl64(k1, 33);
+        k1 = c1i * k1;
+
+        long k2 = h2 ^ th2;
+        k2 = c1i * k2;
+        k2 = rotl64(k2, 31);
+        k2 = c2i * k2;
+
+        collision.putLong(Long.reverseBytes(k1));
+        collision.putLong(Long.reverseBytes(k2));
+        collision.flip();
+
+        return collision;
+    }
+
+    // Assumes a and b are positive
+    private static BigInteger[] xgcd(BigInteger a, BigInteger b) {
+        BigInteger x = a, y = b;
+        BigInteger[] qrem;
+        BigInteger[] result = new BigInteger[3];
+        BigInteger x0 = BigInteger.ONE, x1 = BigInteger.ZERO;
+        BigInteger y0 = BigInteger.ZERO, y1 = BigInteger.ONE;
+        while (true)
+        {
+            qrem = x.divideAndRemainder(y);
+            x = qrem[1];
+            x0 = x0.subtract(y0.multiply(qrem[0]));
+            x1 = x1.subtract(y1.multiply(qrem[0]));
+            if (x.equals(BigInteger.ZERO))
+            {
+                result[0] = y;
+                result[1] = y0;
+                result[2] = y1;
+                return result;
+            }
+
+            qrem = y.divideAndRemainder(x);
+            y = qrem[1];
+            y0 = y0.subtract(x0.multiply(qrem[0]));
+            y1 = y1.subtract(x1.multiply(qrem[0]));
+            if (y.equals(BigInteger.ZERO))
+            {
+                result[0] = x;
+                result[1] = x0;
+                result[2] = x1;
+                return result;
+            }
+        }
+    }
+
+    /**
+     * Find a mupltiplicative inverse for the given multiplier for long, i.e.
+     * such that x * inverse(x) = 1 where * is long multiplication.
+     * In other words, such an integer that x * inverse(x) == 1 (mod 2^64).
+     */
+    public static long inverse(long multiplier)
+    {
+        final BigInteger modulus = BigInteger.ONE.shiftLeft(64);
+        // Add the modulus to the multiplier to avoid problems with negatives (a + m == a (mod m))
+        BigInteger[] gcds = xgcd(BigInteger.valueOf(multiplier).add(modulus), modulus);
+        // xgcd gives g, a and b, such that ax + bm = g
+        // ie, ax = g (mod m). Return a
+        assert gcds[0].equals(BigInteger.ONE) : "Even number " + multiplier + " has no long inverse";
+        return gcds[1].longValueExact();
+    }
+
+    public static long rotl64(long v, int n)
+    {
+        return ((v << n) | (v >>> (64 - n)));
+    }
+
     /**
      * Disable bloom filter on all sstables of given table
      */
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
index 9f53db4966..f325655542 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
@@ -33,6 +33,8 @@ import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.SchemaCQLHelper;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.TupleType;
 import org.apache.cassandra.utils.AbstractTypeGenerators.TypeSupport;
 import org.quicktheories.core.Gen;
@@ -266,7 +268,7 @@ public class TupleTypeTest extends CQLTester
             for (ByteBuffer value : testcase.uniqueRows)
             {
                 map.put(value, count);
-                ByteBuffer[] tupleBuffers = tupleType.split(value);
+                ByteBuffer[] tupleBuffers = tupleType.split(ByteBufferAccessor.instance, value);
 
                 // use cast to avoid warning
                 execute("INSERT INTO %s (id, value) VALUES (?, ?)", tuple((Object[]) tupleBuffers), count);
@@ -304,7 +306,7 @@ public class TupleTypeTest extends CQLTester
             for (ByteBuffer value : testcase.uniqueRows)
             {
                 map.put(value, count);
-                ByteBuffer[] tupleBuffers = tupleType.split(value);
+                ByteBuffer[] tupleBuffers = tupleType.split(ByteBufferAccessor.instance, value);
 
                 // use cast to avoid warning
                 execute("INSERT INTO %s (pk, ck, value) VALUES (?, ?, ?)", 1, tuple((Object[]) tupleBuffers), count);
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index 0b05e8f013..1520b4cab9 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -199,6 +199,56 @@ public class UserTypesTest extends CQLTester
         );
     }
 
+    @Test
+    public void testNullsInIntUDT() throws Throwable
+    {
+        String myType = KEYSPACE + '.' + createType("CREATE TYPE %s (a int)");
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
+        execute("INSERT INTO %s (a, b) VALUES (1, ?)", userType("a", 1));
+
+        assertRows(execute("SELECT b.a FROM %s"), row(1));
+
+        flush();
+
+        schemaChange("ALTER TYPE " + myType + " ADD b int");
+        execute("INSERT INTO %s (a, b) VALUES (2, {a: 2, b: 2})");
+        execute("INSERT INTO %s (a, b) VALUES (3, {b: 3})");
+        execute("INSERT INTO %s (a, b) VALUES (4, {a: null, b: 4})");
+
+        beforeAndAfterFlush(() ->
+                            assertRows(execute("SELECT b.a, b.b FROM %s"),
+                                       row(1, null),
+                                       row(2, 2),
+                                       row(null, 3),
+                                       row(null, 4))
+        );
+    }
+
+    @Test
+    public void testNullsInTextUDT() throws Throwable
+    {
+        String myType = KEYSPACE + '.' + createType("CREATE TYPE %s (a text)");
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
+        execute("INSERT INTO %s (a, b) VALUES (1, {a: ''})");
+
+        assertRows(execute("SELECT b.a FROM %s"), row(""));
+
+        flush();
+
+        schemaChange("ALTER TYPE " + myType + " ADD b text");
+        execute("INSERT INTO %s (a, b) VALUES (2, {a: '', b: ''})");
+        execute("INSERT INTO %s (a, b) VALUES (3, {b: ''})");
+        execute("INSERT INTO %s (a, b) VALUES (4, {a: null, b: ''})");
+
+        beforeAndAfterFlush(() ->
+                            assertRows(execute("SELECT b.a, b.b FROM %s"),
+                                       row("", null),
+                                       row("", ""),
+                                       row(null, ""),
+                                       row(null, ""))
+        );
+    }
+
     @Test
     public void testAlterNonFrozenUDT() throws Throwable
     {
diff --git a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
index d22a8ac933..69c1eb5cd8 100644
--- a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
@@ -47,7 +47,7 @@ public class DynamicCompositeTypeTest
 {
     private static final String KEYSPACE1 = "DynamicCompositeType";
     private static final String CF_STANDARDDYNCOMPOSITE = "StandardDynamicComposite";
-    private static Map<Byte, AbstractType<?>> aliases = new HashMap<>();
+    public static Map<Byte, AbstractType<?>> aliases = new HashMap<>();
 
     private static final DynamicCompositeType comparator;
     static
@@ -60,7 +60,7 @@ public class DynamicCompositeTypeTest
     }
 
     private static final int UUID_COUNT = 3;
-    private static final UUID[] uuids = new UUID[UUID_COUNT];
+    public static final UUID[] uuids = new UUID[UUID_COUNT];
     static
     {
         for (int i = 0; i < UUID_COUNT; ++i)
@@ -320,13 +320,12 @@ public class DynamicCompositeTypeTest
         assert !TypeParser.parse("DynamicCompositeType(a => BytesType)").isCompatibleWith(TypeParser.parse("DynamicCompositeType(a => BytesType, b => AsciiType)"));
     }
 
-    private ByteBuffer createDynamicCompositeKey(String s, UUID uuid, int i, boolean lastIsOne)
+    private static ByteBuffer createDynamicCompositeKey(String s, UUID uuid, int i, boolean lastIsOne)
     {
         return createDynamicCompositeKey(s, uuid, i, lastIsOne, false);
     }
 
-    private ByteBuffer createDynamicCompositeKey(String s, UUID uuid, int i, boolean lastIsOne,
-            final boolean reversed)
+    public static ByteBuffer createDynamicCompositeKey(String s, UUID uuid, int i, boolean lastIsOne, boolean reversed)
     {
         String intType = (reversed ? "ReversedType(IntegerType)" : "IntegerType");
         ByteBuffer bytes = ByteBufferUtil.bytes(s);
diff --git a/test/unit/org/apache/cassandra/db/marshal/TypeValidationTest.java b/test/unit/org/apache/cassandra/db/marshal/TypeValidationTest.java
index 4d25a1f62b..474b867007 100644
--- a/test/unit/org/apache/cassandra/db/marshal/TypeValidationTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/TypeValidationTest.java
@@ -204,7 +204,7 @@ public class TypeValidationTest
         qt().forAll(tupleWithValueGen(baseGen)).checkAssert(pair -> {
             TupleType tuple = pair.left;
             ByteBuffer value = pair.right;
-            Assertions.assertThat(TupleType.buildValue(tuple.split(value)))
+            Assertions.assertThat(TupleType.buildValue(tuple.split(ByteBufferAccessor.instance, value)))
                       .as("TupleType.buildValue(split(value)) == value")
                       .isEqualTo(value);
         });
diff --git a/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java b/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
index 5b5365da09..c24690b8bf 100644
--- a/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
@@ -27,6 +27,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -36,6 +37,8 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -124,5 +127,11 @@ public class KeyCollisionTest
         {
             return 0;
         }
+
+        @Override
+        public ByteSource asComparableBytes(ByteComparable.Version version)
+        {
+            return IntegerType.instance.asComparableBytes(IntegerType.instance.decompose(token), version);
+        }
     }
 }
diff --git a/test/unit/org/apache/cassandra/dht/LengthPartitioner.java b/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
index c4e5db8284..ca6504ced8 100644
--- a/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
+++ b/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
@@ -34,6 +34,8 @@ import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
 
 public class LengthPartitioner implements IPartitioner
 {
@@ -95,6 +97,11 @@ public class LengthPartitioner implements IPartitioner
             return new BigIntegerToken(new BigInteger(ByteBufferUtil.getArray(bytes)));
         }
 
+        public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version)
+        {
+            return fromByteArray(IntegerType.instance.fromComparableBytes(comparableBytes, version));
+        }
+
         public String toString(Token token)
         {
             BigIntegerToken bigIntegerToken = (BigIntegerToken) token;
diff --git a/test/unit/org/apache/cassandra/transport/SerDeserTest.java b/test/unit/org/apache/cassandra/transport/SerDeserTest.java
index da76070b6f..75523e1587 100644
--- a/test/unit/org/apache/cassandra/transport/SerDeserTest.java
+++ b/test/unit/org/apache/cassandra/transport/SerDeserTest.java
@@ -238,7 +238,7 @@ public class SerDeserTest
 
         ByteBuffer serialized = t.bindAndGet(options);
 
-        ByteBuffer[] fields = udt.split(serialized);
+        ByteBuffer[] fields = udt.split(ByteBufferAccessor.instance, serialized);
 
         assertEquals(4, fields.length);
 
diff --git a/test/unit/org/apache/cassandra/utils/bytecomparable/AbstractTypeByteSourceTest.java b/test/unit/org/apache/cassandra/utils/bytecomparable/AbstractTypeByteSourceTest.java
new file mode 100644
index 0000000000..d5e2f1eea9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/bytecomparable/AbstractTypeByteSourceTest.java
@@ -0,0 +1,1015 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.utils.bytecomparable;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.marshal.*;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.cql3.Duration;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.LengthPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.TimeUUID;
+import org.apache.cassandra.utils.UUIDGen;
+
+@RunWith(Parameterized.class)
+public class AbstractTypeByteSourceTest
+{
+    private static final String ALPHABET = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ1234567890!@#$%^&*()";
+
+    @Parameterized.Parameters(name = "version={0}")
+    public static Iterable<ByteComparable.Version> versions()
+    {
+        return ImmutableList.of(ByteComparable.Version.OSS42);
+    }
+
+    private final ByteComparable.Version version;
+
+    public AbstractTypeByteSourceTest(ByteComparable.Version version)
+    {
+        this.version = version;
+    }
+
+    private <T> void testValuesForType(AbstractType<T> type, T... values)
+    {
+        testValuesForType(type, Arrays.asList(values));
+    }
+
+    private <T> void testValuesForType(AbstractType<T> type, List<T> values)
+    {
+        for (T initial : values)
+            decodeAndAssertEquals(type, initial);
+        if (IntegerType.instance.equals(type))
+            // IntegerType tests go through A LOT of values, so short of randomly picking up to, let's say 1000
+            // values to combine with, we'd rather skip the comparison tests for them.
+            return;
+        for (int i = 0; i < values.size(); ++i)
+        {
+            for (int j = i + 1; j < values.size(); ++j)
+            {
+                ByteBuffer left = type.decompose(values.get(i));
+                ByteBuffer right = type.decompose(values.get(j));
+                int compareBuffers = Integer.signum(type.compare(left, right));
+                ByteSource leftSource = type.asComparableBytes(left.duplicate(), version);
+                ByteSource rightSource = type.asComparableBytes(right.duplicate(), version);
+                int compareBytes = Integer.signum(ByteComparable.compare(v -> leftSource, v -> rightSource, version));
+                Assert.assertEquals(compareBuffers, compareBytes);
+            }
+        }
+    }
+
+    private <T> void testValuesForType(AbstractType<T> type, Stream<T> values)
+    {
+        values.forEach(initial -> decodeAndAssertEquals(type, initial));
+    }
+
+    private <T> void decodeAndAssertEquals(AbstractType<T> type, T initial)
+    {
+        ByteBuffer initialBuffer = type.decompose(initial);
+        // Assert that fromComparableBytes decodes correctly.
+        ByteSource.Peekable peekableBytes = ByteSource.peekable(type.asComparableBytes(initialBuffer, version));
+        ByteBuffer decodedBuffer = type.fromComparableBytes(peekableBytes, version);
+        Assert.assertEquals("For " + ByteSourceComparisonTest.safeStr(initial),
+                            ByteBufferUtil.bytesToHex(initialBuffer),
+                            ByteBufferUtil.bytesToHex(decodedBuffer));
+        // Assert that the value composed from fromComparableBytes is the correct one.
+        peekableBytes = ByteSource.peekable(type.asComparableBytes(initialBuffer, version));
+        T decoded = type.compose(type.fromComparableBytes(peekableBytes, version));
+        Assert.assertEquals(initial, decoded);
+    }
+
+    private static String newRandomAlphanumeric(Random prng, int length)
+    {
+        StringBuilder random = new StringBuilder(length);
+        for (int i = 0; i < length; ++i)
+            random.append(ALPHABET.charAt(prng.nextInt(ALPHABET.length())));
+        return random.toString();
+    }
+
+    @Test
+    public void testAsciiType()
+    {
+        String[] asciiStrings = new String[]
+        {
+                "",
+                "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz1234567890",
+                "!@#$%^&*()",
+        };
+        testValuesForType(AsciiType.instance, asciiStrings);
+
+        Random prng = new Random();
+        Stream<String> asciiStream = Stream.generate(() -> newRandomAlphanumeric(prng, 10)).limit(1000);
+        testValuesForType(AsciiType.instance, asciiStream);
+    }
+
+    @Test
+    public void testBooleanType()
+    {
+        testValuesForType(BooleanType.instance, Boolean.TRUE, Boolean.FALSE, null);
+    }
+
+    @Test
+    public void testBytesType()
+    {
+        List<ByteBuffer> byteBuffers = new ArrayList<>();
+        Random prng = new Random();
+        byte[] byteArray;
+        int[] arrayLengths = new int[] {1, 10, 100, 1000};
+        for (int length : arrayLengths)
+        {
+            byteArray = new byte[length];
+            for (int i = 0; i < 1000; ++i)
+            {
+                prng.nextBytes(byteArray);
+                byteBuffers.add(ByteBuffer.wrap(byteArray));
+            }
+        }
+        testValuesForType(BytesType.instance, byteBuffers.toArray(new ByteBuffer[0]));
+    }
+
+    @Test
+    public void testByteType()
+    {
+        testValuesForType(ByteType.instance, new Byte[] { null });
+
+        Stream<Byte> allBytes = IntStream.range(Byte.MIN_VALUE, Byte.MAX_VALUE + 1)
+                                         .mapToObj(value -> (byte) value);
+        testValuesForType(ByteType.instance, allBytes);
+    }
+
+    @Test
+    public void testCompositeType()
+    {
+        CompositeType compType = CompositeType.getInstance(UTF8Type.instance, TimeUUIDType.instance, IntegerType.instance);
+        List<ByteBuffer> byteBuffers = new ArrayList<>();
+        Random prng = new Random();
+        // Test with complete CompositeType rows
+        for (int i = 0; i < 1000; ++i)
+        {
+            String randomString = newRandomAlphanumeric(prng, 10);
+            TimeUUID randomUuid = TimeUUID.Generator.nextTimeUUID();
+            BigInteger randomVarint = BigInteger.probablePrime(80, prng);
+            byteBuffers.add(compType.decompose(randomString, randomUuid, randomVarint));
+        }
+        // Test with incomplete CompositeType rows, where only the first element is present
+        ByteBuffer[] incompleteComposite = new ByteBuffer[1];
+        incompleteComposite[0] = UTF8Type.instance.decompose(newRandomAlphanumeric(prng, 10));
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, true, incompleteComposite));
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, false, incompleteComposite));
+        // ...and the last end-of-component byte is not 0.
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, true, incompleteComposite, (byte) 1));
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, false, incompleteComposite, (byte) 1));
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, true, incompleteComposite, (byte) -1));
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, false, incompleteComposite, (byte) -1));
+        // Test with incomplete CompositeType rows, where only the last element is not present
+        incompleteComposite = new ByteBuffer[2];
+        incompleteComposite[0] = UTF8Type.instance.decompose(newRandomAlphanumeric(prng, 10));
+        incompleteComposite[1] = TimeUUIDType.instance.decompose(TimeUUID.Generator.nextTimeUUID());
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, true, incompleteComposite));
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, false, incompleteComposite));
+        // ...and the last end-of-component byte is not 0.
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, true, incompleteComposite, (byte) 1));
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, false, incompleteComposite, (byte) 1));
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, true, incompleteComposite, (byte) -1));
+        byteBuffers.add(CompositeType.build(ByteBufferAccessor.instance, false, incompleteComposite, (byte) -1));
+
+        testValuesForType(compType, byteBuffers.toArray(new ByteBuffer[0]));
+    }
+
+    @Test
+    public void testDateType()
+    {
+        Stream<Date> dates = Stream.of(null,
+                                       new Date(Long.MIN_VALUE),
+                                       new Date(Long.MAX_VALUE),
+                                       new Date());
+        testValuesForType(DateType.instance, dates);
+
+        dates = new Random().longs(1000).mapToObj(Date::new);
+        testValuesForType(DateType.instance, dates);
+    }
+
+    @Test
+    public void testDecimalType()
+    {
+        // We won't be using testValuesForType for DecimalType (i.e. we won't also be comparing the initial and decoded
+        // ByteBuffer values). That's because the same BigDecimal value can be represented with a couple of different,
+        // even if equivalent pairs of <mantissa, scale> (e.g. 0.1 is 1 * e-1, as well as 10 * e-2, as well as...).
+        // And in practice it's easier to just convert to BigDecimals and then compare, instead of trying to manually
+        // decode and convert to canonical representations, which then to compare. For example of generating canonical
+        // decimals in the first place, see testReversedType().
+        Consumer<BigDecimal> bigDecimalConsumer = initial ->
+        {
+            ByteSource byteSource = DecimalType.instance.asComparableBytes(DecimalType.instance.decompose(initial), version);
+            BigDecimal decoded = DecimalType.instance.compose(DecimalType.instance.fromComparableBytes(ByteSource.peekable(byteSource), version));
+            if (initial == null)
+                Assert.assertNull(decoded);
+            else
+                Assert.assertEquals(0, initial.compareTo(decoded));
+        };
+        // Test some interesting predefined BigDecimal values.
+        Stream.of(null,
+                  BigDecimal.ZERO,
+                  BigDecimal.ONE,
+                  BigDecimal.ONE.add(BigDecimal.ONE),
+                  BigDecimal.TEN,
+                  BigDecimal.valueOf(0.0000000000000000000000000000000001),
+                  BigDecimal.valueOf(-0.0000000000000000000000000000000001),
+                  BigDecimal.valueOf(0.0000000000000001234567891011121314),
+                  BigDecimal.valueOf(-0.0000000000000001234567891011121314),
+                  BigDecimal.valueOf(12345678910111213.141516171819202122),
+                  BigDecimal.valueOf(-12345678910111213.141516171819202122),
+                  new BigDecimal(BigInteger.TEN, Integer.MIN_VALUE),
+                  new BigDecimal(BigInteger.TEN.negate(), Integer.MIN_VALUE),
+                  new BigDecimal(BigInteger.TEN, Integer.MAX_VALUE),
+                  new BigDecimal(BigInteger.TEN.negate(), Integer.MAX_VALUE),
+                  new BigDecimal(BigInteger.TEN.pow(1000), Integer.MIN_VALUE),
+                  new BigDecimal(BigInteger.TEN.pow(1000).negate(), Integer.MIN_VALUE),
+                  new BigDecimal(BigInteger.TEN.pow(1000), Integer.MAX_VALUE),
+                  new BigDecimal(BigInteger.TEN.pow(1000).negate(), Integer.MAX_VALUE))
+              .forEach(bigDecimalConsumer);
+        // Test BigDecimals created from random double values with predefined range modifiers.
+        double[] bounds = {
+                Double.MIN_VALUE,
+                -1_000_000_000.0,
+                -100_000.0,
+                -1.0,
+                1.0,
+                100_000.0,
+                1_000_000_000.0,
+                Double.MAX_VALUE};
+        for (double bound : bounds)
+        {
+            new Random().doubles(1000)
+                        .mapToObj(initial -> BigDecimal.valueOf(initial * bound))
+                        .forEach(bigDecimalConsumer);
+        }
+    }
+
+    @Test
+    public void testDoubleType()
+    {
+        Stream<Double> doubles = Stream.of(null,
+                                           Double.NaN,
+                                           Double.POSITIVE_INFINITY,
+                                           Double.NEGATIVE_INFINITY,
+                                           Double.MAX_VALUE,
+                                           Double.MIN_VALUE,
+                                           +0.0,
+                                           -0.0,
+                                           +1.0,
+                                           -1.0,
+                                           +12345678910.111213141516,
+                                           -12345678910.111213141516);
+        testValuesForType(DoubleType.instance, doubles);
+
+        doubles = new Random().doubles(1000).boxed();
+        testValuesForType(DoubleType.instance, doubles);
+    }
+
+    @Test
+    public void testDurationType()
+    {
+        Random prng = new Random();
+        Stream<Duration> posDurations = Stream.generate(() ->
+                                                        {
+                                                            int months = prng.nextInt(12) + 1;
+                                                            int days = prng.nextInt(28) + 1;
+                                                            long nanos = (Math.abs(prng.nextLong() % 86_400_000_000_000L)) + 1;
+                                                            return Duration.newInstance(months, days, nanos);
+                                                        })
+                                              .limit(1000);
+        testValuesForType(DurationType.instance, posDurations);
+        Stream<Duration> negDurations = Stream.generate(() ->
+                                                        {
+                                                            int months = prng.nextInt(12) + 1;
+                                                            int days = prng.nextInt(28) + 1;
+                                                            long nanos = (Math.abs(prng.nextLong() % 86_400_000_000_000L)) + 1;
+                                                            return Duration.newInstance(-months, -days, -nanos);
+                                                        })
+                                              .limit(1000);
+        testValuesForType(DurationType.instance, negDurations);
+    }
+
+    @Test
+    public void testDynamicCompositeType()
+    {
+        DynamicCompositeType dynamicCompType = DynamicCompositeType.getInstance(new HashMap<>());
+        ImmutableList<String> allTypes = ImmutableList.of("org.apache.cassandra.db.marshal.BytesType",
+                                                          "org.apache.cassandra.db.marshal.TimeUUIDType",
+                                                          "org.apache.cassandra.db.marshal.IntegerType");
+        List<ByteBuffer> allValues = new ArrayList<>();
+        List<ByteBuffer> byteBuffers = new ArrayList<>();
+        Random prng = new Random();
+        for (int i = 0; i < 10; ++i)
+        {
+            String randomString = newRandomAlphanumeric(prng, 10);
+            allValues.add(ByteBufferUtil.bytes(randomString));
+            UUID randomUuid = TimeUUID.Generator.nextTimeAsUUID();
+            allValues.add(ByteBuffer.wrap(UUIDGen.decompose(randomUuid)));
+            byte randomByte = (byte) prng.nextInt();
+            allValues.add(ByteBuffer.allocate(1).put(randomByte));
+
+            // Three-component key with aliased and non-aliased types and end-of-component byte varying (0, 1, -1).
+            byteBuffers.add(DynamicCompositeType.build(allTypes, allValues));
+            byteBuffers.add(createStringUuidVarintDynamicCompositeKey(randomString, randomUuid, randomByte, (byte) 1));
+            byteBuffers.add(createStringUuidVarintDynamicCompositeKey(randomString, randomUuid, randomByte, (byte) -1));
+
+            // Two-component key with aliased and non-aliased types and end-of-component byte varying (0, 1, -1).
+            byteBuffers.add(DynamicCompositeType.build(allTypes.subList(0, 2), allValues.subList(0, 2)));
+            byteBuffers.add(createStringUuidVarintDynamicCompositeKey(randomString, randomUuid, -1, (byte) 1));
+            byteBuffers.add(createStringUuidVarintDynamicCompositeKey(randomString, randomUuid, -1, (byte) -1));
+
+            // One-component key with aliased and non-aliased type and end-of-component byte varying (0, 1, -1).
+            byteBuffers.add(DynamicCompositeType.build(allTypes.subList(0, 1), allValues.subList(0, 1)));
+            byteBuffers.add(createStringUuidVarintDynamicCompositeKey(randomString, null, -1, (byte) 1));
+            byteBuffers.add(createStringUuidVarintDynamicCompositeKey(randomString, null, -1, (byte) -1));
+
+            allValues.clear();
+        }
+        testValuesForType(dynamicCompType, byteBuffers.toArray(new ByteBuffer[0]));
+    }
+
+    // Similar to DynamicCompositeTypeTest.createDynamicCompositeKey(string, uuid, i, true, false), but not using any
+    // aliased types, in order to do an exact comparison of the unmarshalled DynamicCompositeType payload with the
+    // input one. If aliased types are used, due to DynamicCompositeType.build(List<String>, List<ByteBuffer>)
+    // always including the full type info in the newly constructed payload, an exact comparison won't work.
+    private static ByteBuffer createStringUuidVarintDynamicCompositeKey(String string, UUID uuid, int i, byte lastEocByte)
+    {
+        // 1. Calculate how many bytes do we need for a key of this DynamicCompositeType
+        String bytesType = "org.apache.cassandra.db.marshal.BytesType";
+        String timeUuidType = "org.apache.cassandra.db.marshal.TimeUUIDType";
+        String varintType = "org.apache.cassandra.db.marshal.IntegerType";
+        ByteBuffer bytes = ByteBufferUtil.bytes(string);
+        int totalSize = 0;
+        // Take into account the string component data (BytesType is aliased)
+        totalSize += 2 + bytesType.length() + 2 + bytes.remaining() + 1;
+        if (uuid != null)
+        {
+            // Take into account the UUID component data (TimeUUIDType is aliased)
+            totalSize += 2 + timeUuidType.length() + 2 + 16 + 1;
+            if (i != -1)
+            {
+                // Take into account the varint component data (IntegerType is _not_ aliased).
+                // Notice that we account for a single byte of varint data, so we'll downcast the int payload
+                // to byte and use only that as the actual varint payload.
+                totalSize += 2 + varintType.length() + 2 + 1 + 1;
+            }
+        }
+
+        // 2. Allocate a buffer with that many bytes
+        ByteBuffer bb = ByteBuffer.allocate(totalSize);
+
+        // 3. Write the key data for each component in the allocated buffer
+        bb.putShort((short) bytesType.length());
+        bb.put(ByteBufferUtil.bytes(bytesType));
+        bb.putShort((short) bytes.remaining());
+        bb.put(bytes);
+        // Make the end-of-component byte 1 if requested and the time-UUID component is null.
+        bb.put(uuid == null ? lastEocByte : (byte) 0);
+        if (uuid != null)
+        {
+            bb.putShort((short) timeUuidType.length());
+            bb.put(ByteBufferUtil.bytes(timeUuidType));
+            bb.putShort((short) 16);
+            bb.put(UUIDGen.decompose(uuid));
+            // Set the end-of-component byte if requested and the varint component is null.
+            bb.put(i == -1 ? lastEocByte : (byte) 0);
+            if (i != -1)
+            {
+                bb.putShort((short) varintType.length());
+                bb.put(ByteBufferUtil.bytes(varintType));
+                bb.putShort((short) 1);
+                bb.put((byte) i);
+                bb.put(lastEocByte);
+            }
+        }
+        bb.rewind();
+        return bb;
+    }
+
+    @Test
+    public void testFloatType()
+    {
+        Stream<Float> floats = Stream.of(null,
+                                         Float.NaN,
+                                         Float.POSITIVE_INFINITY,
+                                         Float.NEGATIVE_INFINITY,
+                                         Float.MAX_VALUE,
+                                         Float.MIN_VALUE,
+                                         +0.0F,
+                                         -0.0F,
+                                         +1.0F,
+                                         -1.0F,
+                                         +123456.7891011F,
+                                         -123456.7891011F);
+        testValuesForType(FloatType.instance, floats);
+
+        floats = new Random().ints(1000).mapToObj(Float::intBitsToFloat);
+        testValuesForType(FloatType.instance, floats);
+    }
+
+    @Test
+    public void testInetAddressType() throws UnknownHostException
+    {
+        Stream<InetAddress> inetAddresses = Stream.of(null,
+                                                      InetAddress.getLocalHost(),
+                                                      InetAddress.getLoopbackAddress(),
+                                                      InetAddress.getByName("0.0.0.0"),
+                                                      InetAddress.getByName("10.0.0.1"),
+                                                      InetAddress.getByName("172.16.1.1"),
+                                                      InetAddress.getByName("192.168.2.2"),
+                                                      InetAddress.getByName("224.3.3.3"),
+                                                      InetAddress.getByName("255.255.255.255"),
+                                                      InetAddress.getByName("0000:0000:0000:0000:0000:0000:0000:0000"),
+                                                      InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff"),
+                                                      InetAddress.getByName("fe80:1:23:456:7890:1:23:456"));
+        testValuesForType(InetAddressType.instance, inetAddresses);
+
+        Random prng = new Random();
+        byte[] ipv4Bytes = new byte[4];
+        byte[] ipv6Bytes = new byte[16];
+        InetAddress[] addresses = new InetAddress[2000];
+        for (int i = 0; i < addresses.length / 2; ++i)
+        {
+            prng.nextBytes(ipv4Bytes);
+            addresses[2 * i] = InetAddress.getByAddress(ipv4Bytes);
+            addresses[2 * i + 1] = InetAddress.getByAddress(ipv6Bytes);
+        }
+        testValuesForType(InetAddressType.instance, addresses);
+
+    }
+
+    @Test
+    public void testInt32Type()
+    {
+        Stream<Integer> ints = Stream.of(null,
+                                         Integer.MIN_VALUE,
+                                         Integer.MIN_VALUE + 1,
+                                         -256, -255, -128, -127, -1,
+                                         0,
+                                         1, 127, 128, 255, 256,
+                                         Integer.MAX_VALUE - 1,
+                                         Integer.MAX_VALUE);
+        testValuesForType(Int32Type.instance, ints);
+
+        ints = new Random().ints(1000).boxed();
+        testValuesForType(Int32Type.instance, ints);
+    }
+
+    @Test
+    public void testIntegerType()
+    {
+        Stream<BigInteger> varints = IntStream.range(-1000000, 1000000).mapToObj(BigInteger::valueOf);
+        testValuesForType(IntegerType.instance, varints);
+
+        varints = Stream.of(null,
+                            BigInteger.valueOf(12345678910111213L),
+                            BigInteger.valueOf(12345678910111213L).negate(),
+                            BigInteger.valueOf(Long.MAX_VALUE),
+                            BigInteger.valueOf(Long.MAX_VALUE).negate(),
+                            BigInteger.valueOf(Long.MAX_VALUE - 1).multiply(BigInteger.valueOf(Long.MAX_VALUE - 1)),
+                            BigInteger.valueOf(Long.MAX_VALUE - 1).multiply(BigInteger.valueOf(Long.MAX_VALUE - 1)).negate());
+        testValuesForType(IntegerType.instance, varints);
+
+        List<BigInteger> varintList = new ArrayList<>();
+        for (int i = 0; i < 10000; ++i)
+        {
+            BigInteger initial = BigInteger.ONE.shiftLeft(i);
+            varintList.add(initial);
+            BigInteger plusOne = initial.add(BigInteger.ONE);
+            varintList.add(plusOne);
+            varintList.add(plusOne.negate());
+            BigInteger minusOne = initial.subtract(BigInteger.ONE);
+            varintList.add(minusOne);
+            varintList.add(minusOne.negate());
+        }
+        testValuesForType(IntegerType.instance, varintList.toArray(new BigInteger[0]));
+    }
+
+    @Test
+    public void testUuidTypes()
+    {
+        Random prng = new Random();
+        UUID[] testUuids = new UUID[3001];
+        for (int i = 0; i < testUuids.length / 3; ++i)
+        {
+            testUuids[3 * i] = UUID.randomUUID();
+            testUuids[3 * i + 1] = TimeUUID.Generator.nextTimeAsUUID();
+            testUuids[3 * i + 2] = TimeUUID.atUnixMicrosWithLsbAsUUID(prng.nextLong(), prng.nextLong());
+        }
+        testUuids[testUuids.length - 1] = null;
+        testValuesForType(UUIDType.instance, testUuids);
+        testValuesForType(LexicalUUIDType.instance, testUuids);
+        testValuesForType(TimeUUIDType.instance, Arrays.stream(testUuids)
+                                                       .filter(u -> u == null || u.version() == 1)
+                                                       .map(u -> u != null ? TimeUUID.fromUuid(u) : null));
+    }
+
+    private static <E, C extends Collection<E>> List<C> newRandomElementCollections(Supplier<? extends C> collectionProducer,
+                                                                                    Supplier<? extends E> elementProducer,
+                                                                                    int numCollections,
+                                                                                    int numElementsInCollection)
+    {
+        List<C> result = new ArrayList<>();
+        for (int i = 0; i < numCollections; ++i)
+        {
+            C coll = collectionProducer.get();
+            for (int j = 0; j < numElementsInCollection; ++j)
+            {
+                coll.add(elementProducer.get());
+            }
+            result.add(coll);
+        }
+        return result;
+    }
+
+    @Test
+    public void testListType()
+    {
+        // Test lists with element components not having known/computable length (e.g. strings).
+        Random prng = new Random();
+        List<List<String>> stringLists = newRandomElementCollections(ArrayList::new,
+                                                                     () -> newRandomAlphanumeric(prng, 10),
+                                                                     100,
+                                                                     100);
+        testValuesForType(ListType.getInstance(UTF8Type.instance, false), stringLists);
+        testValuesForType(ListType.getInstance(UTF8Type.instance, true), stringLists);
+        // Test lists with element components with known/computable length (e.g. 128-bit UUIDs).
+        List<List<UUID>> uuidLists = newRandomElementCollections(ArrayList::new,
+                                                                 UUID::randomUUID,
+                                                                 100,
+                                                                 100);
+        testValuesForType(ListType.getInstance(UUIDType.instance, false), uuidLists);
+        testValuesForType(ListType.getInstance(UUIDType.instance, true), uuidLists);
+    }
+
+    @Test
+    public void testLongType()
+    {
+        Stream<Long> longs = Stream.of(null,
+                                       Long.MIN_VALUE,
+                                       Long.MIN_VALUE + 1,
+                                       (long) Integer.MIN_VALUE - 1,
+                                       -256L, -255L, -128L, -127L, -1L,
+                                       0L,
+                                       1L, 127L, 128L, 255L, 256L,
+                                       (long) Integer.MAX_VALUE + 1,
+                                       Long.MAX_VALUE - 1,
+                                       Long.MAX_VALUE);
+        testValuesForType(LongType.instance, longs);
+
+        longs = new Random().longs(1000).boxed();
+        testValuesForType(LongType.instance, longs);
+    }
+
+    private static <K, V> List<Map<K, V>> newRandomEntryMaps(Supplier<? extends K> keyProducer,
+                                                             Supplier<? extends V> valueProducer,
+                                                             int numMaps,
+                                                             int numEntries)
+    {
+        List<Map<K, V>> result = new ArrayList<>();
+        for (int i = 0; i < numMaps; ++i)
+        {
+            Map<K, V> map = new HashMap<>();
+            for (int j = 0; j < numEntries; ++j)
+            {
+                K key = keyProducer.get();
+                V value = valueProducer.get();
+                map.put(key, value);
+            }
+            result.add(map);
+        }
+        return result;
+    }
+
+    @Test
+    public void testMapType()
+    {
+        Random prng = new Random();
+        List<Map<String, UUID>> stringToUuidMaps = newRandomEntryMaps(() -> newRandomAlphanumeric(prng, 10),
+                                                                      UUID::randomUUID,
+                                                                      100,
+                                                                      100);
+        testValuesForType(MapType.getInstance(UTF8Type.instance, UUIDType.instance, false), stringToUuidMaps);
+        testValuesForType(MapType.getInstance(UTF8Type.instance, UUIDType.instance, true), stringToUuidMaps);
+
+        List<Map<UUID, String>> uuidToStringMaps = newRandomEntryMaps(UUID::randomUUID,
+                                                                      () -> newRandomAlphanumeric(prng, 10),
+                                                                      100,
+                                                                      100);
+        testValuesForType(MapType.getInstance(UUIDType.instance, UTF8Type.instance, false), uuidToStringMaps);
+        testValuesForType(MapType.getInstance(UUIDType.instance, UTF8Type.instance, true), uuidToStringMaps);
+    }
+
+    @Test
+    public void testPartitionerDefinedOrder()
+    {
+        Random prng = new Random();
+        List<ByteBuffer> byteBuffers = new ArrayList<>();
+        byteBuffers.add(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        for (int i = 0; i < 1000; ++i)
+        {
+            String randomString = newRandomAlphanumeric(prng, 10);
+            byteBuffers.add(UTF8Type.instance.decompose(randomString));
+            int randomInt = prng.nextInt();
+            byteBuffers.add(Int32Type.instance.decompose(randomInt));
+            double randomDouble = prng.nextDouble();
+            byteBuffers.add(DoubleType.instance.decompose(randomDouble));
+            BigInteger randomishVarint = BigInteger.probablePrime(100, prng);
+            byteBuffers.add(IntegerType.instance.decompose(randomishVarint));
+            BigDecimal randomishDecimal = BigDecimal.valueOf(prng.nextLong(), prng.nextInt(100) - 50);
+            byteBuffers.add(DecimalType.instance.decompose(randomishDecimal));
+        }
+
+        byte[] bytes = new byte[100];
+        prng.nextBytes(bytes);
+        ByteBuffer exhausted = ByteBuffer.wrap(bytes);
+        ByteBufferUtil.readBytes(exhausted, 100);
+
+        List<IPartitioner> partitioners = Arrays.asList(
+                Murmur3Partitioner.instance,
+                RandomPartitioner.instance,
+                LengthPartitioner.instance
+                // NOTE LocalPartitioner, OrderPreservingPartitioner, and ByteOrderedPartitioner don't need a dedicated
+                // PartitionerDefinedOrder.
+                //   1) LocalPartitioner uses its inner AbstractType
+                //   2) OrderPreservingPartitioner uses UTF8Type
+                //   3) ByteOrderedPartitioner uses BytesType
+        );
+        for (IPartitioner partitioner : partitioners)
+        {
+            AbstractType<?> partitionOrdering = partitioner.partitionOrdering();
+            Assert.assertTrue(partitionOrdering instanceof PartitionerDefinedOrder);
+            for (ByteBuffer input : byteBuffers)
+            {
+                ByteSource byteSource = partitionOrdering.asComparableBytes(input, version);
+                ByteBuffer output = partitionOrdering.fromComparableBytes(ByteSource.peekable(byteSource), version);
+                Assert.assertEquals("For partitioner " + partitioner.getClass().getSimpleName(),
+                                    ByteBufferUtil.bytesToHex(input),
+                                    ByteBufferUtil.bytesToHex(output));
+            }
+            ByteSource byteSource = partitionOrdering.asComparableBytes(exhausted, version);
+            ByteBuffer output = partitionOrdering.fromComparableBytes(ByteSource.peekable(byteSource), version);
+            Assert.assertEquals(ByteBufferUtil.EMPTY_BYTE_BUFFER, output);
+        }
+    }
+
+    @Test
+    public void testReversedType()
+    {
+        // Test how ReversedType handles null ByteSource.Peekable - here the choice of base type is important, as
+        // the base type should also be able to handle null ByteSource.Peekable.
+        ReversedType<BigInteger> reversedVarintType = ReversedType.getInstance(IntegerType.instance);
+        ByteBuffer decodedNull = reversedVarintType.fromComparableBytes(null, ByteComparable.Version.OSS42);
+        Assert.assertEquals(ByteBufferUtil.EMPTY_BYTE_BUFFER, decodedNull);
+
+        // Test how ReversedType handles random data with some common and important base types.
+        Map<AbstractType<?>, BiFunction<Random, Integer, ByteBuffer>> bufferGeneratorByType = new HashMap<>();
+        bufferGeneratorByType.put(UTF8Type.instance, (prng, length) -> UTF8Type.instance.decompose(newRandomAlphanumeric(prng, length)));
+        bufferGeneratorByType.put(BytesType.instance, (prng, length) ->
+        {
+            byte[] randomBytes = new byte[length];
+            prng.nextBytes(randomBytes);
+            return ByteBuffer.wrap(randomBytes);
+        });
+        bufferGeneratorByType.put(IntegerType.instance, (prng, length) ->
+        {
+            BigInteger randomVarint = BigInteger.valueOf(prng.nextLong());
+            for (int i = 1; i < length / 8; ++i)
+                randomVarint = randomVarint.multiply(BigInteger.valueOf(prng.nextLong()));
+            return IntegerType.instance.decompose(randomVarint);
+        });
+        bufferGeneratorByType.put(DecimalType.instance, (prng, length) ->
+        {
+            BigInteger randomMantissa = BigInteger.valueOf(prng.nextLong());
+            for (int i = 1; i < length / 8; ++i)
+                randomMantissa = randomMantissa.multiply(BigInteger.valueOf(prng.nextLong()));
+            // Remove all trailing zeros from the mantissa and use an even scale, in order to have a "canonically
+            // represented" (in the context of DecimalType's encoding) decimal, i.e. one which wouldn't be re-scaled to
+            // conform with the "compacted mantissa between 0 and 1, scale as a power of 100" rule.
+            while (randomMantissa.remainder(BigInteger.TEN).equals(BigInteger.ZERO))
+                randomMantissa = randomMantissa.divide(BigInteger.TEN);
+            int randomScale = prng.nextInt() & -2;
+            BigDecimal randomDecimal = new BigDecimal(randomMantissa, randomScale);
+            return DecimalType.instance.decompose(randomDecimal);
+        });
+        Random prng = new Random();
+        for (Map.Entry<AbstractType<?>, BiFunction<Random, Integer, ByteBuffer>> entry : bufferGeneratorByType.entrySet())
+        {
+            ReversedType<?> reversedType = ReversedType.getInstance(entry.getKey());
+            for (int length = 32; length <= 512; length *= 4)
+            {
+                for (int i = 0; i < 100; ++i)
+                {
+                    ByteBuffer initial = entry.getValue().apply(prng, length);
+                    ByteSource.Peekable reversedPeekable = ByteSource.peekable(reversedType.asComparableBytes(initial, ByteComparable.Version.OSS42));
+                    ByteBuffer decoded = reversedType.fromComparableBytes(reversedPeekable, ByteComparable.Version.OSS42);
+                    Assert.assertEquals(initial, decoded);
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testSetType()
+    {
+        // Test sets with element components not having known/computable length (e.g. strings).
+        Random prng = new Random();
+        List<Set<String>> stringSets = newRandomElementCollections(HashSet::new,
+                                                                   () -> newRandomAlphanumeric(prng, 10),
+                                                                   100,
+                                                                   100);
+        testValuesForType(SetType.getInstance(UTF8Type.instance, false), stringSets);
+        testValuesForType(SetType.getInstance(UTF8Type.instance, true), stringSets);
+        // Test sets with element components with known/computable length (e.g. 128-bit UUIDs).
+        List<Set<UUID>> uuidSets = newRandomElementCollections(HashSet::new,
+                                                               UUID::randomUUID,
+                                                               100,
+                                                               100);
+        testValuesForType(SetType.getInstance(UUIDType.instance, false), uuidSets);
+        testValuesForType(SetType.getInstance(UUIDType.instance, true), uuidSets);
+    }
+
+    @Test
+    public void testShortType()
+    {
+        testValuesForType(ShortType.instance, new Short[] { null });
+
+        Stream<Short> allShorts = IntStream.range(Short.MIN_VALUE, Short.MAX_VALUE + 1)
+                                           .mapToObj(value -> (short) value);
+        testValuesForType(ShortType.instance, allShorts);
+    }
+
+    @Test
+    public void testSimpleDateType()
+    {
+        testValuesForType(SimpleDateType.instance, new Integer[] { null });
+
+        testValuesForType(SimpleDateType.instance, new Random().ints(1000).boxed());
+
+        // Test by manually creating and manually interpreting simple dates from random millis.
+        new Random().ints(1000).forEach(initialMillis ->
+                                         {
+                                             initialMillis = Math.abs(initialMillis);
+                                             Integer initialDays = SimpleDateSerializer.timeInMillisToDay(initialMillis);
+                                             ByteBuffer simpleDateBuffer = SimpleDateType.instance.fromTimeInMillis(initialMillis);
+                                             ByteSource byteSource = SimpleDateType.instance.asComparableBytes(simpleDateBuffer, version);
+                                             Integer decodedDays = SimpleDateType.instance.compose(SimpleDateType.instance.fromComparableBytes(ByteSource.peekable(byteSource), version));
+                                             Assert.assertEquals(initialDays, decodedDays);
+                                         });
+
+        // Test by manually creating and manually interpreting simple dates from strings.
+        String[] simpleDateStrings = new String[]
+                                             {
+                                                     "1970-01-01",
+                                                     "1970-01-02",
+                                                     "1969-12-31",
+                                                     "-0001-01-02",
+                                                     "-5877521-01-02",
+                                                     "2014-01-01",
+                                                     "+5881580-01-10",
+                                                     "1920-12-01",
+                                                     "1582-10-19"
+                                             };
+        for (String simpleDate : simpleDateStrings)
+        {
+            ByteBuffer simpleDataBuffer = SimpleDateType.instance.fromString(simpleDate);
+            ByteSource byteSource = SimpleDateType.instance.asComparableBytes(simpleDataBuffer, version);
+            Integer decodedDays = SimpleDateType.instance.compose(SimpleDateType.instance.fromComparableBytes(ByteSource.peekable(byteSource), version));
+            String decodedDate = SimpleDateSerializer.instance.toString(decodedDays);
+            Assert.assertEquals(simpleDate, decodedDate);
+        }
+    }
+
+    @Test
+    public void testTimestampType()
+    {
+        Date[] dates = new Date[]
+                               {
+                                       null,
+                                       new Date(),
+                                       new Date(0L),
+                                       new Date(-1L),
+                                       new Date(Long.MAX_VALUE),
+                                       new Date(Long.MIN_VALUE)
+                               };
+        testValuesForType(TimestampType.instance, dates);
+        testValuesForType(TimestampType.instance, new Random().longs(1000).mapToObj(Date::new));
+    }
+
+    @Test
+    public void testTimeType()
+    {
+        testValuesForType(TimeType.instance, new Long[] { null });
+
+        testValuesForType(TimeType.instance, new Random().longs(1000).boxed());
+    }
+
+    @Test
+    public void testTupleType()
+    {
+        TupleType tt = new TupleType(Arrays.asList(UTF8Type.instance,
+                                                   DecimalType.instance,
+                                                   IntegerType.instance,
+                                                   BytesType.instance));
+        Random prng = new Random();
+        List<ByteBuffer> tuplesData = new ArrayList<>();
+        String[] utf8Values = new String[]
+                                      {
+                                              "a",
+                                              "©",
+                                              newRandomAlphanumeric(prng, 10),
+                                              newRandomAlphanumeric(prng, 100)
+                                      };
+        BigDecimal[] decimalValues = new BigDecimal[]
+                                             {
+                                                     null,
+                                                     BigDecimal.ZERO,
+                                                     BigDecimal.ONE,
+                                                     BigDecimal.valueOf(1234567891011121314L, 50),
+                                                     BigDecimal.valueOf(1234567891011121314L, 50).negate()
+                                             };
+        BigInteger[] varintValues = new BigInteger[]
+                                            {
+                                                    null,
+                                                    BigInteger.ZERO,
+                                                    BigInteger.TEN.pow(1000),
+                                                    BigInteger.TEN.pow(1000).negate()
+                                            };
+        byte[] oneByte = new byte[1];
+        byte[] tenBytes = new byte[10];
+        byte[] hundredBytes = new byte[100];
+        byte[] thousandBytes = new byte[1000];
+        prng.nextBytes(oneByte);
+        prng.nextBytes(tenBytes);
+        prng.nextBytes(hundredBytes);
+        prng.nextBytes(thousandBytes);
+        byte[][] bytesValues = new byte[][]
+                                       {
+                                               new byte[0],
+                                               oneByte,
+                                               tenBytes,
+                                               hundredBytes,
+                                               thousandBytes
+                                       };
+        for (String utf8 : utf8Values)
+        {
+            for (BigDecimal decimal : decimalValues)
+            {
+                for (BigInteger varint : varintValues)
+                {
+                    for (byte[] bytes : bytesValues)
+                    {
+                        ByteBuffer tupleData = TupleType.buildValue(UTF8Type.instance.decompose(utf8),
+                                                                    decimal != null ? DecimalType.instance.decompose(decimal) : null,
+                                                                    varint != null ? IntegerType.instance.decompose(varint) : null,
+                                                                    // We could also use the wrapped bytes directly
+                                                                    BytesType.instance.decompose(ByteBuffer.wrap(bytes)));
+                        tuplesData.add(tupleData);
+                    }
+                }
+            }
+        }
+        testValuesForType(tt, tuplesData.toArray(new ByteBuffer[0]));
+    }
+
+    @Test
+    public void testUtf8Type()
+    {
+        Random prng = new Random();
+        testValuesForType(UTF8Type.instance, Stream.generate(() -> newRandomAlphanumeric(prng, 100)).limit(1000));
+    }
+
+    @Test
+    public void testTypeWithByteOrderedComparison()
+    {
+        Random prng = new Random();
+        byte[] singleByte = new byte[] { (byte) prng.nextInt() };
+        byte[] tenBytes = new byte[10];
+        prng.nextBytes(tenBytes);
+        byte[] hundredBytes = new byte[100];
+        prng.nextBytes(hundredBytes);
+        byte[] thousandBytes = new byte[1000];
+        prng.nextBytes(thousandBytes);
+        // No null here, as the default asComparableBytes(ByteBuffer, Version) implementation (and more specifically
+        // the ByteSource.of(ByteBuffer, Version) encoding) would throw then.
+        testValuesForType(ByteOrderedType.instance, Stream.of(ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                              ByteBuffer.wrap(singleByte),
+                                                              ByteBuffer.wrap(tenBytes),
+                                                              ByteBuffer.wrap(hundredBytes),
+                                                              ByteBuffer.wrap(thousandBytes)));
+    }
+
+    private static class ByteOrderedType extends AbstractType<ByteBuffer>
+    {
+        public static final ByteOrderedType instance = new ByteOrderedType();
+
+        private ByteOrderedType()
+        {
+            super(ComparisonType.BYTE_ORDER);
+        }
+
+        @Override
+        public ByteBuffer fromString(String source) throws MarshalException
+        {
+            return null;
+        }
+
+        @Override
+        public Term fromJSONObject(Object parsed) throws MarshalException
+        {
+            return null;
+        }
+
+        @Override
+        public TypeSerializer<ByteBuffer> getSerializer()
+        {
+            return ByteOrderedSerializer.instance;
+        }
+
+        static class ByteOrderedSerializer extends TypeSerializer<ByteBuffer>
+        {
+
+            static final ByteOrderedSerializer instance = new ByteOrderedSerializer();
+
+            @Override
+            public ByteBuffer serialize(ByteBuffer value)
+            {
+                return value != null ? value.duplicate() : null;
+            }
+
+            @Override
+            public <V> ByteBuffer deserialize(V bytes, ValueAccessor<V> accessor)
+            {
+                return accessor.toBuffer(bytes);
+            }
+
+            @Override
+            public <V> void validate(V bytes, ValueAccessor<V> accessor) throws MarshalException
+            {
+
+            }
+
+            @Override
+            public String toString(ByteBuffer value)
+            {
+                return ByteBufferUtil.bytesToHex(value);
+            }
+
+            @Override
+            public Class<ByteBuffer> getType()
+            {
+                return ByteBuffer.class;
+            }
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/utils/bytecomparable/ByteSourceComparisonTest.java b/test/unit/org/apache/cassandra/utils/bytecomparable/ByteSourceComparisonTest.java
new file mode 100644
index 0000000000..f5cf2b639a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/bytecomparable/ByteSourceComparisonTest.java
@@ -0,0 +1,1178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.utils.bytecomparable;
+
+import java.math.BigDecimal;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Ordering;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.MurmurHash;
+import org.apache.cassandra.utils.TimeUUID;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests forward conversion to ByteSource/ByteComparable and that the result compares correctly.
+ */
+public class ByteSourceComparisonTest extends ByteSourceTestBase
+{
+    private final static Logger logger = LoggerFactory.getLogger(ByteSourceComparisonTest.class);
+
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Test
+    public void testStringsAscii()
+    {
+        testType(AsciiType.instance, testStrings);
+    }
+
+    @Test
+    public void testStringsUTF8()
+    {
+        testType(UTF8Type.instance, testStrings);
+        testDirect(x -> ByteSource.of(x, Version.OSS42), Ordering.<String>natural()::compare, testStrings);
+    }
+
+    @Test
+    public void testBooleans()
+    {
+        testType(BooleanType.instance, testBools);
+    }
+
+    @Test
+    public void testInts()
+    {
+        testType(Int32Type.instance, testInts);
+        testDirect(x -> ByteSource.of(x), Integer::compare, testInts);
+    }
+
+    @Test
+    public void randomTestInts()
+    {
+        Random rand = new Random();
+        for (int i=0; i<10000; ++i)
+        {
+            int i1 = rand.nextInt();
+            int i2 = rand.nextInt();
+            assertComparesSame(Int32Type.instance, i1, i2);
+        }
+
+    }
+
+    @Test
+    public void testLongs()
+    {
+        testType(LongType.instance, testLongs);
+        testDirect(x -> ByteSource.of(x), Long::compare, testLongs);
+    }
+
+    @Test
+    public void testShorts()
+    {
+        testType(ShortType.instance, testShorts);
+    }
+
+    @Test
+    public void testBytes()
+    {
+        testType(ByteType.instance, testBytes);
+    }
+
+    @Test
+    public void testDoubles()
+    {
+        testType(DoubleType.instance, testDoubles);
+    }
+
+    @Test
+    public void testFloats()
+    {
+        testType(FloatType.instance, testFloats);
+    }
+
+    @Test
+    public void testBigInts()
+    {
+        testType(IntegerType.instance, testBigInts);
+    }
+
+    @Test
+    public void testBigDecimals()
+    {
+        testType(DecimalType.instance, testBigDecimals);
+    }
+
+    @Test
+    public void testBigDecimalInCombination()
+    {
+        BigDecimal b1 = new BigDecimal("123456.78901201");
+        BigDecimal b2 = new BigDecimal("123456.789012");
+        Boolean b = false;
+
+        assertClusteringPairComparesSame(DecimalType.instance, BooleanType.instance, b1, b, b2, b);
+        assertClusteringPairComparesSame(BooleanType.instance, DecimalType.instance, b, b1, b, b2);
+
+        b1 = b1.negate();
+        b2 = b2.negate();
+
+        assertClusteringPairComparesSame(DecimalType.instance, BooleanType.instance, b1, b, b2, b);
+        assertClusteringPairComparesSame(BooleanType.instance, DecimalType.instance, b, b1, b, b2);
+
+        b1 = new BigDecimal("-123456.78901289");
+        b2 = new BigDecimal("-123456.789012");
+
+        assertClusteringPairComparesSame(DecimalType.instance, BooleanType.instance, b1, b, b2, b);
+        assertClusteringPairComparesSame(BooleanType.instance, DecimalType.instance, b, b1, b, b2);
+
+        b1 = new BigDecimal("1");
+        b2 = new BigDecimal("1.1");
+
+        assertClusteringPairComparesSame(DecimalType.instance, BooleanType.instance, b1, b, b2, b);
+        assertClusteringPairComparesSame(BooleanType.instance, DecimalType.instance, b, b1, b, b2);
+
+        b1 = b1.negate();
+        b2 = b2.negate();
+
+        assertClusteringPairComparesSame(DecimalType.instance, BooleanType.instance, b1, b, b2, b);
+        assertClusteringPairComparesSame(BooleanType.instance, DecimalType.instance, b, b1, b, b2);
+    }
+
+    @Test
+    public void testUUIDs()
+    {
+        testType(UUIDType.instance, testUUIDs);
+    }
+
+    @Test
+    public void testTimeUUIDs()
+    {
+        testType(TimeUUIDType.instance, Arrays.stream(testUUIDs)
+                                              .filter(x -> x == null || x.version() == 1)
+                                              .map(x -> x != null ? TimeUUID.fromUuid(x) : null)
+                                              .toArray());
+    }
+
+    @Test
+    public void testLexicalUUIDs()
+    {
+        testType(LexicalUUIDType.instance, testUUIDs);
+    }
+
+    @Test
+    public void testSimpleDate()
+    {
+        testType(SimpleDateType.instance, Arrays.stream(testInts).filter(x -> x != null).toArray());
+    }
+
+    @Test
+    public void testTimeType()
+    {
+        testType(TimeType.instance, Arrays.stream(testLongs).filter(x -> x != null && x >= 0 && x <= 24L * 60 * 60 * 1000 * 1000 * 1000).toArray());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testDateType()
+    {
+        testType(DateType.instance, testDates);
+    }
+
+    @Test
+    public void testTimestampType()
+    {
+        testType(TimestampType.instance, testDates);
+    }
+
+    @Test
+    public void testBytesType()
+    {
+        List<ByteBuffer> values = new ArrayList<>();
+        for (int i = 0; i < testValues.length; ++i)
+            for (Object o : testValues[i])
+                values.add(testTypes[i].decompose(o));
+
+        testType(BytesType.instance, values.toArray());
+    }
+
+    @Test
+    public void testInetAddressType() throws UnknownHostException
+    {
+        testType(InetAddressType.instance, testInets);
+    }
+
+    @Test
+    public void testEmptyType()
+    {
+        testType(EmptyType.instance, new Void[] { null });
+    }
+
+    @Test
+    public void testPatitionerDefinedOrder()
+    {
+        List<ByteBuffer> values = new ArrayList<>();
+        for (int i = 0; i < testValues.length; ++i)
+            for (Object o : testValues[i])
+                values.add(testTypes[i].decompose(o));
+
+        testBuffers(new PartitionerDefinedOrder(Murmur3Partitioner.instance), values);
+        testBuffers(new PartitionerDefinedOrder(RandomPartitioner.instance), values);
+        testBuffers(new PartitionerDefinedOrder(ByteOrderedPartitioner.instance), values);
+    }
+
+    @Test
+    public void testPatitionerOrder()
+    {
+        List<ByteBuffer> values = new ArrayList<>();
+        for (int i = 0; i < testValues.length; ++i)
+            for (Object o : testValues[i])
+                values.add(testTypes[i].decompose(o));
+
+        testDecoratedKeys(Murmur3Partitioner.instance, values);
+        testDecoratedKeys(RandomPartitioner.instance, values);
+        testDecoratedKeys(ByteOrderedPartitioner.instance, values);
+    }
+
+    @Test
+    public void testLocalPatitionerOrder()
+    {
+        for (int i = 0; i < testValues.length; ++i)
+        {
+            final AbstractType testType = testTypes[i];
+            testDecoratedKeys(new LocalPartitioner(testType), Lists.transform(Arrays.asList(testValues[i]),
+                                                                                            v -> testType.decompose(v)));
+        }
+    }
+
+    interface PairTester
+    {
+        void test(AbstractType t1, AbstractType t2, Object o1, Object o2, Object o3, Object o4);
+    }
+
+    void testCombinationSampling(Random rand, PairTester tester)
+    {
+        for (int i=0;i<testTypes.length;++i)
+            for (int j=0;j<testTypes.length;++j)
+            {
+                Object[] tv1 = new Object[3];
+                Object[] tv2 = new Object[3];
+                for (int t=0; t<tv1.length; ++t)
+                {
+                    tv1[t] = testValues[i][rand.nextInt(testValues[i].length)];
+                    tv2[t] = testValues[j][rand.nextInt(testValues[j].length)];
+                }
+
+                for (Object o1 : tv1)
+                    for (Object o2 : tv2)
+                        for (Object o3 : tv1)
+                            for (Object o4 : tv2)
+
+                {
+                    tester.test(testTypes[i], testTypes[j], o1, o2, o3, o4);
+                }
+            }
+    }
+
+    @Test
+    public void testCombinations()
+    {
+        Random rand = new Random(0);
+        testCombinationSampling(rand, this::assertClusteringPairComparesSame);
+    }
+
+    @Test
+    public void testNullsInClustering()
+    {
+        ByteBuffer[][] inputs = new ByteBuffer[][]
+                                {
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, "a"),
+                                                  decomposeAndRandomPad(Int32Type.instance, 0)},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, "a"),
+                                                  decomposeAndRandomPad(Int32Type.instance, null)},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, "a"),
+                                                  null},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, ""),
+                                                  decomposeAndRandomPad(Int32Type.instance, 0)},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, ""),
+                                                  decomposeAndRandomPad(Int32Type.instance, null)},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, ""),
+                                                  null},
+                                new ByteBuffer[] {null,
+                                                  decomposeAndRandomPad(Int32Type.instance, 0)},
+                                new ByteBuffer[] {null,
+                                                  decomposeAndRandomPad(Int32Type.instance, null)},
+                                new ByteBuffer[] {null,
+                                                  null}
+                                };
+        for (ByteBuffer[] input1 : inputs)
+            for (ByteBuffer[] input2 : inputs)
+            {
+                assertClusteringPairComparesSame(UTF8Type.instance, Int32Type.instance,
+                                                 input1[0], input1[1], input2[0], input2[1],
+                                                 (t, v) -> (ByteBuffer) v,
+                                                 input1[0] != null && input1[1] != null && input2[0] != null && input2[1] != null);
+            }
+    }
+
+    @Test
+    public void testNullsInClusteringLegacy()
+    {
+        // verify the legacy encoding treats null clustering the same as null value
+        ClusteringPrefix<ByteBuffer> aNull = makeBound(ClusteringPrefix.Kind.CLUSTERING,
+                                                       decomposeAndRandomPad(UTF8Type.instance, "a"),
+                                                       decomposeAndRandomPad(Int32Type.instance, null));
+        ClusteringPrefix<ByteBuffer> aEmpty = makeBound(ClusteringPrefix.Kind.CLUSTERING,
+                                                        decomposeAndRandomPad(UTF8Type.instance, "a"),
+                                                        null);
+        ClusteringComparator comp = new ClusteringComparator(UTF8Type.instance, Int32Type.instance);
+        assertEquals(0, ByteComparable.compare(comp.asByteComparable(aNull), comp.asByteComparable(aEmpty), Version.LEGACY));
+        ClusteringComparator compReversed = new ClusteringComparator(UTF8Type.instance, ReversedType.getInstance(Int32Type.instance));
+        assertEquals(0, ByteComparable.compare(compReversed.asByteComparable(aNull), compReversed.asByteComparable(aEmpty), Version.LEGACY));
+    }
+
+    @Test
+    public void testEmptyClustering()
+    {
+        assertEmptyComparedToStatic(1, ClusteringPrefix.Kind.CLUSTERING, Version.OSS42);
+        assertEmptyComparedToStatic(0, ClusteringPrefix.Kind.STATIC_CLUSTERING, Version.OSS42);
+        assertEmptyComparedToStatic(1, ClusteringPrefix.Kind.INCL_START_BOUND, Version.OSS42);
+        assertEmptyComparedToStatic(1, ClusteringPrefix.Kind.INCL_END_BOUND, Version.OSS42);
+
+        assertEmptyComparedToStatic(1, ClusteringPrefix.Kind.CLUSTERING, Version.LEGACY);
+        assertEmptyComparedToStatic(0, ClusteringPrefix.Kind.STATIC_CLUSTERING, Version.LEGACY);
+        assertEmptyComparedToStatic(-1, ClusteringPrefix.Kind.INCL_START_BOUND, Version.LEGACY);
+        assertEmptyComparedToStatic(1, ClusteringPrefix.Kind.INCL_END_BOUND, Version.LEGACY);
+    }
+
+    private void assertEmptyComparedToStatic(int expected, ClusteringPrefix.Kind kind, Version version)
+    {
+        ClusteringPrefix<ByteBuffer> empty = makeBound(kind);
+        ClusteringComparator compEmpty = new ClusteringComparator();
+        assertEquals(expected, Integer.signum(ByteComparable.compare(compEmpty.asByteComparable(empty),
+                                                                     compEmpty.asByteComparable(Clustering.STATIC_CLUSTERING),
+                                                                     version)));
+    }
+
+    void assertClusteringPairComparesSame(AbstractType<?> t1, AbstractType<?> t2, Object o1, Object o2, Object o3, Object o4)
+    {
+        assertClusteringPairComparesSame(t1, t2, o1, o2, o3, o4, AbstractType::decompose, true);
+    }
+
+    void assertClusteringPairComparesSame(AbstractType<?> t1, AbstractType<?> t2,
+                                          Object o1, Object o2, Object o3, Object o4,
+                                          BiFunction<AbstractType, Object, ByteBuffer> decompose,
+                                          boolean testLegacy)
+    {
+        for (Version v : Version.values())
+            for (ClusteringPrefix.Kind k1 : ClusteringPrefix.Kind.values())
+                for (ClusteringPrefix.Kind k2 : ClusteringPrefix.Kind.values())
+                {
+                    if (!testLegacy && v == Version.LEGACY)
+                        continue;
+
+                    ClusteringComparator comp = new ClusteringComparator(t1, t2);
+                    ByteBuffer[] b = new ByteBuffer[2];
+                    ByteBuffer[] d = new ByteBuffer[2];
+                    b[0] = decompose.apply(t1, o1);
+                    b[1] = decompose.apply(t2, o2);
+                    d[0] = decompose.apply(t1, o3);
+                    d[1] = decompose.apply(t2, o4);
+                    ClusteringPrefix<ByteBuffer> c = makeBound(k1, b);
+                    ClusteringPrefix<ByteBuffer> e = makeBound(k2, d);
+                    final ByteComparable bsc = comp.asByteComparable(c);
+                    final ByteComparable bse = comp.asByteComparable(e);
+                    int expected = Integer.signum(comp.compare(c, e));
+                    assertEquals(String.format("Failed comparing %s and %s, %s vs %s version %s",
+                                               safeStr(c.clusteringString(comp.subtypes())),
+                                               safeStr(e.clusteringString(comp.subtypes())), bsc, bse, v),
+                                 expected, Integer.signum(ByteComparable.compare(bsc, bse, v)));
+                    maybeCheck41Properties(expected, bsc, bse, v);
+                    maybeAssertNotPrefix(bsc, bse, v);
+
+                    ClusteringComparator compR = new ClusteringComparator(ReversedType.getInstance(t1), ReversedType.getInstance(t2));
+                    final ByteComparable bsrc = compR.asByteComparable(c);
+                    final ByteComparable bsre = compR.asByteComparable(e);
+                    int expectedR = Integer.signum(compR.compare(c, e));
+                    assertEquals(String.format("Failed comparing reversed %s and %s, %s vs %s version %s",
+                                               safeStr(c.clusteringString(comp.subtypes())),
+                                               safeStr(e.clusteringString(comp.subtypes())), bsrc, bsre, v),
+                                 expectedR, Integer.signum(ByteComparable.compare(bsrc, bsre, v)));
+                    maybeCheck41Properties(expectedR, bsrc, bsre, v);
+                    maybeAssertNotPrefix(bsrc, bsre, v);
+                }
+    }
+
+    static ClusteringPrefix<ByteBuffer> makeBound(ClusteringPrefix.Kind k1, ByteBuffer... b)
+    {
+        return makeBound(ByteBufferAccessor.instance.factory(), k1, b);
+    }
+
+    static <T> ClusteringPrefix<T> makeBound(ValueAccessor.ObjectFactory<T> factory, ClusteringPrefix.Kind k1, T[] b)
+    {
+        switch (k1)
+        {
+        case INCL_END_EXCL_START_BOUNDARY:
+        case EXCL_END_INCL_START_BOUNDARY:
+            return factory.boundary(k1, b);
+
+        case INCL_END_BOUND:
+        case EXCL_END_BOUND:
+        case INCL_START_BOUND:
+        case EXCL_START_BOUND:
+            return factory.bound(k1, b);
+
+        case CLUSTERING:
+            return factory.clustering(b);
+
+        case STATIC_CLUSTERING:
+            return factory.staticClustering();
+
+        default:
+            throw new AssertionError();
+        }
+    }
+
+    @Test
+    public void testTupleType()
+    {
+        Random rand = ThreadLocalRandom.current();
+        testCombinationSampling(rand, this::assertTupleComparesSame);
+    }
+
+    @Test
+    public void testTupleTypeNonFull()
+    {
+        TupleType tt = new TupleType(ImmutableList.of(UTF8Type.instance, Int32Type.instance));
+        List<ByteBuffer> tests = ImmutableList.of
+            (
+            TupleType.buildValue(ByteBufferAccessor.instance,
+                                 decomposeAndRandomPad(UTF8Type.instance, ""),
+                                 decomposeAndRandomPad(Int32Type.instance, 0)),
+            // Note: a decomposed null (e.g. decomposeAndRandomPad(Int32Type.instance, null)) should not reach a tuple
+            TupleType.buildValue(ByteBufferAccessor.instance,
+                                 decomposeAndRandomPad(UTF8Type.instance, ""),
+                                 null),
+            TupleType.buildValue(ByteBufferAccessor.instance,
+                                 null,
+                                 decomposeAndRandomPad(Int32Type.instance, 0)),
+            TupleType.buildValue(ByteBufferAccessor.instance,
+                                 decomposeAndRandomPad(UTF8Type.instance, "")),
+            TupleType.buildValue(ByteBufferAccessor.instance, (ByteBuffer) null),
+            TupleType.buildValue(ByteBufferAccessor.instance)
+            );
+        testBuffers(tt, tests);
+    }
+
+    @Test
+    public void testTupleNewField()
+    {
+        TupleType t1 = new TupleType(ImmutableList.of(UTF8Type.instance));
+        TupleType t2 = new TupleType(ImmutableList.of(UTF8Type.instance, Int32Type.instance));
+
+        ByteBuffer vOne = TupleType.buildValue(ByteBufferAccessor.instance,
+                                               decomposeAndRandomPad(UTF8Type.instance, "str"));
+        ByteBuffer vOneAndNull = TupleType.buildValue(ByteBufferAccessor.instance,
+                                                      decomposeAndRandomPad(UTF8Type.instance, "str"),
+                                                      null);
+
+        ByteComparable bOne1 = typeToComparable(t1, vOne);
+        ByteComparable bOne2 = typeToComparable(t2, vOne);
+        ByteComparable bOneAndNull2 = typeToComparable(t2, vOneAndNull);
+
+        assertEquals("The byte-comparable version of a one-field tuple must be the same as a two-field tuple with non-present second component.",
+                     bOne1.byteComparableAsString(Version.OSS42),
+                     bOne2.byteComparableAsString(Version.OSS42));
+        assertEquals("The byte-comparable version of a one-field tuple must be the same as a two-field tuple with null as second component.",
+                     bOne1.byteComparableAsString(Version.OSS42),
+                     bOneAndNull2.byteComparableAsString(Version.OSS42));
+    }
+
+
+    void assertTupleComparesSame(AbstractType t1, AbstractType t2, Object o1, Object o2, Object o3, Object o4)
+    {
+        TupleType tt = new TupleType(ImmutableList.of(t1, t2));
+        ByteBuffer b1 = TupleType.buildValue(ByteBufferAccessor.instance,
+                                             decomposeForTuple(t1, o1),
+                                             decomposeForTuple(t2, o2));
+        ByteBuffer b2 = TupleType.buildValue(ByteBufferAccessor.instance,
+                                             decomposeForTuple(t1, o3),
+                                             decomposeForTuple(t2, o4));
+        assertComparesSameBuffers(tt, b1, b2);
+    }
+
+    static <T> ByteBuffer decomposeForTuple(AbstractType<T> t, T o)
+    {
+        return o != null ? t.decompose(o) : null;
+    }
+
+    @Test
+    public void testCompositeType()
+    {
+        Random rand = new Random(0);
+        testCombinationSampling(rand, this::assertCompositeComparesSame);
+    }
+
+    @Test
+    public void testCompositeTypeNonFull()
+    {
+        CompositeType tt = CompositeType.getInstance(UTF8Type.instance, Int32Type.instance);
+        List<ByteBuffer> tests = ImmutableList.of
+            (
+            CompositeType.build(ByteBufferAccessor.instance, decomposeAndRandomPad(UTF8Type.instance, ""), decomposeAndRandomPad(Int32Type.instance, 0)),
+            CompositeType.build(ByteBufferAccessor.instance, decomposeAndRandomPad(UTF8Type.instance, ""), decomposeAndRandomPad(Int32Type.instance, null)),
+            CompositeType.build(ByteBufferAccessor.instance, decomposeAndRandomPad(UTF8Type.instance, "")),
+            CompositeType.build(ByteBufferAccessor.instance),
+            CompositeType.build(ByteBufferAccessor.instance, true, decomposeAndRandomPad(UTF8Type.instance, "")),
+            CompositeType.build(ByteBufferAccessor.instance,true)
+            );
+        for (ByteBuffer b : tests)
+            tt.validate(b);
+        testBuffers(tt, tests);
+    }
+
+    void assertCompositeComparesSame(AbstractType t1, AbstractType t2, Object o1, Object o2, Object o3, Object o4)
+    {
+        CompositeType tt = CompositeType.getInstance(t1, t2);
+        ByteBuffer b1 = CompositeType.build(ByteBufferAccessor.instance, decomposeAndRandomPad(t1, o1), decomposeAndRandomPad(t2, o2));
+        ByteBuffer b2 = CompositeType.build(ByteBufferAccessor.instance, decomposeAndRandomPad(t1, o3), decomposeAndRandomPad(t2, o4));
+        assertComparesSameBuffers(tt, b1, b2);
+    }
+
+    @Test
+    public void testDynamicComposite()
+    {
+        DynamicCompositeType tt = DynamicCompositeType.getInstance(DynamicCompositeTypeTest.aliases);
+        UUID[] uuids = DynamicCompositeTypeTest.uuids;
+        List<ByteBuffer> tests = ImmutableList.of
+            (
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test1", null, -1, false, true),
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test1", uuids[0], 24, false, true),
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test1", uuids[0], 42, false, true),
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test2", uuids[0], -1, false, true),
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test2", uuids[1], 42, false, true)
+            );
+        for (ByteBuffer b : tests)
+            tt.validate(b);
+        testBuffers(tt, tests);
+    }
+
+    @Test
+    public void testListTypeString()
+    {
+        testCollection(ListType.getInstance(UTF8Type.instance, true), testStrings, () -> new ArrayList<>(), new Random());
+    }
+
+    @Test
+    public void testListTypeLong()
+    {
+        testCollection(ListType.getInstance(LongType.instance, true), testLongs, () -> new ArrayList<>(), new Random());
+    }
+
+    @Test
+    public void testSetTypeString()
+    {
+        testCollection(SetType.getInstance(UTF8Type.instance, true), testStrings, () -> new HashSet<>(), new Random());
+    }
+
+    @Test
+    public void testSetTypeLong()
+    {
+        testCollection(SetType.getInstance(LongType.instance, true), testLongs, () -> new HashSet<>(), new Random());
+    }
+
+    <T, CT extends Collection<T>> void testCollection(CollectionType<CT> tt, T[] values, Supplier<CT> gen, Random rand)
+    {
+        int cnt = 0;
+        List<CT> tests = new ArrayList<>();
+        tests.add(gen.get());
+        for (int c = 1; c <= 3; ++c)
+            for (int j = 0; j < 5; ++j)
+            {
+                CT l = gen.get();
+                for (int i = 0; i < c; ++i)
+                    l.add(values[cnt++ % values.length]);
+
+                tests.add(l);
+            }
+        testType(tt, tests);
+    }
+
+    @Test
+    public void testMapTypeStringLong()
+    {
+        testMap(MapType.getInstance(UTF8Type.instance, LongType.instance, true), testStrings, testLongs, () -> new HashMap<>(), new Random());
+    }
+
+    @Test
+    public void testMapTypeStringLongTree()
+    {
+        testMap(MapType.getInstance(UTF8Type.instance, LongType.instance, true), testStrings, testLongs, () -> new TreeMap<>(), new Random());
+    }
+
+    @Test
+    public void testDecoratedKeyPrefixesVOSS42()
+    {
+        // This should pass with the OSS 4.1 encoding
+        testDecoratedKeyPrefixes(Version.OSS42);
+    }
+
+    @Test
+    public void testDecoratedKeyPrefixesVLegacy()
+    {
+        // ... and fail with the legacy encoding
+        try
+        {
+            testDecoratedKeyPrefixes(Version.LEGACY);
+        }
+        catch (AssertionError e)
+        {
+            // Correct path, test failing.
+            return;
+        }
+        Assert.fail("Test expected to fail.");
+    }
+
+    @Test
+    public void testFixedLengthWithOffset()
+    {
+        byte[] bytes = new byte[]{ 1, 2, 3, 4, 5, 6, 7, 8, 9 };
+
+        ByteSource source = ByteSource.fixedLength(bytes, 0, 1);
+        assertEquals(1, source.next());
+        assertEquals(ByteSource.END_OF_STREAM, source.next());
+
+        source = ByteSource.fixedLength(bytes, 4, 5);
+        assertEquals(5, source.next());
+        assertEquals(6, source.next());
+        assertEquals(7, source.next());
+        assertEquals(8, source.next());
+        assertEquals(9, source.next());
+        assertEquals(ByteSource.END_OF_STREAM, source.next());
+
+        ByteSource.fixedLength(bytes, 9, 0);
+        assertEquals(ByteSource.END_OF_STREAM, source.next());
+    }
+
+    @Test
+    public void testFixedLengthNegativeLength()
+    {
+        byte[] bytes = new byte[]{ 1, 2, 3 };
+
+        expectedException.expect(IllegalArgumentException.class);
+        ByteSource.fixedLength(bytes, 0, -1);
+    }
+
+    @Test
+    public void testFixedLengthNegativeOffset()
+    {
+        byte[] bytes = new byte[]{ 1, 2, 3 };
+
+        expectedException.expect(IllegalArgumentException.class);
+        ByteSource.fixedLength(bytes, -1, 1);
+    }
+
+    @Test
+    public void testFixedLengthOutOfBounds()
+    {
+        byte[] bytes = new byte[]{ 1, 2, 3 };
+
+        expectedException.expect(IllegalArgumentException.class);
+        ByteSource.fixedLength(bytes, 0, 4);
+    }
+
+    @Test
+    public void testFixedOffsetOutOfBounds()
+    {
+        byte[] bytes = new byte[]{ 1, 2, 3 };
+
+        expectedException.expect(IllegalArgumentException.class);
+        ByteSource.fixedLength(bytes, 4, 1);
+    }
+
+    @Test
+    public void testSeparatorGT()
+    {
+        testSeparator(ByteComparable::separatorGt, testLongs, LongType.instance);
+    }
+
+    @Test
+    public void testSeparatorPrefix()
+    {
+        testSeparator(ByteComparable::separatorPrefix, testLongs, LongType.instance);
+    }
+
+    @Test
+    public void testSeparatorPrefixViaDiffPoint()
+    {
+        testSeparator((x, y) -> version -> ByteSource.cut(y.asComparableBytes(version),
+                                                          ByteComparable.diffPoint(x, y, version)),
+                      testLongs,
+                      LongType.instance);
+    }
+    @Test
+    public void testSeparatorNext()
+    {
+        // Appending a 00 byte at the end gives the immediate next possible value after x.
+        testSeparator((x, y) -> version -> ByteSource.cutOrRightPad(x.asComparableBytes(version),
+                                                                    ByteComparable.length(x, version) + 1,
+                                                                    0),
+                      testLongs,
+                      LongType.instance);
+    }
+
+    private <T> void testSeparator(BiFunction<ByteComparable, ByteComparable, ByteComparable> separatorMethod, T[] testValues, AbstractType<T> type)
+    {
+        for (T v1 : testValues)
+            for (T v2 : testValues)
+            {
+                if (v1 == null || v2 == null)
+                    continue;
+                if (type.compare(type.decompose(v1), type.decompose(v2)) >= 0)
+                    continue;
+                ByteComparable bc1 = getByteComparable(type, v1);
+                ByteComparable bc2 = getByteComparable(type, v2);
+                ByteComparable separator = separatorMethod.apply(bc1, bc2);
+
+                for (Version version : Version.values())
+                {
+                    Assert.assertTrue("Sanity check failed", ByteComparable.compare(bc1, bc2, version) < 0);
+                    Assert.assertTrue(String.format("Separator %s must be greater than left %s (for %s) (version %s)",
+                                                    separator.byteComparableAsString(version),
+                                                    bc1.byteComparableAsString(version),
+                                                    v1,
+                                                    version),
+                                      ByteComparable.compare(bc1, separator, version) < 0);
+                    Assert.assertTrue(String.format("Separator %s must be less than or equal to right %s (for %s) (version %s)",
+                                                    separator.byteComparableAsString(version),
+                                                    bc2.byteComparableAsString(version),
+                                                    v2,
+                                                    version),
+                                      ByteComparable.compare(separator, bc2, version) <= 0);
+                }
+            }
+    }
+
+    private <T> ByteComparable getByteComparable(AbstractType<T> type, T v1)
+    {
+        return version -> type.asComparableBytes(type.decompose(v1), version);
+    }
+
+    public void testDecoratedKeyPrefixes(Version version)
+    {
+        testDecoratedKeyPrefixes("012345678BCDE\0", "", version);
+        testDecoratedKeyPrefixes("012345678ABCDE\0", "ABC", version);
+        testDecoratedKeyPrefixes("0123456789ABCDE\0", "\0AB", version);
+        testDecoratedKeyPrefixes("0123456789ABCDEF\0", "\0", version);
+
+        testDecoratedKeyPrefixes("0123456789ABCDEF0", "ABC", version);
+        testDecoratedKeyPrefixes("0123456789ABCDEF", "", version);
+        testDecoratedKeyPrefixes("0123456789ABCDE", "", version);
+        testDecoratedKeyPrefixes("0123456789ABCD", "\0AB", version);
+        testDecoratedKeyPrefixes("0123456789ABC", "\0", version);
+
+    }
+
+    public void testDecoratedKeyPrefixes(String key, String append, Version version)
+    {
+        logger.info("Testing {} + {}", safeStr(key), safeStr(append));
+        IPartitioner partitioner = Murmur3Partitioner.instance;
+        ByteBuffer original = ByteBufferUtil.bytes(key);
+        ByteBuffer collision = Util.generateMurmurCollision(original, append.getBytes(StandardCharsets.UTF_8));
+
+        long[] hash = new long[2];
+        MurmurHash.hash3_x64_128(original, 0, original.limit(), 0, hash);
+        logger.info(String.format("Original hash  %016x,%016x", hash[0], hash[1]));
+        MurmurHash.hash3_x64_128(collision, 0, collision.limit(), 0, hash);
+        logger.info(String.format("Collision hash %016x,%016x", hash[0], hash[1]));
+
+        DecoratedKey kk1 = partitioner.decorateKey(original);
+        DecoratedKey kk2 = partitioner.decorateKey(collision);
+        logger.info("{}\n{}\n{}\n{}", kk1, kk2, kk1.byteComparableAsString(version), kk2.byteComparableAsString(version));
+
+        final ByteSource s1 = kk1.asComparableBytes(version);
+        final ByteSource s2 = kk2.asComparableBytes(version);
+        logger.info("{}\n{}", s1, s2);
+
+        // Check that the representations compare correctly
+        Assert.assertEquals(Long.signum(kk1.compareTo(kk2)), ByteComparable.compare(kk1, kk2, version));
+        // s1 must not be a prefix of s2
+        assertNotPrefix(s1, s2);
+    }
+
+    private void assertNotPrefix(ByteSource s1, ByteSource s2)
+    {
+        int c1, c2;
+        do
+        {
+            c1 = s1.next();
+            c2 = s2.next();
+        }
+        while (c1 == c2 && c1 != ByteSource.END_OF_STREAM);
+
+        // Equal is ok
+        if (c1 == c2)
+            return;
+
+        Assert.assertNotEquals("ByteComparable is a prefix of other", ByteSource.END_OF_STREAM, c1);
+        Assert.assertNotEquals("ByteComparable is a prefix of other", ByteSource.END_OF_STREAM, c2);
+    }
+
+    private int compare(ByteSource s1, ByteSource s2)
+    {
+        int c1, c2;
+        do
+        {
+            c1 = s1.next();
+            c2 = s2.next();
+        }
+        while (c1 == c2 && c1 != ByteSource.END_OF_STREAM);
+
+        return Integer.compare(c1, c2);
+    }
+
+    private void maybeAssertNotPrefix(ByteComparable s1, ByteComparable s2, Version version)
+    {
+        if (version == Version.OSS42)
+            assertNotPrefix(s1.asComparableBytes(version), s2.asComparableBytes(version));
+    }
+
+    private void maybeCheck41Properties(int expectedComparison, ByteComparable s1, ByteComparable s2, Version version)
+    {
+        if (version != Version.OSS42)
+            return;
+
+        if (s1 == null || s2 == null || 0 == expectedComparison)
+            return;
+        int b1 = randomTerminator();
+        int b2 = randomTerminator();
+        assertEquals(String.format("Comparison failed for %s(%s + %02x) and %s(%s + %02x)", s1, s1.byteComparableAsString(version), b1, s2, s2.byteComparableAsString(version), b2),
+                expectedComparison, Integer.signum(compare(ByteSource.withTerminator(b1, s1.asComparableBytes(version)), ByteSource.withTerminator(b2, s2.asComparableBytes(version)))));
+        assertNotPrefix(ByteSource.withTerminator(b1, s1.asComparableBytes(version)), ByteSource.withTerminator(b2, s2.asComparableBytes(version)));
+    }
+
+    private int randomTerminator()
+    {
+        int term;
+        do
+        {
+            term = ThreadLocalRandom.current().nextInt(ByteSource.MIN_SEPARATOR, ByteSource.MAX_SEPARATOR + 1);
+        }
+        while (term >= ByteSource.MIN_NEXT_COMPONENT && term <= ByteSource.MAX_NEXT_COMPONENT);
+        return term;
+    }
+
+    <K, V, M extends Map<K, V>> void testMap(MapType<K, V> tt, K[] keys, V[] values, Supplier<M> gen, Random rand)
+    {
+        List<M> tests = new ArrayList<>();
+        tests.add(gen.get());
+        for (int c = 1; c <= 3; ++c)
+            for (int j = 0; j < 5; ++j)
+            {
+                M l = gen.get();
+                for (int i = 0; i < c; ++i)
+                    l.put(keys[rand.nextInt(keys.length)], values[rand.nextInt(values.length)]);
+
+                tests.add(l);
+            }
+        testType(tt, tests);
+    }
+
+    /*
+     * Convert type to a comparable.
+     */
+    private ByteComparable typeToComparable(AbstractType<?> type, ByteBuffer value)
+    {
+        return new ByteComparable()
+        {
+            @Override
+            public ByteSource asComparableBytes(Version v)
+            {
+                return type.asComparableBytes(value, v);
+            }
+
+            @Override
+            public String toString()
+            {
+                return type.getString(value);
+            }
+        };
+    }
+
+    public <T> void testType(AbstractType<T> type, Object[] values)
+    {
+        testType(type, Iterables.transform(Arrays.asList(values), x -> (T) x));
+    }
+
+    public <T> void testType(AbstractType<? super T> type, Iterable<T> values)
+    {
+        for (T i : values) {
+            ByteBuffer b = decomposeAndRandomPad(type, i);
+            logger.info("Value {} ({}) bytes {} ByteSource {}",
+                              safeStr(i),
+                              safeStr(type.getSerializer().toCQLLiteral(b)),
+                              safeStr(ByteBufferUtil.bytesToHex(b)),
+                              typeToComparable(type, b).byteComparableAsString(Version.OSS42));
+        }
+        for (T i : values)
+            for (T j : values)
+                assertComparesSame(type, i, j);
+        if (!type.isReversed())
+            testType(ReversedType.getInstance(type), values);
+    }
+
+    public void testBuffers(AbstractType<?> type, List<ByteBuffer> values)
+    {
+        try
+        {
+            for (ByteBuffer b : values) {
+                logger.info("Value {} bytes {} ByteSource {}",
+                            safeStr(type.getSerializer().toCQLLiteral(b)),
+                            safeStr(ByteBufferUtil.bytesToHex(b)),
+                            typeToComparable(type, b).byteComparableAsString(Version.OSS42));
+            }
+        }
+        catch (UnsupportedOperationException e)
+        {
+            // Continue without listing values.
+        }
+
+        for (ByteBuffer i : values)
+            for (ByteBuffer j : values)
+                assertComparesSameBuffers(type, i, j);
+    }
+
+    void assertComparesSameBuffers(AbstractType<?> type, ByteBuffer b1, ByteBuffer b2)
+    {
+        int expected = Integer.signum(type.compare(b1, b2));
+        final ByteComparable bs1 = typeToComparable(type, b1);
+        final ByteComparable bs2 = typeToComparable(type, b2);
+
+        for (Version version : Version.values())
+        {
+            int actual = Integer.signum(ByteComparable.compare(bs1, bs2, version));
+            assertEquals(String.format("Failed comparing %s(%s) and %s(%s)", ByteBufferUtil.bytesToHex(b1), bs1.byteComparableAsString(version), ByteBufferUtil.bytesToHex(b2), bs2.byteComparableAsString(version)),
+                         expected,
+                         actual);
+            maybeCheck41Properties(expected, bs1, bs2, version);
+        }
+    }
+
+    public void testDecoratedKeys(IPartitioner type, List<ByteBuffer> values)
+    {
+        for (ByteBuffer i : values)
+            for (ByteBuffer j : values)
+                assertComparesSameDecoratedKeys(type, i, j);
+        for (ByteBuffer i : values)
+            assertDecoratedKeyBounds(type, i);
+    }
+
+    void assertComparesSameDecoratedKeys(IPartitioner type, ByteBuffer b1, ByteBuffer b2)
+    {
+        DecoratedKey k1 = type.decorateKey(b1);
+        DecoratedKey k2 = type.decorateKey(b2);
+        int expected = Integer.signum(k1.compareTo(k2));
+
+        for (Version version : Version.values())
+        {
+            int actual = Integer.signum(ByteComparable.compare(k1, k2, version));
+            assertEquals(String.format("Failed comparing %s[%s](%s) and %s[%s](%s)\npartitioner %s version %s",
+                                       ByteBufferUtil.bytesToHex(b1),
+                                       k1,
+                                       k1.byteComparableAsString(version),
+                                       ByteBufferUtil.bytesToHex(b2),
+                                       k2,
+                                       k2.byteComparableAsString(version),
+                                       type,
+                                       version),
+                         expected,
+                         actual);
+            maybeAssertNotPrefix(k1, k2, version);
+        }
+    }
+
+    void assertDecoratedKeyBounds(IPartitioner type, ByteBuffer b)
+    {
+        Version version = Version.OSS42;
+        DecoratedKey k = type.decorateKey(b);
+        final ByteComparable after = k.asComparableBound(false);
+        final ByteComparable before = k.asComparableBound(true);
+
+        int actual = Integer.signum(ByteComparable.compare(k, before, version));
+        assertEquals(String.format("Failed comparing bound before (%s) for %s[%s](%s)\npartitioner %s version %s",
+                                   before.byteComparableAsString(version),
+                                   ByteBufferUtil.bytesToHex(b),
+                                   k,
+                                   k.byteComparableAsString(version),
+                                   type,
+                                   version),
+                     1,
+                     actual);
+        maybeAssertNotPrefix(k, before, version);
+
+        actual = Integer.signum(ByteComparable.compare(k, after, version));
+        assertEquals(String.format("Failed comparing bound after (%s) for %s[%s](%s)\npartitioner %s version %s",
+                                   after.byteComparableAsString(version),
+                                   ByteBufferUtil.bytesToHex(b),
+                                   k,
+                                   k.byteComparableAsString(version),
+                                   type,
+                                   version),
+                     -1,
+                     actual);
+        maybeAssertNotPrefix(k, after, version);
+
+        actual = Integer.signum(ByteComparable.compare(before, after, version));
+        assertEquals(String.format("Failed comparing bound before (%s) to after (%s) for %s[%s](%s)\npartitioner %s version %s",
+                                   before.byteComparableAsString(version),
+                                   after.byteComparableAsString(version),
+                                   ByteBufferUtil.bytesToHex(b),
+                                   k,
+                                   k.byteComparableAsString(version),
+                                   type,
+                                   version),
+                     -1,
+                     actual);
+        maybeAssertNotPrefix(after, before, version);
+    }
+
+    static Object safeStr(Object i)
+    {
+        if (i == null)
+            return null;
+        String s = i.toString();
+        if (s.length() > 100)
+            s = s.substring(0, 100) + "...";
+        return s.replaceAll("\0", "<0>");
+    }
+
+    public <T> void testDirect(Function<T, ByteSource> convertor, BiFunction<T, T, Integer> comparator, T[] values)
+    {
+        for (T i : values) {
+            if (i == null)
+                continue;
+
+            logger.info("Value {} ByteSource {}\n",
+                              safeStr(i),
+                              convertor.apply(i));
+        }
+        for (T i : values)
+            if (i != null)
+                for (T j : values)
+                    if (j != null)
+                        assertComparesSame(convertor, comparator, i, j);
+    }
+
+    <T> void assertComparesSame(Function<T, ByteSource> convertor, BiFunction<T, T, Integer> comparator, T v1, T v2)
+    {
+        ByteComparable b1 = v -> convertor.apply(v1);
+        ByteComparable b2 = v -> convertor.apply(v2);
+        int expected = Integer.signum(comparator.apply(v1, v2));
+        int actual = Integer.signum(ByteComparable.compare(b1, b2, null));  // version ignored above
+        assertEquals(String.format("Failed comparing %s and %s", v1, v2), expected, actual);
+    }
+
+    <T> void assertComparesSame(AbstractType<T> type, T v1, T v2)
+    {
+        ByteBuffer b1 = decomposeAndRandomPad(type, v1);
+        ByteBuffer b2 = decomposeAndRandomPad(type, v2);
+        int expected = Integer.signum(type.compare(b1, b2));
+        final ByteComparable bc1 = typeToComparable(type, b1);
+        final ByteComparable bc2 = typeToComparable(type, b2);
+
+        for (Version version : Version.values())
+        {
+            int actual = Integer.signum(ByteComparable.compare(bc1, bc2, version));
+            if (expected != actual)
+            {
+                if (type.isReversed())
+                {
+                    // This can happen for reverse of nulls and prefixes. Check that it's ok within multi-component
+                    ClusteringComparator cc = new ClusteringComparator(type);
+                    ByteComparable c1 = cc.asByteComparable(Clustering.make(b1));
+                    ByteComparable c2 = cc.asByteComparable(Clustering.make(b2));
+                    int actualcc = Integer.signum(ByteComparable.compare(c1, c2, version));
+                    if (actualcc == expected)
+                        return;
+                    assertEquals(String.format("Failed comparing reversed %s(%s, %s) and %s(%s, %s) direct (%d) and as clustering", safeStr(v1), ByteBufferUtil.bytesToHex(b1), c1, safeStr(v2), ByteBufferUtil.bytesToHex(b2), c2, actual), expected, actualcc);
+                }
+                else
+                    assertEquals(String.format("Failed comparing %s(%s BC %s) and %s(%s BC %s) version %s",
+                                               safeStr(v1),
+                                               ByteBufferUtil.bytesToHex(b1),
+                                               bc1.byteComparableAsString(version),
+                                               safeStr(v2),
+                                               ByteBufferUtil.bytesToHex(b2),
+                                               bc2.byteComparableAsString(version),
+                                               version),
+                                 expected,
+                                 actual);
+            }
+            maybeCheck41Properties(expected, bc1, bc2, version);
+        }
+    }
+
+    <T> ByteBuffer decomposeAndRandomPad(AbstractType<T> type, T v)
+    {
+        ByteBuffer b = type.decompose(v);
+        Random rand = new Random(0);
+        int padBefore = rand.nextInt(16);
+        int padAfter = rand.nextInt(16);
+        int paddedCapacity = b.remaining() + padBefore + padAfter;
+        ByteBuffer padded = allocateBuffer(paddedCapacity);
+        rand.ints(padBefore).forEach(x -> padded.put((byte) x));
+        padded.put(b.duplicate());
+        rand.ints(padAfter).forEach(x -> padded.put((byte) x));
+        padded.clear().limit(padded.capacity() - padAfter).position(padBefore);
+        return padded;
+    }
+
+    protected ByteBuffer allocateBuffer(int paddedCapacity)
+    {
+        return ByteBuffer.allocate(paddedCapacity);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/utils/bytecomparable/ByteSourceConversionTest.java b/test/unit/org/apache/cassandra/utils/bytecomparable/ByteSourceConversionTest.java
new file mode 100644
index 0000000000..5a59ddfe1f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/bytecomparable/ByteSourceConversionTest.java
@@ -0,0 +1,784 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.utils.bytecomparable;
+
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.BufferDecoratedKey;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.TimeUUID;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
+
+import static org.apache.cassandra.utils.bytecomparable.ByteSourceComparisonTest.decomposeForTuple;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests that the result of forward + backward ByteSource translation is the same as the original.
+ */
+public class ByteSourceConversionTest extends ByteSourceTestBase
+{
+    private final static Logger logger = LoggerFactory.getLogger(ByteSourceConversionTest.class);
+    public static final Version VERSION = Version.OSS42;
+
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Test
+    public void testStringsAscii()
+    {
+        testType(AsciiType.instance, Arrays.stream(testStrings)
+                                           .filter(s -> s.equals(new String(s.getBytes(StandardCharsets.US_ASCII),
+                                                                            StandardCharsets.US_ASCII)))
+                                           .toArray());
+    }
+
+    @Test
+    public void testStringsUTF8()
+    {
+        testType(UTF8Type.instance, testStrings);
+        testDirect(x -> ByteSource.of(x, VERSION), ByteSourceInverse::getString, testStrings);
+    }
+
+    @Test
+    public void testBooleans()
+    {
+        testType(BooleanType.instance, testBools);
+    }
+
+    @Test
+    public void testInts()
+    {
+        testType(Int32Type.instance, testInts);
+        testDirect(ByteSource::of, ByteSourceInverse::getSignedInt, testInts);
+    }
+
+    @Test
+    public void randomTestInts()
+    {
+        Random rand = new Random();
+        for (int i=0; i<10000; ++i)
+        {
+            int i1 = rand.nextInt();
+            assertConvertsSame(Int32Type.instance, i1);
+        }
+
+    }
+
+    @Test
+    public void testLongs()
+    {
+        testType(LongType.instance, testLongs);
+        testDirect(ByteSource::of, ByteSourceInverse::getSignedLong, testLongs);
+    }
+
+    @Test
+    public void testShorts()
+    {
+        testType(ShortType.instance, testShorts);
+    }
+
+    @Test
+    public void testBytes()
+    {
+        testType(ByteType.instance, testBytes);
+    }
+
+    @Test
+    public void testDoubles()
+    {
+        testType(DoubleType.instance, testDoubles);
+    }
+
+    @Test
+    public void testFloats()
+    {
+        testType(FloatType.instance, testFloats);
+    }
+
+    @Test
+    public void testBigInts()
+    {
+        testType(IntegerType.instance, testBigInts);
+    }
+
+    @Test
+    public void testBigDecimals()
+    {
+        testTypeBuffers(DecimalType.instance, testBigDecimals);
+    }
+
+    @Test
+    public void testUUIDs()
+    {
+        testType(UUIDType.instance, testUUIDs);
+    }
+
+    @Test
+    public void testTimeUUIDs()
+    {
+        testType(TimeUUIDType.instance, Arrays.stream(testUUIDs)
+                                              .filter(x -> x == null || x.version() == 1)
+                                              .map(x -> x != null ? TimeUUID.fromUuid(x) : null)
+                                              .toArray());
+    }
+
+    @Test
+    public void testLexicalUUIDs()
+    {
+        testType(LexicalUUIDType.instance, testUUIDs);
+    }
+
+    @Test
+    public void testSimpleDate()
+    {
+        testType(SimpleDateType.instance, Arrays.stream(testInts).filter(x -> x != null).toArray());
+    }
+
+    @Test
+    public void testTimeType()
+    {
+        testType(TimeType.instance, Arrays.stream(testLongs).filter(x -> x != null && x >= 0 && x <= 24L * 60 * 60 * 1000 * 1000 * 1000).toArray());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testDateType()
+    {
+        testType(DateType.instance, testDates);
+    }
+
+    @Test
+    public void testTimestampType()
+    {
+        testType(TimestampType.instance, testDates);
+    }
+
+    @Test
+    public void testBytesType()
+    {
+        List<ByteBuffer> values = new ArrayList<>();
+        for (int i = 0; i < testValues.length; ++i)
+            for (Object o : testValues[i])
+                values.add(testTypes[i].decompose(o));
+
+        testType(BytesType.instance, values);
+    }
+
+    @Test
+    public void testInetAddressType() throws UnknownHostException
+    {
+        testType(InetAddressType.instance, testInets);
+    }
+
+    @Test
+    public void testEmptyType()
+    {
+        testType(EmptyType.instance, new Void[] { null });
+    }
+
+    @Test
+    public void testPatitionerDefinedOrder()
+    {
+        List<ByteBuffer> values = new ArrayList<>();
+        for (int i = 0; i < testValues.length; ++i)
+            for (Object o : testValues[i])
+                values.add(testTypes[i].decompose(o));
+
+        testBuffers(new PartitionerDefinedOrder(Murmur3Partitioner.instance), values);
+        testBuffers(new PartitionerDefinedOrder(RandomPartitioner.instance), values);
+        testBuffers(new PartitionerDefinedOrder(ByteOrderedPartitioner.instance), values);
+    }
+
+    @Test
+    public void testPatitionerOrder()
+    {
+        List<ByteBuffer> values = new ArrayList<>();
+        for (int i = 0; i < testValues.length; ++i)
+            for (Object o : testValues[i])
+                values.add(testTypes[i].decompose(o));
+
+        testDecoratedKeys(Murmur3Partitioner.instance, values);
+        testDecoratedKeys(RandomPartitioner.instance, values);
+        testDecoratedKeys(ByteOrderedPartitioner.instance, values);
+    }
+
+    @Test
+    public void testLocalPatitionerOrder()
+    {
+        for (int i = 0; i < testValues.length; ++i)
+        {
+            final AbstractType testType = testTypes[i];
+            testDecoratedKeys(new LocalPartitioner(testType), Lists.transform(Arrays.asList(testValues[i]),
+                                                                                            v -> testType.decompose(v)));
+        }
+    }
+
+    interface PairTester
+    {
+        void test(AbstractType t1, AbstractType t2, Object o1, Object o2);
+    }
+
+    void testCombinationSampling(Random rand, PairTester tester)
+    {
+        for (int i=0;i<testTypes.length;++i)
+            for (int j=0;j<testTypes.length;++j)
+            {
+                Object[] tv1 = new Object[3];
+                Object[] tv2 = new Object[3];
+                for (int t=0; t<tv1.length; ++t)
+                {
+                    tv1[t] = testValues[i][rand.nextInt(testValues[i].length)];
+                    tv2[t] = testValues[j][rand.nextInt(testValues[j].length)];
+                }
+
+                for (Object o1 : tv1)
+                    for (Object o2 : tv2)
+
+                {
+                    tester.test(testTypes[i], testTypes[j], o1, o2);
+                }
+            }
+    }
+
+    @Test
+    public void testCombinations()
+    {
+        Random rand = new Random(0);
+        testCombinationSampling(rand, this::assertClusteringPairConvertsSame);
+    }
+
+    @Test
+    public void testNullsInClustering()
+    {
+        ByteBuffer[][] inputs = new ByteBuffer[][]
+                                {
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, "a"),
+                                                  decomposeAndRandomPad(Int32Type.instance, 0)},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, "a"),
+                                                  decomposeAndRandomPad(Int32Type.instance, null)},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, "a"),
+                                                  null},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, ""),
+                                                  decomposeAndRandomPad(Int32Type.instance, 0)},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, ""),
+                                                  decomposeAndRandomPad(Int32Type.instance, null)},
+                                new ByteBuffer[] {decomposeAndRandomPad(UTF8Type.instance, ""),
+                                                  null},
+                                new ByteBuffer[] {null,
+                                                  decomposeAndRandomPad(Int32Type.instance, 0)},
+                                new ByteBuffer[] {null,
+                                                  decomposeAndRandomPad(Int32Type.instance, null)},
+                                new ByteBuffer[] {null,
+                                                  null},
+                                };
+        for (ByteBuffer[] input : inputs)
+        {
+            assertClusteringPairConvertsSame(ByteBufferAccessor.instance,
+                                             UTF8Type.instance,
+                                             Int32Type.instance,
+                                             input[0],
+                                             input[1],
+                                             (t, v) -> (ByteBuffer) v);
+        }
+    }
+
+    @Test
+    public void testEmptyClustering()
+    {
+        ValueAccessor<ByteBuffer> accessor = ByteBufferAccessor.instance;
+        ClusteringComparator comp = new ClusteringComparator();
+        for (ClusteringPrefix.Kind kind : ClusteringPrefix.Kind.values())
+        {
+            if (kind.isBoundary())
+                continue;
+
+            ClusteringPrefix<ByteBuffer> empty = ByteSourceComparisonTest.makeBound(kind);
+            ClusteringPrefix<ByteBuffer> converted = getClusteringPrefix(accessor, kind, comp, comp.asByteComparable(empty));
+            assertEquals(empty, converted);
+        }
+    }
+
+    void assertClusteringPairConvertsSame(AbstractType t1, AbstractType t2, Object o1, Object o2)
+    {
+        for (ValueAccessor<?> accessor : ValueAccessors.ACCESSORS)
+            assertClusteringPairConvertsSame(accessor, t1, t2, o1, o2, AbstractType::decompose);
+    }
+
+    <V> void assertClusteringPairConvertsSame(ValueAccessor<V> accessor,
+                                              AbstractType<?> t1, AbstractType<?> t2,
+                                              Object o1, Object o2,
+                                              BiFunction<AbstractType, Object, ByteBuffer> decompose)
+    {
+        boolean checkEquals = t1 != DecimalType.instance && t2 != DecimalType.instance;
+        for (ClusteringPrefix.Kind k1 : ClusteringPrefix.Kind.values())
+            {
+                ClusteringComparator comp = new ClusteringComparator(t1, t2);
+                V[] b = accessor.createArray(2);
+                b[0] = accessor.valueOf(decompose.apply(t1, o1));
+                b[1] = accessor.valueOf(decompose.apply(t2, o2));
+                ClusteringPrefix<V> c = ByteSourceComparisonTest.makeBound(accessor.factory(), k1, b);
+                final ByteComparable bsc = comp.asByteComparable(c);
+                logger.info("Clustering {} bytesource {}", c.clusteringString(comp.subtypes()), bsc.byteComparableAsString(VERSION));
+                ClusteringPrefix<V> converted = getClusteringPrefix(accessor, k1, comp, bsc);
+                assertEquals(String.format("Failed compare(%s, converted %s ByteSource %s) == 0\ntype %s",
+                                           safeStr(c.clusteringString(comp.subtypes())),
+                                           safeStr(converted.clusteringString(comp.subtypes())),
+                                           bsc.byteComparableAsString(VERSION),
+                                           comp),
+                             0, comp.compare(c, converted));
+                if (checkEquals)
+                    assertEquals(String.format("Failed equals %s, got %s ByteSource %s\ntype %s",
+                                               safeStr(c.clusteringString(comp.subtypes())),
+                                               safeStr(converted.clusteringString(comp.subtypes())),
+                                               bsc.byteComparableAsString(VERSION),
+                                               comp),
+                                 c, converted);
+
+                ClusteringComparator compR = new ClusteringComparator(ReversedType.getInstance(t1), ReversedType.getInstance(t2));
+                final ByteComparable bsrc = compR.asByteComparable(c);
+                converted = getClusteringPrefix(accessor, k1, compR, bsrc);
+                assertEquals(String.format("Failed reverse compare(%s, converted %s ByteSource %s) == 0\ntype %s",
+                                           safeStr(c.clusteringString(compR.subtypes())),
+                                           safeStr(converted.clusteringString(compR.subtypes())),
+                                           bsrc.byteComparableAsString(VERSION),
+                                           compR),
+                             0, compR.compare(c, converted));
+                if (checkEquals)
+                    assertEquals(String.format("Failed reverse equals %s, got %s ByteSource %s\ntype %s",
+                                               safeStr(c.clusteringString(compR.subtypes())),
+                                               safeStr(converted.clusteringString(compR.subtypes())),
+                                               bsrc.byteComparableAsString(VERSION),
+                                               compR),
+                                 c, converted);
+            }
+    }
+
+    private static <V> ClusteringPrefix<V> getClusteringPrefix(ValueAccessor<V> accessor,
+                                                               ClusteringPrefix.Kind k1,
+                                                               ClusteringComparator comp,
+                                                               ByteComparable bsc)
+    {
+        switch (k1)
+        {
+        case STATIC_CLUSTERING:
+        case CLUSTERING:
+            return comp.clusteringFromByteComparable(accessor, bsc);
+        case EXCL_END_BOUND:
+        case INCL_END_BOUND:
+            return comp.boundFromByteComparable(accessor, bsc, true);
+        case INCL_START_BOUND:
+        case EXCL_START_BOUND:
+            return comp.boundFromByteComparable(accessor, bsc, false);
+        case EXCL_END_INCL_START_BOUNDARY:
+        case INCL_END_EXCL_START_BOUNDARY:
+            return comp.boundaryFromByteComparable(accessor, bsc);
+        default:
+            throw new AssertionError();
+        }
+    }
+
+    private static ByteSource.Peekable source(ByteComparable bsc)
+    {
+        if (bsc == null)
+            return null;
+        return ByteSource.peekable(bsc.asComparableBytes(VERSION));
+    }
+
+    @Test
+    public void testTupleType()
+    {
+        Random rand = ThreadLocalRandom.current();
+        testCombinationSampling(rand, this::assertTupleConvertsSame);
+    }
+
+    @Test
+    public void testTupleTypeNonFull()
+    {
+        TupleType tt = new TupleType(ImmutableList.of(UTF8Type.instance, Int32Type.instance));
+        List<ByteBuffer> tests = ImmutableList.of
+            (
+            TupleType.buildValue(ByteBufferAccessor.instance,
+                                 decomposeAndRandomPad(UTF8Type.instance, ""),
+                                 decomposeAndRandomPad(Int32Type.instance, 0)),
+            // Note: a decomposed null (e.g. decomposeAndRandomPad(Int32Type.instance, null)) should not reach a tuple
+            TupleType.buildValue(ByteBufferAccessor.instance,
+                                 decomposeAndRandomPad(UTF8Type.instance, ""),
+                                 null),
+            TupleType.buildValue(ByteBufferAccessor.instance,
+                                 null,
+                                 decomposeAndRandomPad(Int32Type.instance, 0)),
+            TupleType.buildValue(ByteBufferAccessor.instance, decomposeAndRandomPad(UTF8Type.instance, "")),
+            TupleType.buildValue(ByteBufferAccessor.instance, (ByteBuffer) null),
+            TupleType.buildValue(ByteBufferAccessor.instance)
+            );
+        testBuffers(tt, tests);
+    }
+
+    void assertTupleConvertsSame(AbstractType t1, AbstractType t2, Object o1, Object o2)
+    {
+        TupleType tt = new TupleType(ImmutableList.of(t1, t2));
+        ByteBuffer b1 = TupleType.buildValue(ByteBufferAccessor.instance,
+                                             decomposeForTuple(t1, o1),
+                                             decomposeForTuple(t2, o2));
+        assertConvertsSameBuffers(tt, b1);
+    }
+
+    @Test
+    public void testCompositeType()
+    {
+        Random rand = new Random(0);
+        testCombinationSampling(rand, this::assertCompositeConvertsSame);
+    }
+
+    @Test
+    public void testCompositeTypeNonFull()
+    {
+        CompositeType tt = CompositeType.getInstance(UTF8Type.instance, Int32Type.instance);
+        List<ByteBuffer> tests = ImmutableList.of
+            (
+            CompositeType.build(ByteBufferAccessor.instance, decomposeAndRandomPad(UTF8Type.instance, ""), decomposeAndRandomPad(Int32Type.instance, 0)),
+            CompositeType.build(ByteBufferAccessor.instance, decomposeAndRandomPad(UTF8Type.instance, ""), decomposeAndRandomPad(Int32Type.instance, null)),
+            CompositeType.build(ByteBufferAccessor.instance, decomposeAndRandomPad(UTF8Type.instance, "")),
+            CompositeType.build(ByteBufferAccessor.instance),
+            CompositeType.build(ByteBufferAccessor.instance, true, decomposeAndRandomPad(UTF8Type.instance, "")),
+            CompositeType.build(ByteBufferAccessor.instance,true)
+            );
+        for (ByteBuffer b : tests)
+            tt.validate(b);
+        testBuffers(tt, tests);
+    }
+
+    void assertCompositeConvertsSame(AbstractType t1, AbstractType t2, Object o1, Object o2)
+    {
+        CompositeType tt = CompositeType.getInstance(t1, t2);
+        ByteBuffer b1 = CompositeType.build(ByteBufferAccessor.instance, decomposeAndRandomPad(t1, o1), decomposeAndRandomPad(t2, o2));
+        assertConvertsSameBuffers(tt, b1);
+    }
+
+    @Test
+    public void testDynamicComposite()
+    {
+        DynamicCompositeType tt = DynamicCompositeType.getInstance(DynamicCompositeTypeTest.aliases);
+        UUID[] uuids = DynamicCompositeTypeTest.uuids;
+        List<ByteBuffer> tests = ImmutableList.of
+            (
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test1", null, -1, false, true),
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test1", uuids[0], 24, false, true),
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test1", uuids[0], 42, false, true),
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test2", uuids[0], -1, false, true),
+            DynamicCompositeTypeTest.createDynamicCompositeKey("test2", uuids[1], 42, false, true)
+            );
+        for (ByteBuffer b : tests)
+            tt.validate(b);
+        testBuffers(tt, tests);
+    }
+
+    @Test
+    public void testListTypeString()
+    {
+        testCollection(ListType.getInstance(UTF8Type.instance, true), testStrings, () -> new ArrayList<>(), new Random());
+    }
+
+    @Test
+    public void testListTypeLong()
+    {
+        testCollection(ListType.getInstance(LongType.instance, true), testLongs, () -> new ArrayList<>(), new Random());
+    }
+
+    @Test
+    public void testSetTypeString()
+    {
+        testCollection(SetType.getInstance(UTF8Type.instance, true), testStrings, () -> new HashSet<>(), new Random());
+    }
+
+    @Test
+    public void testSetTypeLong()
+    {
+        testCollection(SetType.getInstance(LongType.instance, true), testLongs, () -> new HashSet<>(), new Random());
+    }
+
+    <T, CT extends Collection<T>> void testCollection(CollectionType<CT> tt, T[] values, Supplier<CT> gen, Random rand)
+    {
+        int cnt = 0;
+        List<CT> tests = new ArrayList<>();
+        tests.add(gen.get());
+        for (int c = 1; c <= 3; ++c)
+            for (int j = 0; j < 5; ++j)
+            {
+                CT l = gen.get();
+                for (int i = 0; i < c; ++i)
+                {
+                    T value = values[cnt++ % values.length];
+                    if (value != null)
+                        l.add(value);
+                }
+
+                tests.add(l);
+            }
+        testType(tt, tests);
+    }
+
+    @Test
+    public void testMapTypeStringLong()
+    {
+        testMap(MapType.getInstance(UTF8Type.instance, LongType.instance, true), testStrings, testLongs, () -> new HashMap<>(), new Random());
+    }
+
+    @Test
+    public void testMapTypeStringLongTree()
+    {
+        testMap(MapType.getInstance(UTF8Type.instance, LongType.instance, true), testStrings, testLongs, () -> new TreeMap<>(), new Random());
+    }
+
+    <K, V, M extends Map<K, V>> void testMap(MapType<K, V> tt, K[] keys, V[] values, Supplier<M> gen, Random rand)
+    {
+        List<M> tests = new ArrayList<>();
+        tests.add(gen.get());
+        for (int c = 1; c <= 3; ++c)
+            for (int j = 0; j < 5; ++j)
+            {
+                M l = gen.get();
+                for (int i = 0; i < c; ++i)
+                {
+                    V value = values[rand.nextInt(values.length)];
+                    if (value != null)
+                        l.put(keys[rand.nextInt(keys.length)], value);
+                }
+
+                tests.add(l);
+            }
+        testType(tt, tests);
+    }
+
+    /*
+     * Convert type to a comparable.
+     */
+    private ByteComparable typeToComparable(AbstractType<?> type, ByteBuffer value)
+    {
+        return new ByteComparable()
+        {
+            @Override
+            public ByteSource asComparableBytes(Version v)
+            {
+                return type.asComparableBytes(value, v);
+            }
+
+            @Override
+            public String toString()
+            {
+                return type.getString(value);
+            }
+        };
+    }
+
+    public <T> void testType(AbstractType<T> type, Object[] values)
+    {
+        testType(type, Iterables.transform(Arrays.asList(values), x -> (T) x));
+    }
+
+    public <T> void testType(AbstractType<? super T> type, Iterable<T> values)
+    {
+        for (T i : values) {
+            ByteBuffer b = decomposeAndRandomPad(type, i);
+            logger.info("Value {} ({}) bytes {} ByteSource {}",
+                              safeStr(i),
+                              safeStr(type.getSerializer().toCQLLiteral(b)),
+                              safeStr(ByteBufferUtil.bytesToHex(b)),
+                              typeToComparable(type, b).byteComparableAsString(VERSION));
+            assertConvertsSame(type, i);
+        }
+        if (!type.isReversed())
+            testType(ReversedType.getInstance(type), values);
+    }
+
+    public <T> void testTypeBuffers(AbstractType<T> type, Object[] values)
+    {
+        testTypeBuffers(type, Lists.transform(Arrays.asList(values), x -> (T) x));
+    }
+
+    public <T> void testTypeBuffers(AbstractType<T> type, List<T> values)
+    {
+        // Main difference with above is that we use type.compare instead of checking equals
+        testBuffers(type, Lists.transform(values, value -> decomposeAndRandomPad(type, value)));
+
+    }
+    public void testBuffers(AbstractType<?> type, List<ByteBuffer> values)
+    {
+        try
+        {
+            for (ByteBuffer b : values) {
+                logger.info("Value {} bytes {} ByteSource {}",
+                            safeStr(type.getSerializer().toCQLLiteral(b)),
+                            safeStr(ByteBufferUtil.bytesToHex(b)),
+                            typeToComparable(type, b).byteComparableAsString(VERSION));
+            }
+        }
+        catch (UnsupportedOperationException e)
+        {
+            // Continue without listing values.
+        }
+
+        for (ByteBuffer i : values)
+            assertConvertsSameBuffers(type, i);
+    }
+
+    void assertConvertsSameBuffers(AbstractType<?> type, ByteBuffer b1)
+    {
+        final ByteComparable bs1 = typeToComparable(type, b1);
+
+        ByteBuffer actual = type.fromComparableBytes(source(bs1), VERSION);
+        assertEquals(String.format("Failed compare(%s, converted %s (bytesource %s))",
+                                   ByteBufferUtil.bytesToHex(b1),
+                                   ByteBufferUtil.bytesToHex(actual),
+                                   bs1.byteComparableAsString(VERSION)),
+                     0,
+                     type.compare(b1, actual));
+    }
+
+    public void testDecoratedKeys(IPartitioner type, List<ByteBuffer> values)
+    {
+        for (ByteBuffer i : values)
+            assertConvertsSameDecoratedKeys(type, i);
+    }
+
+    void assertConvertsSameDecoratedKeys(IPartitioner type, ByteBuffer b1)
+    {
+        DecoratedKey k1 = type.decorateKey(b1);
+        DecoratedKey actual = BufferDecoratedKey.fromByteComparable(k1, VERSION, type);
+
+        assertEquals(String.format("Failed compare(%s[%s bs %s], %s[%s bs %s])\npartitioner %s",
+                                   k1,
+                                   ByteBufferUtil.bytesToHex(b1),
+                                   k1.byteComparableAsString(VERSION),
+                                   actual,
+                                   ByteBufferUtil.bytesToHex(actual.getKey()),
+                                   actual.byteComparableAsString(VERSION),
+                                   type),
+                     0,
+                     k1.compareTo(actual));
+        assertEquals(String.format("Failed equals(%s[%s bs %s], %s[%s bs %s])\npartitioner %s",
+                                   k1,
+                                   ByteBufferUtil.bytesToHex(b1),
+                                   k1.byteComparableAsString(VERSION),
+                                   actual,
+                                   ByteBufferUtil.bytesToHex(actual.getKey()),
+                                   actual.byteComparableAsString(VERSION),
+                                   type),
+                     k1,
+                     actual);
+    }
+
+    static Object safeStr(Object i)
+    {
+        if (i == null)
+            return null;
+        if (i instanceof ByteBuffer)
+        {
+            ByteBuffer buf = (ByteBuffer) i;
+            i = ByteBufferUtil.bytesToHex(buf);
+        }
+        String s = i.toString();
+        if (s.length() > 100)
+            s = s.substring(0, 100) + "...";
+        return s.replaceAll("\0", "<0>");
+    }
+
+    public <T> void testDirect(Function<T, ByteSource> convertor, Function<ByteSource.Peekable, T> inverse, T[] values)
+    {
+        for (T i : values) {
+            if (i == null)
+                continue;
+
+            logger.info("Value {} ByteSource {}\n",
+                              safeStr(i),
+                              convertor.apply(i));
+
+        }
+        for (T i : values)
+            if (i != null)
+                assertConvertsSame(convertor, inverse, i);
+    }
+
+    <T> void assertConvertsSame(Function<T, ByteSource> convertor, Function<ByteSource.Peekable, T> inverse, T v1)
+    {
+        ByteComparable b1 = v -> convertor.apply(v1);
+        T actual = inverse.apply(source(b1));
+        assertEquals(String.format("ByteSource %s", b1.byteComparableAsString(VERSION)), v1, actual);
+    }
+
+    <T> void assertConvertsSame(AbstractType<T> type, T v1)
+    {
+        ByteBuffer b1 = decomposeAndRandomPad(type, v1);
+        final ByteComparable bc1 = typeToComparable(type, b1);
+        ByteBuffer convertedBuffer = type.fromComparableBytes(source(bc1), VERSION);
+        T actual = type.compose(convertedBuffer);
+
+        assertEquals(String.format("Failed equals %s(%s bs %s), got %s",
+                                   safeStr(v1),
+                                   ByteBufferUtil.bytesToHex(b1),
+                                   safeStr(bc1.byteComparableAsString(VERSION)),
+                                   safeStr(actual)),
+                     v1,
+                     actual);
+    }
+
+    <T> ByteBuffer decomposeAndRandomPad(AbstractType<T> type, T v)
+    {
+        ByteBuffer b = type.decompose(v);
+        Random rand = new Random(0);
+        int padBefore = rand.nextInt(16);
+        int padAfter = rand.nextInt(16);
+        int paddedCapacity = b.remaining() + padBefore + padAfter;
+        ByteBuffer padded = allocateBuffer(paddedCapacity);
+        rand.ints(padBefore).forEach(x -> padded.put((byte) x));
+        padded.put(b.duplicate());
+        rand.ints(padAfter).forEach(x -> padded.put((byte) x));
+        padded.clear().limit(padded.capacity() - padAfter).position(padBefore);
+        return padded;
+    }
+
+    protected ByteBuffer allocateBuffer(int paddedCapacity)
+    {
+        return ByteBuffer.allocate(paddedCapacity);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/utils/bytecomparable/ByteSourceInverseTest.java b/test/unit/org/apache/cassandra/utils/bytecomparable/ByteSourceInverseTest.java
new file mode 100644
index 0000000000..391a8d383f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/bytecomparable/ByteSourceInverseTest.java
@@ -0,0 +1,397 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.utils.bytecomparable;
+
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.IntConsumer;
+import java.util.function.LongConsumer;
+import java.util.stream.*;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+@RunWith(Parameterized.class)
+public class ByteSourceInverseTest
+{
+    private static final String ALPHABET = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ1234567890!@#$%^&*()";
+
+    @Parameterized.Parameters(name = "version={0}")
+    public static Iterable<ByteComparable.Version> versions()
+    {
+        return ImmutableList.of(ByteComparable.Version.OSS42);
+    }
+
+    private final ByteComparable.Version version;
+
+    public ByteSourceInverseTest(ByteComparable.Version version)
+    {
+        this.version = version;
+    }
+
+    @Test
+    public void testGetSignedInt()
+    {
+        IntConsumer intConsumer = initial ->
+        {
+            ByteSource byteSource = ByteSource.of(initial);
+            int decoded = ByteSourceInverse.getSignedInt(byteSource);
+            Assert.assertEquals(initial, decoded);
+        };
+
+        IntStream.of(Integer.MIN_VALUE, Integer.MIN_VALUE + 1,
+                     -256, -255, -128, -127, -1, 0, 1, 127, 128, 255, 256,
+                     Integer.MAX_VALUE - 1, Integer.MAX_VALUE)
+                 .forEach(intConsumer);
+        new Random().ints(1000)
+                    .forEach(intConsumer);
+    }
+
+    @Test
+    public void testNextInt()
+    {
+        // The high and low 32 bits of this long differ only in the first and last bit (in the high 32 bits they are
+        // both 0s instead of 1s). The first bit difference will be negated by the bit flipping when writing down a
+        // fixed length signed number, so the only remaining difference will be in the last bit.
+        int hi = 0b0001_0010_0011_0100_0101_0110_0111_1000;
+        int lo = hi | 1 | 1 << 31;
+        long l1 = Integer.toUnsignedLong(hi) << 32 | Integer.toUnsignedLong(lo);
+
+        ByteSource byteSource = ByteSource.of(l1);
+        int i1 = ByteSourceInverse.getSignedInt(byteSource);
+        int i2 = ByteSourceInverse.getSignedInt(byteSource);
+        Assert.assertEquals(i1 + 1, i2);
+
+        try
+        {
+            ByteSourceInverse.getSignedInt(byteSource);
+            Assert.fail();
+        }
+        catch (IllegalArgumentException e)
+        {
+            // Expected.
+        }
+
+        byteSource = ByteSource.of(l1);
+        int iFirst = ByteSourceInverse.getSignedInt(byteSource);
+        Assert.assertEquals(i1, iFirst);
+        int iNext = ByteSourceInverse.getSignedInt(byteSource);
+        Assert.assertEquals(i2, iNext);
+    }
+
+    @Test
+    public void testGetSignedLong()
+    {
+        LongConsumer longConsumer = initial ->
+        {
+            ByteSource byteSource = ByteSource.of(initial);
+            long decoded = ByteSourceInverse.getSignedLong(byteSource);
+            Assert.assertEquals(initial, decoded);
+        };
+
+        LongStream.of(Long.MIN_VALUE, Long.MIN_VALUE + 1, Integer.MIN_VALUE - 1L,
+                      -256L, -255L, -128L, -127L, -1L, 0L, 1L, 127L, 128L, 255L, 256L,
+                      Integer.MAX_VALUE + 1L, Long.MAX_VALUE - 1, Long.MAX_VALUE)
+                  .forEach(longConsumer);
+        new Random().longs(1000)
+                    .forEach(longConsumer);
+    }
+
+    @Test
+    public void testGetSignedByte()
+    {
+        Consumer<Byte> byteConsumer = boxedByte ->
+        {
+            byte initial = boxedByte;
+            ByteBuffer byteBuffer = ByteType.instance.decompose(initial);
+            ByteSource byteSource = ByteType.instance.asComparableBytes(byteBuffer, version);
+            byte decoded = ByteSourceInverse.getSignedByte(byteSource);
+            Assert.assertEquals(initial, decoded);
+        };
+
+        IntStream.range(Byte.MIN_VALUE, Byte.MAX_VALUE + 1)
+                 .forEach(byteInteger -> byteConsumer.accept((byte) byteInteger));
+    }
+
+    @Test
+    public void testGetSignedShort()
+    {
+        Consumer<Short> shortConsumer = boxedShort ->
+        {
+            short initial = boxedShort;
+            ByteBuffer shortBuffer = ShortType.instance.decompose(initial);
+            ByteSource byteSource = ShortType.instance.asComparableBytes(shortBuffer, version);
+            short decoded = ByteSourceInverse.getSignedShort(byteSource);
+            Assert.assertEquals(initial, decoded);
+        };
+
+        IntStream.range(Short.MIN_VALUE, Short.MAX_VALUE + 1)
+                 .forEach(shortInteger -> shortConsumer.accept((short) shortInteger));
+    }
+
+    @Test
+    public void testBadByteSourceForFixedLengthNumbers()
+    {
+        byte[] bytes = new byte[8];
+        new Random().nextBytes(bytes);
+        for (Map.Entry<String, Integer> entries : ImmutableMap.of("getSignedInt", 4,
+                  "getSignedLong", 8,
+                  "getSignedByte", 1,
+                  "getSignedShort", 2).entrySet())
+        {
+            String methodName = entries.getKey();
+            int length = entries.getValue();
+            try
+            {
+                Method fixedLengthNumberMethod = ByteSourceInverse.class.getMethod(methodName, ByteSource.class);
+                ArrayList<ByteSource> sources = new ArrayList<>();
+                sources.add(null);
+                sources.add(ByteSource.EMPTY);
+                for (int i = 0; i < length; ++i)
+                    sources.add(ByteSource.fixedLength(bytes, 0, i));
+                // Note: not testing invalid bytes (e.g. using the construction below) as they signify a programming
+                // error (throwing AssertionError) rather than something that could happen due to e.g. bad files.
+                //      ByteSource.withTerminatorLegacy(257, ByteSource.fixedLength(bytes, 0, length - 1));
+                for (ByteSource badSource : sources)
+                {
+                    try
+                    {
+                        fixedLengthNumberMethod.invoke(ByteSourceInverse.class, badSource);
+                        Assert.fail("Expected exception not thrown");
+                    }
+                    catch (Throwable maybe)
+                    {
+                        maybe = Throwables.unwrapped(maybe);
+                        final String message = "Unexpected throwable " + maybe + " with cause " + maybe.getCause();
+                        if (badSource == null)
+                            Assert.assertTrue(message,
+                                              maybe instanceof NullPointerException);
+                        else
+                            Assert.assertTrue(message,
+                                              maybe instanceof IllegalArgumentException);
+                    }
+                }
+            }
+            catch (NoSuchMethodException e)
+            {
+                Assert.fail("Expected ByteSourceInverse to have method called " + methodName
+                            + " with a single parameter of type ByteSource");
+            }
+        }
+    }
... 1588 lines suppressed ...


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