You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bd...@apache.org on 2020/09/14 15:34:56 UTC

[cassandra] branch trunk updated: Add byte array backed cells

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

bdeggleston 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 ccab496  Add byte array backed cells
ccab496 is described below

commit ccab496d2d37c86341d364dea6c27513fda27331
Author: Blake Eggleston <bd...@gmail.com>
AuthorDate: Tue Oct 15 13:04:26 2019 -0700

    Add byte array backed cells
    
    Patch by Blake Eggleston; reviewed by Caleb Rackliffe and Marcus Eriksson for CASSANDRA-15393
---
 CHANGES.txt                                        |   1 +
 .../apache/cassandra/cache/CounterCacheKey.java    |  21 +-
 src/java/org/apache/cassandra/cql3/CQL3Type.java   |  17 +-
 src/java/org/apache/cassandra/cql3/Lists.java      |   7 +-
 src/java/org/apache/cassandra/cql3/Maps.java       |   2 +-
 src/java/org/apache/cassandra/cql3/Sets.java       |   2 +-
 src/java/org/apache/cassandra/cql3/Tuples.java     |   2 +-
 .../apache/cassandra/cql3/UntypedResultSet.java    |   8 +-
 .../apache/cassandra/cql3/UpdateParameters.java    |  14 +-
 .../cassandra/cql3/conditions/ColumnCondition.java |  58 +--
 .../cql3/conditions/ColumnConditions.java          |   2 +-
 .../cassandra/cql3/conditions/Conditions.java      |   2 +-
 .../cql3/conditions/IfExistsCondition.java         |   2 +-
 .../cql3/conditions/IfNotExistsCondition.java      |   2 +-
 .../restrictions/ClusteringColumnRestrictions.java |   4 +-
 .../cql3/restrictions/StatementRestrictions.java   |   4 +-
 .../cassandra/cql3/selection/ResultSetBuilder.java |  12 +-
 .../cassandra/cql3/statements/BatchStatement.java  |   2 +-
 .../cassandra/cql3/statements/CQL3CasRequest.java  |  30 +-
 .../cassandra/cql3/statements/DeleteStatement.java |   2 +-
 .../cql3/statements/ModificationStatement.java     |  45 ++-
 .../cassandra/cql3/statements/SelectStatement.java |  24 +-
 .../cassandra/cql3/statements/UpdateStatement.java |   2 +-
 .../AbstractArrayClusteringPrefix.java}            |  36 +-
 .../db/AbstractBufferClusteringPrefix.java         |  46 +--
 .../cassandra/db/AbstractClusteringPrefix.java     |  77 ----
 ...ix.java => AbstractOnHeapClusteringPrefix.java} |  28 +-
 .../cassandra/db/AbstractReadCommandBuilder.java   |  41 +-
 .../db/{Clusterable.java => ArrayClustering.java}  |  31 +-
 .../apache/cassandra/db/ArrayClusteringBound.java  |  58 +++
 ...le.java => ArrayClusteringBoundOrBoundary.java} |  22 +-
 .../cassandra/db/ArrayClusteringBoundary.java      |  60 +++
 .../org/apache/cassandra/db/BufferClustering.java  |  23 +-
 .../apache/cassandra/db/BufferClusteringBound.java |  79 ++++
 ...e.java => BufferClusteringBoundOrBoundary.java} |  18 +-
 .../cassandra/db/BufferClusteringBoundary.java     |  70 ++++
 src/java/org/apache/cassandra/db/CBuilder.java     |  94 ++---
 src/java/org/apache/cassandra/db/Clusterable.java  |   4 +-
 src/java/org/apache/cassandra/db/Clustering.java   |  36 +-
 .../org/apache/cassandra/db/ClusteringBound.java   | 154 +++-----
 .../cassandra/db/ClusteringBoundOrBoundary.java    |  87 ++---
 .../apache/cassandra/db/ClusteringBoundary.java    |  45 +--
 .../apache/cassandra/db/ClusteringComparator.java  |  40 +-
 .../org/apache/cassandra/db/ClusteringPrefix.java  | 184 +++++++--
 .../org/apache/cassandra/db/ColumnFamilyStore.java |   4 +-
 src/java/org/apache/cassandra/db/ColumnIndex.java  |   4 +-
 .../org/apache/cassandra/db/CounterMutation.java   |  11 +-
 src/java/org/apache/cassandra/db/DataRange.java    |  11 +-
 src/java/org/apache/cassandra/db/DeletionInfo.java |   2 +-
 src/java/org/apache/cassandra/db/DeletionTime.java |   2 +-
 src/java/org/apache/cassandra/db/Digest.java       |  25 +-
 .../org/apache/cassandra/db/MultiCBuilder.java     |  46 +--
 .../apache/cassandra/db/MutableDeletionInfo.java   |   6 +-
 .../org/apache/cassandra/db/NativeClustering.java  |  38 +-
 .../cassandra/db/PartitionRangeReadQuery.java      |   2 +-
 .../apache/cassandra/db/RangeTombstoneList.java    |  66 ++--
 src/java/org/apache/cassandra/db/ReadCommand.java  |   4 +-
 src/java/org/apache/cassandra/db/ReadQuery.java    |   4 +-
 .../org/apache/cassandra/db/RowIndexEntry.java     |   6 +-
 .../org/apache/cassandra/db/SimpleBuilders.java    |  14 +-
 .../cassandra/db/SinglePartitionReadCommand.java   |  20 +-
 .../cassandra/db/SinglePartitionReadQuery.java     |   6 +-
 src/java/org/apache/cassandra/db/Slice.java        |  64 ++--
 src/java/org/apache/cassandra/db/Slices.java       |  44 +--
 src/java/org/apache/cassandra/db/TypeSizes.java    |  32 +-
 .../cassandra/db/UnfilteredDeserializer.java       |   5 +-
 .../db/VirtualTableSinglePartitionReadQuery.java   |   2 +-
 .../cassandra/db/aggregation/GroupMaker.java       |   8 +-
 .../cassandra/db/aggregation/GroupingState.java    |   9 +-
 .../db/columniterator/AbstractSSTableIterator.java |   8 +-
 .../db/columniterator/SSTableIterator.java         |   8 +-
 .../db/columniterator/SSTableReversedIterator.java |   8 +-
 .../apache/cassandra/db/compaction/Scrubber.java   |   8 +-
 .../cassandra/db/context/CounterContext.java       |  71 ++--
 .../cassandra/db/filter/ClusteringIndexFilter.java |   4 +-
 .../db/filter/ClusteringIndexNamesFilter.java      |  36 +-
 .../db/filter/ClusteringIndexSliceFilter.java      |   4 +-
 .../apache/cassandra/db/filter/ColumnFilter.java   |   2 +-
 .../org/apache/cassandra/db/filter/RowFilter.java  |  22 +-
 .../db/filter/TombstoneOverwhelmingException.java  |   6 +-
 .../db/marshal/AbstractCompositeType.java          | 139 ++++---
 .../apache/cassandra/db/marshal/AbstractType.java  | 146 ++++---
 .../apache/cassandra/db/marshal/BooleanType.java   |  14 +-
 .../cassandra/db/marshal/ByteArrayAccessor.java    | 294 +++++++++++++++
 .../db/marshal/ByteArrayObjectFactory.java         |  82 ++++
 .../cassandra/db/marshal/ByteBufferAccessor.java   | 292 ++++++++++++++
 .../db/marshal/ByteBufferObjectFactory.java        |  86 +++++
 .../org/apache/cassandra/db/marshal/ByteType.java  |   4 +-
 .../cassandra/db/marshal/CollectionType.java       |  16 +-
 .../apache/cassandra/db/marshal/CompositeType.java |  96 +++--
 .../cassandra/db/marshal/CounterColumnType.java    |  13 +-
 .../apache/cassandra/db/marshal/DecimalType.java   |   7 +-
 .../apache/cassandra/db/marshal/DoubleType.java    |   7 +-
 .../cassandra/db/marshal/DynamicCompositeType.java |  68 ++--
 .../org/apache/cassandra/db/marshal/EmptyType.java |  13 +-
 .../org/apache/cassandra/db/marshal/FloatType.java |   7 +-
 .../apache/cassandra/db/marshal/FrozenType.java    |   2 +-
 .../org/apache/cassandra/db/marshal/Int32Type.java |  10 +-
 .../apache/cassandra/db/marshal/IntegerType.java   |  30 +-
 .../cassandra/db/marshal/LexicalUUIDType.java      |  10 +-
 .../org/apache/cassandra/db/marshal/ListType.java  |  46 +--
 .../org/apache/cassandra/db/marshal/LongType.java  |  14 +-
 .../org/apache/cassandra/db/marshal/MapType.java   |  63 ++--
 .../db/marshal/PartitionerDefinedOrder.java        |  13 +-
 .../apache/cassandra/db/marshal/ReversedType.java  |  12 +-
 .../org/apache/cassandra/db/marshal/SetType.java   |  13 +-
 .../org/apache/cassandra/db/marshal/ShortType.java |   7 +-
 .../apache/cassandra/db/marshal/TimeUUIDType.java  |  21 +-
 .../apache/cassandra/db/marshal/TimestampType.java |   4 +-
 .../org/apache/cassandra/db/marshal/TupleType.java |  95 +++--
 .../org/apache/cassandra/db/marshal/UTF8Type.java  |   1 -
 .../org/apache/cassandra/db/marshal/UUIDType.java  |  20 +-
 .../org/apache/cassandra/db/marshal/UserType.java  |  16 +-
 .../apache/cassandra/db/marshal/ValueAccessor.java | 418 +++++++++++++++++++++
 .../marshal/ValueComparators.java}                 |  29 +-
 .../db/partitions/AbstractBTreePartition.java      |  12 +-
 .../db/partitions/AtomicBTreePartition.java        |   6 +-
 .../db/partitions/CachedBTreePartition.java        |   2 +-
 .../apache/cassandra/db/partitions/Partition.java  |   4 +-
 .../partitions/PartitionStatisticsCollector.java   |   2 +-
 .../cassandra/db/partitions/PartitionUpdate.java   |  16 +-
 .../org/apache/cassandra/db/rows/AbstractCell.java |  57 +--
 .../db/rows/AbstractRangeTombstoneMarker.java      |   8 +-
 .../org/apache/cassandra/db/rows/AbstractRow.java  |  27 +-
 .../org/apache/cassandra/db/rows/ArrayCell.java    | 111 ++++++
 .../org/apache/cassandra/db/rows/BTreeRow.java     |  82 ++--
 .../org/apache/cassandra/db/rows/BufferCell.java   |  17 +-
 src/java/org/apache/cassandra/db/rows/Cell.java    |  64 ++--
 src/java/org/apache/cassandra/db/rows/Cells.java   |  73 ++--
 .../cassandra/db/rows/ComplexColumnData.java       |  41 +-
 .../org/apache/cassandra/db/rows/CounterCells.java |   4 +-
 .../cassandra/db/rows/DeserializationHelper.java   |  13 +-
 .../apache/cassandra/db/rows/EncodingStats.java    |   2 +-
 .../org/apache/cassandra/db/rows/NativeCell.java   |  21 +-
 .../db/rows/RangeTombstoneBoundMarker.java         |  46 ++-
 .../db/rows/RangeTombstoneBoundaryMarker.java      |  58 ++-
 .../cassandra/db/rows/RangeTombstoneMarker.java    |  22 +-
 src/java/org/apache/cassandra/db/rows/Row.java     |  40 +-
 .../db/rows/RowAndDeletionMergeIterator.java       |   4 +-
 .../apache/cassandra/db/rows/RowDiffListener.java  |   8 +-
 src/java/org/apache/cassandra/db/rows/Rows.java    |  36 +-
 .../db/rows/ThrottledUnfilteredIterator.java       |   5 +-
 .../org/apache/cassandra/db/rows/Unfiltered.java   |  10 +-
 .../rows/UnfilteredRowIteratorWithLowerBound.java  |  22 +-
 .../cassandra/db/rows/UnfilteredSerializer.java    |  25 +-
 .../db/transform/DuplicateRowChecker.java          |   2 +-
 .../cassandra/db/transform/RTBoundCloser.java      |   4 +-
 .../org/apache/cassandra/db/view/TableViews.java   |   6 +-
 .../cassandra/db/view/ViewUpdateGenerator.java     |  41 +-
 .../apache/cassandra/db/virtual/SimpleDataSet.java |  13 +-
 .../org/apache/cassandra/gms/TokenSerializer.java  |   4 +-
 .../cassandra/hadoop/cql3/CqlRecordWriter.java     |   3 +-
 .../cassandra/index/SecondaryIndexManager.java     |  23 +-
 .../cassandra/index/internal/CassandraIndex.java   |  52 +--
 .../index/internal/CassandraIndexSearcher.java     |  18 +-
 .../cassandra/index/internal/IndexEntry.java       |   8 +-
 .../internal/composites/ClusteringColumnIndex.java |  22 +-
 .../internal/composites/CollectionEntryIndex.java  |   9 +-
 .../internal/composites/CollectionKeyIndex.java    |   4 +-
 .../composites/CollectionKeyIndexBase.java         |  16 +-
 .../internal/composites/CollectionValueIndex.java  |  22 +-
 .../internal/composites/CompositesSearcher.java    |   8 +-
 .../internal/composites/PartitionKeyIndex.java     |  16 +-
 .../internal/composites/RegularColumnIndex.java    |  33 +-
 .../cassandra/index/internal/keys/KeysIndex.java   |  20 +-
 .../index/internal/keys/KeysSearcher.java          |   2 +-
 .../cassandra/index/sasi/conf/ColumnIndex.java     |   6 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java     |   4 +-
 .../org/apache/cassandra/io/sstable/IndexInfo.java |  13 +-
 .../io/sstable/metadata/CompactionMetadata.java    |   3 +-
 .../io/sstable/metadata/MetadataCollector.java     |  12 +-
 .../apache/cassandra/schema/ColumnMetadata.java    |  12 +-
 .../serializers/AbstractTextSerializer.java        |  16 +-
 .../cassandra/serializers/AsciiSerializer.java     |   9 +-
 .../cassandra/serializers/BooleanSerializer.java   |  22 +-
 .../cassandra/serializers/ByteSerializer.java      |  17 +-
 .../cassandra/serializers/BytesSerializer.java     |  13 +-
 .../serializers/CollectionSerializer.java          |  81 ++--
 .../cassandra/serializers/DecimalSerializer.java   |  21 +-
 .../cassandra/serializers/DoubleSerializer.java    |  19 +-
 .../cassandra/serializers/DurationSerializer.java  |  21 +-
 .../cassandra/serializers/EmptySerializer.java     |  15 +-
 .../cassandra/serializers/FloatSerializer.java     |  19 +-
 .../serializers/InetAddressSerializer.java         |  17 +-
 .../cassandra/serializers/Int32Serializer.java     |  17 +-
 .../cassandra/serializers/IntegerSerializer.java   |   9 +-
 .../cassandra/serializers/ListSerializer.java      |  67 ++--
 .../cassandra/serializers/LongSerializer.java      |  17 +-
 .../cassandra/serializers/MapSerializer.java       |  84 +++--
 .../cassandra/serializers/SetSerializer.java       |  70 ++--
 .../cassandra/serializers/ShortSerializer.java     |  17 +-
 .../serializers/SimpleDateSerializer.java          |  13 +-
 .../cassandra/serializers/TimeSerializer.java      |  13 +-
 .../cassandra/serializers/TimeUUIDSerializer.java  |  14 +-
 .../cassandra/serializers/TimestampSerializer.java |  17 +-
 .../cassandra/serializers/TupleSerializer.java     |  25 +-
 .../cassandra/serializers/TypeSerializer.java      |  36 +-
 .../cassandra/serializers/UTF8Serializer.java      |  17 +-
 .../cassandra/serializers/UUIDSerializer.java      |  14 +-
 .../cassandra/serializers/UserTypeSerializer.java  |  26 +-
 .../org/apache/cassandra/service/CacheService.java |   5 +-
 .../service/pager/AbstractQueryPager.java          |   2 +-
 .../service/pager/AggregationQueryPager.java       |   6 +-
 .../cassandra/service/pager/PagingState.java       |  24 +-
 .../service/pager/SinglePartitionPager.java        |   2 +-
 .../service/reads/ReplicaFilteringProtection.java  |   4 +-
 .../service/reads/ShortReadRowsProtection.java     |   4 +-
 .../reads/repair/RowIteratorMergeListener.java     |  20 +-
 .../apache/cassandra/tools/JsonTransformer.java    |  14 +-
 .../cassandra/tools/SSTableMetadataViewer.java     |   2 +-
 .../org/apache/cassandra/utils/ByteArrayUtil.java  | 267 +++++++------
 .../org/apache/cassandra/utils/ByteBufferUtil.java |  17 +-
 .../apache/cassandra/utils/FastByteOperations.java |  42 +++
 .../cassandra/utils/NativeSSTableLoaderClient.java |   2 -
 .../cassandra/utils/memory/AbstractAllocator.java  |  29 +-
 .../cassandra/utils/memory/EnsureOnHeap.java       |  10 +-
 .../cassandra/utils/memory/NativeAllocator.java    |   4 +-
 .../org/apache/cassandra/distributed/Cluster.java  |   1 -
 .../db/commitlog/CommitLogStressTest.java          |   6 +-
 .../cassandra/hints/HintsWriteThenReadTest.java    |   2 +-
 test/unit/org/apache/cassandra/Util.java           |  22 +-
 .../apache/cassandra/cql3/CQL3TypeLiteralTest.java |  20 +-
 test/unit/org/apache/cassandra/cql3/ListsTest.java |   6 +-
 .../cql3/TombstonesWithIndexedSSTableTest.java     |   4 +-
 .../org/apache/cassandra/cql3/UDHelperTest.java    |   5 +-
 .../ClusteringColumnRestrictionsTest.java          |  64 ++--
 .../validation/entities/SecondaryIndexTest.java    |   8 +-
 .../cql3/validation/entities/VirtualTableTest.java |   2 +-
 test/unit/org/apache/cassandra/db/CellTest.java    |  54 +--
 .../apache/cassandra/db/ClusteringBoundTest.java   |  28 +-
 .../apache/cassandra/db/ColumnFamilyStoreTest.java |   4 +-
 .../org/apache/cassandra/db/CounterCacheTest.java  |  12 +-
 .../org/apache/cassandra/db/CounterCellTest.java   |  65 ++--
 .../apache/cassandra/db/CounterMutationTest.java   |  20 +-
 .../unit/org/apache/cassandra/db/KeyspaceTest.java |  40 +-
 .../unit/org/apache/cassandra/db/NameSortTest.java |   2 +-
 .../org/apache/cassandra/db/NativeCellTest.java    |   4 +-
 .../cassandra/db/PartitionRangeReadTest.java       |  20 +-
 .../org/apache/cassandra/db/PartitionTest.java     |   2 +-
 .../cassandra/db/RangeTombstoneListTest.java       |  16 +-
 .../apache/cassandra/db/RangeTombstoneTest.java    |   2 +-
 .../org/apache/cassandra/db/ReadCommandTest.java   |   8 +-
 .../apache/cassandra/db/RecoveryManagerTest.java   |   2 +-
 .../apache/cassandra/db/RepairedDataInfoTest.java  |   6 +-
 .../cassandra/db/RepairedDataTombstonesTest.java   |   4 +-
 .../unit/org/apache/cassandra/db/RowCacheTest.java |  25 +-
 .../org/apache/cassandra/db/RowIndexEntryTest.java |  20 +-
 test/unit/org/apache/cassandra/db/RowTest.java     |  18 +-
 .../apache/cassandra/db/SecondaryIndexTest.java    |   2 +-
 .../cassandra/db/SerializationHeaderTest.java      |  10 +-
 .../db/SinglePartitionSliceCommandTest.java        |  17 +-
 .../org/apache/cassandra/db/TransformerTest.java   |   4 +-
 .../cassandra/db/aggregation/GroupMakerTest.java   |   2 +-
 .../db/commitlog/CommitLogReaderTest.java          |   2 +-
 .../db/commitlog/CommitLogUpgradeTest.java         |   6 +-
 .../db/compaction/CompactionsCQLTest.java          |   2 +-
 .../cassandra/db/compaction/CompactionsTest.java   |  13 +-
 .../cassandra/db/compaction/NeverPurgeTest.java    |   2 +-
 .../apache/cassandra/db/composites/CTypeTest.java  |  17 +-
 .../cassandra/db/context/CounterContextTest.java   |  35 +-
 .../org/apache/cassandra/db/filter/SliceTest.java  |   4 +-
 .../db/marshal/ByteBufferAccessorTest.java         |  85 +++++
 .../cassandra/db/marshal/CollectionTypesTest.java  | 124 ++++++
 .../db/marshal/CompositeAndTupleTypesTest.java     | 138 +++++++
 .../cassandra/db/marshal/CompositeTypeTest.java    |   6 +-
 .../db/marshal/DynamicCompositeTypeTest.java       |   4 +-
 .../apache/cassandra/db/marshal/EmptyTypeTest.java |   4 +-
 .../cassandra/db/marshal/ValueAccessorTest.java    | 132 +++++++
 .../cassandra/db/marshal/ValueAccessors.java       |  62 +++
 .../cassandra/db/marshal/ValueGenerator.java       |  95 +++++
 .../db/partition/PartitionImplementationTest.java  |  49 ++-
 .../cassandra/db/partitions/PurgeFunctionTest.java |   4 +-
 .../db/rows/RowAndDeletionMergeIteratorTest.java   |  36 +-
 .../org/apache/cassandra/db/rows/RowBuilder.java   |  10 +-
 .../org/apache/cassandra/db/rows/RowsTest.java     |  88 ++---
 .../db/rows/ThrottledUnfilteredIteratorTest.java   |  22 +-
 .../db/rows/UnfilteredRowIteratorsMergeTest.java   |  24 +-
 .../db/rows/UnfilteredRowIteratorsTest.java        |   6 +-
 .../cassandra/db/rows/UnfilteredRowsGenerator.java |  20 +-
 .../db/transform/RTTransformationsTest.java        |   4 +-
 .../apache/cassandra/hints/HintsBufferTest.java    |   7 +-
 .../apache/cassandra/hints/HintsReaderTest.java    |   7 +-
 .../apache/cassandra/index/CustomIndexTest.java    |   2 +-
 .../index/internal/CustomCassandraIndex.java       |  54 +--
 .../apache/cassandra/index/sasi/SASIIndexTest.java |  22 +-
 .../cassandra/index/sasi/plan/OperationTest.java   |  14 +-
 .../io/sstable/IndexSummaryManagerTest.java        |   4 +-
 .../cassandra/io/sstable/SSTableLoaderTest.java    |   4 +-
 .../cassandra/io/sstable/SSTableMetadataTest.java  |   3 +-
 .../cassandra/io/sstable/SSTableReaderTest.java    |   2 +-
 .../sstable/format/SSTableFlushObserverTest.java   |   8 +-
 .../apache/cassandra/io/util/DataOutputTest.java   |  17 +-
 .../apache/cassandra/service/PaxosStateTest.java   |   2 +-
 .../apache/cassandra/service/QueryPagerTest.java   |   8 +-
 .../service/reads/AbstractReadResponseTest.java    |  12 +-
 .../cassandra/service/reads/DataResolverTest.java  |  24 +-
 .../reads/repair/AbstractReadRepairTest.java       |  18 +-
 .../service/reads/repair/ReadRepairTest.java       |  12 +-
 .../cassandra/streaming/StreamingTransferTest.java |   4 +-
 .../cassandra/triggers/TriggerExecutorTest.java    |   2 +-
 .../apache/cassandra/utils/ByteArrayUtilTest.java  |  18 +-
 .../cassandra/utils/CassandraGenerators.java       |   3 +-
 .../io/sstable/StressCQLSSTableWriter.java         |   4 +-
 303 files changed, 5504 insertions(+), 2880 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 90238ef..fa28101 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-beta3
+ * Add byte array backed cells (CASSANDRA-15393)
  * Correctly handle pending ranges with adjacent range movements (CASSANDRA-14801)
  * Avoid adding locahost when streaming trivial ranges (CASSANDRA-16099)
  * Add nodetool getfullquerylog (CASSANDRA-15988)
diff --git a/src/java/org/apache/cassandra/cache/CounterCacheKey.java b/src/java/org/apache/cassandra/cache/CounterCacheKey.java
index dc3ce4e..1f85263 100644
--- a/src/java/org/apache/cassandra/cache/CounterCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/CounterCacheKey.java
@@ -27,6 +27,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
 import org.apache.cassandra.db.filter.ClusteringIndexNamesFilter;
 import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.rows.CellPath;
@@ -61,22 +62,22 @@ public final class CounterCacheKey extends CacheKey
         this(tableMetadata, ByteBufferUtil.getArray(partitionKey), ByteBufferUtil.getArray(cellName));
     }
 
-    public static CounterCacheKey create(TableMetadata tableMetadata, ByteBuffer partitionKey, Clustering clustering, ColumnMetadata c, CellPath path)
+    public static CounterCacheKey create(TableMetadata tableMetadata, ByteBuffer partitionKey, Clustering<?> clustering, ColumnMetadata c, CellPath path)
     {
         return new CounterCacheKey(tableMetadata, partitionKey, makeCellName(clustering, c, path));
     }
 
-    private static ByteBuffer makeCellName(Clustering clustering, ColumnMetadata c, CellPath path)
+    private static ByteBuffer makeCellName(Clustering<?> clustering, ColumnMetadata c, CellPath path)
     {
         int cs = clustering.size();
         ByteBuffer[] values = new ByteBuffer[cs + 1 + (path == null ? 0 : path.size())];
         for (int i = 0; i < cs; i++)
-            values[i] = clustering.get(i);
+            values[i] = clustering.bufferAt(i);
         values[cs] = c.name.bytes;
         if (path != null)
             for (int i = 0; i < path.size(); i++)
                 values[cs + 1 + i] = path.get(i);
-        return CompositeType.build(values);
+        return CompositeType.build(ByteBufferAccessor.instance, values);
     }
 
     public ByteBuffer partitionKey()
@@ -99,10 +100,10 @@ public final class CounterCacheKey extends CacheKey
         DecoratedKey key = cfs.decorateKey(partitionKey());
 
         int clusteringSize = metadata.comparator.size();
-        List<ByteBuffer> buffers = CompositeType.splitName(ByteBuffer.wrap(cellName));
+        List<ByteBuffer> buffers = CompositeType.splitName(ByteBuffer.wrap(cellName), ByteBufferAccessor.instance);
         assert buffers.size() >= clusteringSize + 1; // See makeCellName above
 
-        Clustering clustering = Clustering.make(buffers.subList(0, clusteringSize).toArray(new ByteBuffer[clusteringSize]));
+        Clustering<?> clustering = Clustering.make(buffers.subList(0, clusteringSize).toArray(new ByteBuffer[clusteringSize]));
         ColumnMetadata column = metadata.getColumn(buffers.get(clusteringSize));
         // This can theoretically happen if a column is dropped after the cache is saved and we
         // try to load it. Not point if failing in any case, just skip the value.
@@ -125,9 +126,9 @@ public final class CounterCacheKey extends CacheKey
         {
             ByteBuffer value = null;
             if (column.isStatic())
-                value = iter.staticRow().getCell(column).value();
+                value = iter.staticRow().getCell(column).buffer();
             else if (iter.hasNext())
-                value = iter.next().getCell(column).value();
+                value = iter.next().getCell(column).buffer();
 
             return value;
         }
@@ -136,8 +137,8 @@ public final class CounterCacheKey extends CacheKey
     public void write(DataOutputPlus out)
     throws IOException
     {
-        ByteBufferUtil.writeWithLength(partitionKey, out);
-        ByteBufferUtil.writeWithLength(cellName, out);
+        ByteArrayUtil.writeWithLength(partitionKey, out);
+        ByteArrayUtil.writeWithLength(cellName, out);
     }
 
     public static CounterCacheKey read(TableMetadata tableMetadata, DataInputPlus in)
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java
index 84aab07..5059104 100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Type.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.db.marshal.CollectionType.Kind;
@@ -57,11 +58,11 @@ public interface CQL3Type
 
     /**
      * Generates CQL literal from a binary value of this type.
-     *  @param buffer the value to convert to a CQL literal. This value must be
+     *  @param bytes the value to convert to a CQL literal. This value must be
      * serialized with {@code version} of the native protocol.
      * @param version the native protocol version in which {@code buffer} is encoded.
      */
-    public String toCQLLiteral(ByteBuffer buffer, ProtocolVersion version);
+    String toCQLLiteral(ByteBuffer bytes, ProtocolVersion version);
 
     public enum Native implements CQL3Type
     {
@@ -196,6 +197,7 @@ public interface CQL3Type
             StringBuilder target = new StringBuilder();
             buffer = buffer.duplicate();
             int size = CollectionSerializer.readCollectionSize(buffer, version);
+            buffer.position(buffer.position() + CollectionSerializer.sizeOfCollectionSize(size, version));
 
             switch (type.kind)
             {
@@ -224,25 +226,30 @@ public interface CQL3Type
         {
             CQL3Type keys = ((MapType) type).getKeysType().asCQL3Type();
             CQL3Type values = ((MapType) type).getValuesType().asCQL3Type();
+            int offset = 0;
             for (int i = 0; i < size; i++)
             {
                 if (i > 0)
                     target.append(", ");
-                ByteBuffer element = CollectionSerializer.readValue(buffer, version);
+                ByteBuffer element = CollectionSerializer.readValue(buffer, ByteBufferAccessor.instance, offset, version);
+                offset += CollectionSerializer.sizeOfValue(element, ByteBufferAccessor.instance, version);
                 target.append(keys.toCQLLiteral(element, version));
                 target.append(": ");
-                element = CollectionSerializer.readValue(buffer, version);
+                element = CollectionSerializer.readValue(buffer, ByteBufferAccessor.instance, offset, version);
+                offset += CollectionSerializer.sizeOfValue(element, ByteBufferAccessor.instance, version);
                 target.append(values.toCQLLiteral(element, version));
             }
         }
 
         private static void generateSetOrListCQLLiteral(ByteBuffer buffer, ProtocolVersion version, StringBuilder target, int size, CQL3Type elements)
         {
+            int offset = 0;
             for (int i = 0; i < size; i++)
             {
                 if (i > 0)
                     target.append(", ");
-                ByteBuffer element = CollectionSerializer.readValue(buffer, version);
+                ByteBuffer element = CollectionSerializer.readValue(buffer, ByteBufferAccessor.instance, offset, version);
+                offset += CollectionSerializer.sizeOfValue(element, ByteBufferAccessor.instance, version);
                 target.append(elements.toCQLLiteral(element, version));
             }
         }
diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java
index 14961c9..1d94d69 100644
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.schema.ColumnMetadata;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.cassandra.cql3.functions.Function;
@@ -210,7 +211,7 @@ public abstract class Lists
             {
                 // Collections have this small hack that validate cannot be called on a serialized object,
                 // but compose does the validation (so we're fine).
-                List<?> l = type.getSerializer().deserializeForNativeProtocol(value, version);
+                List<?> l = type.getSerializer().deserializeForNativeProtocol(value, ByteBufferAccessor.instance, version);
                 List<ByteBuffer> elements = new ArrayList<>(l.size());
                 for (Object element : l)
                     // elements can be null in lists that represent a set of IN values
@@ -576,9 +577,9 @@ public abstract class Lists
             // the read-before-write this operation requires limits its usefulness on big lists, so in practice
             // toDiscard will be small and keeping a list will be more efficient.
             List<ByteBuffer> toDiscard = ((Value)value).elements;
-            for (Cell cell : complexData)
+            for (Cell<?> cell : complexData)
             {
-                if (toDiscard.contains(cell.value()))
+                if (toDiscard.contains(cell.buffer()))
                     params.addTombstone(column, cell.path());
             }
         }
diff --git a/src/java/org/apache/cassandra/cql3/Maps.java b/src/java/org/apache/cassandra/cql3/Maps.java
index 4ae98f5..6e7e07b 100644
--- a/src/java/org/apache/cassandra/cql3/Maps.java
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@ -228,7 +228,7 @@ public abstract class Maps
             {
                 // Collections have this small hack that validate cannot be called on a serialized object,
                 // but compose does the validation (so we're fine).
-                Map<?, ?> m = type.getSerializer().deserializeForNativeProtocol(value, version);
+                Map<?, ?> m = type.getSerializer().deserializeForNativeProtocol(value, ByteBufferAccessor.instance, version);
                 Map<ByteBuffer, ByteBuffer> map = new LinkedHashMap<>(m.size());
                 for (Map.Entry<?, ?> entry : m.entrySet())
                     map.put(type.getKeysType().decompose(entry.getKey()), type.getValuesType().decompose(entry.getValue()));
diff --git a/src/java/org/apache/cassandra/cql3/Sets.java b/src/java/org/apache/cassandra/cql3/Sets.java
index 7084477..aab4192 100644
--- a/src/java/org/apache/cassandra/cql3/Sets.java
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@ -214,7 +214,7 @@ public abstract class Sets
             {
                 // Collections have this small hack that validate cannot be called on a serialized object,
                 // but compose does the validation (so we're fine).
-                Set<?> s = type.getSerializer().deserializeForNativeProtocol(value, version);
+                Set<?> s = type.getSerializer().deserializeForNativeProtocol(value, ByteBufferAccessor.instance, version);
                 SortedSet<ByteBuffer> elements = new TreeSet<>(type.getElementsType());
                 for (Object element : s)
                     elements.add(type.getElementsType().decompose(element));
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index 317e192..b8acd59 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -264,7 +264,7 @@ public class Tuples
             {
                 // Collections have this small hack that validate cannot be called on a serialized object,
                 // but the deserialization does the validation (so we're fine).
-                List<?> l = type.getSerializer().deserializeForNativeProtocol(value, options.getProtocolVersion());
+                List<?> l = type.getSerializer().deserializeForNativeProtocol(value, ByteBufferAccessor.instance, options.getProtocolVersion());
 
                 assert type.getElementsType() instanceof TupleType;
                 TupleType tupleType = Tuples.getTupleType(type.getElementsType());
diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
index 5de4eae..f4ac99f 100644
--- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
@@ -310,17 +310,17 @@ public abstract class UntypedResultSet implements Iterable<UntypedResultSet.Row>
             for (ColumnMetadata def : metadata.partitionKeyColumns())
                 data.put(def.name.toString(), keyComponents[def.position()]);
 
-            Clustering clustering = row.clustering();
+            Clustering<?> clustering = row.clustering();
             for (ColumnMetadata def : metadata.clusteringColumns())
-                data.put(def.name.toString(), clustering.get(def.position()));
+                data.put(def.name.toString(), clustering.bufferAt(def.position()));
 
             for (ColumnMetadata def : metadata.regularAndStaticColumns())
             {
                 if (def.isSimple())
                 {
-                    Cell cell = row.getCell(def);
+                    Cell<?> cell = row.getCell(def);
                     if (cell != null)
-                        data.put(def.name.toString(), cell.value());
+                        data.put(def.name.toString(), cell.buffer());
                 }
                 else
                 {
diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
index 5579e10..f90958b 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -80,7 +80,7 @@ public class UpdateParameters
             throw new InvalidRequestException(String.format("Out of bound timestamp, must be in [%d, %d]", Long.MIN_VALUE + 1, Long.MAX_VALUE));
     }
 
-    public void newRow(Clustering clustering) throws InvalidRequestException
+    public <V> void newRow(Clustering<V> clustering) throws InvalidRequestException
     {
         if (clustering == Clustering.STATIC_CLUSTERING)
         {
@@ -98,7 +98,7 @@ public class UpdateParameters
         builder.newRow(clustering);
     }
 
-    public Clustering currentClustering()
+    public Clustering<?> currentClustering()
     {
         return builder.clustering();
     }
@@ -130,9 +130,9 @@ public class UpdateParameters
 
     public void addCell(ColumnMetadata column, CellPath path, ByteBuffer value) throws InvalidRequestException
     {
-        Cell cell = ttl == LivenessInfo.NO_TTL
-                  ? BufferCell.live(column, timestamp, value, path)
-                  : BufferCell.expiring(column, timestamp, ttl, nowInSec, value, path);
+        Cell<?> cell = ttl == LivenessInfo.NO_TTL
+                       ? BufferCell.live(column, timestamp, value, path)
+                       : BufferCell.expiring(column, timestamp, ttl, nowInSec, value, path);
         builder.addCell(cell);
     }
 
@@ -177,7 +177,7 @@ public class UpdateParameters
         return deletionTime;
     }
 
-    public RangeTombstone makeRangeTombstone(ClusteringComparator comparator, Clustering clustering)
+    public RangeTombstone makeRangeTombstone(ClusteringComparator comparator, Clustering<?> clustering)
     {
         return makeRangeTombstone(Slice.make(comparator, clustering));
     }
@@ -197,7 +197,7 @@ public class UpdateParameters
      * @param clustering the row clustering
      * @return the prefetched row with the already performed modifications
      */
-    public Row getPrefetchedRow(DecoratedKey key, Clustering clustering)
+    public Row getPrefetchedRow(DecoratedKey key, Clustering<?> clustering)
     {
         if (prefetchedRows == null)
             return null;
diff --git a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
index aa5c10d..93ed6ae 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
@@ -259,29 +259,29 @@ public abstract class ColumnCondition
         }
     }
 
-    protected static final Cell getCell(Row row, ColumnMetadata column)
+    protected static final Cell<?> getCell(Row row, ColumnMetadata column)
     {
         // If we're asking for a given cell, and we didn't got any row from our read, it's
         // the same as not having said cell.
         return row == null ? null : row.getCell(column);
     }
 
-    protected static final Cell getCell(Row row, ColumnMetadata column, CellPath path)
+    protected static final Cell<?> getCell(Row row, ColumnMetadata column, CellPath path)
     {
         // If we're asking for a given cell, and we didn't got any row from our read, it's
         // the same as not having said cell.
         return row == null ? null : row.getCell(column, path);
     }
 
-    protected static final Iterator<Cell> getCells(Row row, ColumnMetadata column)
+    protected static final Iterator<Cell<?>> getCells(Row row, ColumnMetadata column)
     {
         // If we're asking for a complex cells, and we didn't got any row from our read, it's
         // the same as not having any cells for that column.
         if (row == null)
-            return Collections.<Cell>emptyIterator();
+            return Collections.emptyIterator();
 
         ComplexColumnData complexData = row.getComplexColumnData(column);
-        return complexData == null ? Collections.<Cell>emptyIterator() : complexData.iterator();
+        return complexData == null ? Collections.<Cell<?>>emptyIterator() : complexData.iterator();
     }
 
     protected static final boolean evaluateComparisonWithOperator(int comparison, Operator operator)
@@ -328,8 +328,8 @@ public abstract class ColumnCondition
 
         private ByteBuffer rowValue(Row row)
         {
-            Cell c = getCell(row, column);
-            return c == null ? null : c.value();
+            Cell<?> c = getCell(row, column);
+            return c == null ? null : c.buffer();
         }
 
         private boolean isSatisfiedBy(ByteBuffer rowValue)
@@ -393,14 +393,14 @@ public abstract class ColumnCondition
         {
             if (column.type.isMultiCell())
             {
-                Cell cell = getCell(row, column, CellPath.create(collectionElement));
-                return cell == null ? null : cell.value();
+                Cell<?> cell = getCell(row, column, CellPath.create(collectionElement));
+                return cell == null ? null : cell.buffer();
             }
 
-            Cell cell = getCell(row, column);
+            Cell<?> cell = getCell(row, column);
             return cell == null
                     ? null
-                    : type.getSerializer().getSerializedValue(cell.value(), collectionElement, type.getKeysType());
+                    : type.getSerializer().getSerializedValue(cell.buffer(), collectionElement, type.getKeysType());
         }
 
         private ByteBuffer rowListValue(ListType<?> type, Row row)
@@ -408,17 +408,17 @@ public abstract class ColumnCondition
             if (column.type.isMultiCell())
                 return cellValueAtIndex(getCells(row, column), getListIndex(collectionElement));
 
-            Cell cell = getCell(row, column);
+            Cell<?> cell = getCell(row, column);
             return cell == null
                     ? null
-                    : type.getSerializer().getElement(cell.value(), getListIndex(collectionElement));
+                    : type.getSerializer().getElement(cell.buffer(), getListIndex(collectionElement));
         }
 
-        private static ByteBuffer cellValueAtIndex(Iterator<Cell> iter, int index)
+        private static ByteBuffer cellValueAtIndex(Iterator<Cell<?>> iter, int index)
         {
             int adv = Iterators.advance(iter, index);
             if (adv == index && iter.hasNext())
-                return iter.next().value();
+                return iter.next().buffer();
 
             return null;
         }
@@ -468,7 +468,7 @@ public abstract class ColumnCondition
             // copy iterator contents so that we can properly reuse them for each comparison with an IN value
             for (Term.Terminal value : values)
             {
-                Iterator<Cell> iter = getCells(row, column);
+                Iterator<Cell<?>> iter = getCells(row, column);
                 if (value == null)
                 {
                     if (comparisonOperator == Operator.EQ)
@@ -490,7 +490,7 @@ public abstract class ColumnCondition
             return false;
         }
 
-        private static boolean valueAppliesTo(CollectionType<?> type, Iterator<Cell> iter, Term.Terminal value, Operator operator)
+        private static boolean valueAppliesTo(CollectionType<?> type, Iterator<Cell<?>> iter, Term.Terminal value, Operator operator)
         {
             if (value == null)
                 return !iter.hasNext();
@@ -510,7 +510,7 @@ public abstract class ColumnCondition
             throw new AssertionError();
         }
 
-        private static boolean setOrListAppliesTo(AbstractType<?> type, Iterator<Cell> iter, Iterator<ByteBuffer> conditionIter, Operator operator, boolean isSet)
+        private static boolean setOrListAppliesTo(AbstractType<?> type, Iterator<Cell<?>> iter, Iterator<ByteBuffer> conditionIter, Operator operator, boolean isSet)
         {
             while(iter.hasNext())
             {
@@ -518,7 +518,7 @@ public abstract class ColumnCondition
                     return (operator == Operator.GT) || (operator == Operator.GTE) || (operator == Operator.NEQ);
 
                 // for lists we use the cell value; for sets we use the cell name
-                ByteBuffer cellValue = isSet ? iter.next().path().get(0) : iter.next().value();
+                ByteBuffer cellValue = isSet ? iter.next().path().get(0) : iter.next().buffer();
                 int comparison = type.compare(cellValue, conditionIter.next());
                 if (comparison != 0)
                     return evaluateComparisonWithOperator(comparison, operator);
@@ -531,19 +531,19 @@ public abstract class ColumnCondition
             return operator == Operator.EQ || operator == Operator.LTE || operator == Operator.GTE;
         }
 
-        private static boolean listAppliesTo(ListType<?> type, Iterator<Cell> iter, List<ByteBuffer> elements, Operator operator)
+        private static boolean listAppliesTo(ListType<?> type, Iterator<Cell<?>> iter, List<ByteBuffer> elements, Operator operator)
         {
             return setOrListAppliesTo(type.getElementsType(), iter, elements.iterator(), operator, false);
         }
 
-        private static boolean setAppliesTo(SetType<?> type, Iterator<Cell> iter, Set<ByteBuffer> elements, Operator operator)
+        private static boolean setAppliesTo(SetType<?> type, Iterator<Cell<?>> iter, Set<ByteBuffer> elements, Operator operator)
         {
             ArrayList<ByteBuffer> sortedElements = new ArrayList<>(elements);
             Collections.sort(sortedElements, type.getElementsType());
             return setOrListAppliesTo(type.getElementsType(), iter, sortedElements.iterator(), operator, true);
         }
 
-        private static boolean mapAppliesTo(MapType<?, ?> type, Iterator<Cell> iter, Map<ByteBuffer, ByteBuffer> elements, Operator operator)
+        private static boolean mapAppliesTo(MapType<?, ?> type, Iterator<Cell<?>> iter, Map<ByteBuffer, ByteBuffer> elements, Operator operator)
         {
             Iterator<Map.Entry<ByteBuffer, ByteBuffer>> conditionIter = elements.entrySet().iterator();
             while(iter.hasNext())
@@ -552,7 +552,7 @@ public abstract class ColumnCondition
                     return (operator == Operator.GT) || (operator == Operator.GTE) || (operator == Operator.NEQ);
 
                 Map.Entry<ByteBuffer, ByteBuffer> conditionEntry = conditionIter.next();
-                Cell c = iter.next();
+                Cell<?> c = iter.next();
 
                 // compare the keys
                 int comparison = type.getKeysType().compare(c.path().get(0), conditionEntry.getKey());
@@ -560,7 +560,7 @@ public abstract class ColumnCondition
                     return evaluateComparisonWithOperator(comparison, operator);
 
                 // compare the values
-                comparison = type.getValuesType().compare(c.value(), conditionEntry.getValue());
+                comparison = type.getValuesType().compare(c.buffer(), conditionEntry.getValue());
                 if (comparison != 0)
                     return evaluateComparisonWithOperator(comparison, operator);
             }
@@ -608,14 +608,14 @@ public abstract class ColumnCondition
 
             if (column.type.isMultiCell())
             {
-                Cell cell = getCell(row, column, userType.cellPathForField(field));
-                return cell == null ? null : cell.value();
+                Cell<?> cell = getCell(row, column, userType.cellPathForField(field));
+                return cell == null ? null : cell.buffer();
             }
 
-            Cell cell = getCell(row, column);
+            Cell<?> cell = getCell(row, column);
             return cell == null
                       ? null
-                      : userType.split(cell.value())[userType.fieldPosition(field)];
+                      : userType.split(cell.buffer())[userType.fieldPosition(field)];
         }
 
         private boolean isSatisfiedBy(ByteBuffer rowValue)
@@ -670,7 +670,7 @@ public abstract class ColumnCondition
         private final ByteBuffer rowValue(Row row)
         {
             UserType userType = (UserType) column.type;
-            Iterator<Cell> iter = getCells(row, column);
+            Iterator<Cell<?>> iter = getCells(row, column);
             return iter.hasNext() ? userType.serializeForNativeProtocol(iter, protocolVersion) : null;
         }
 
diff --git a/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java b/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
index ba82643..35d4a95 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
@@ -91,7 +91,7 @@ public final class ColumnConditions extends AbstractConditions
      * @param options the query options
      */
     public void addConditionsTo(CQL3CasRequest request,
-                                Clustering clustering,
+                                Clustering<?> clustering,
                                 QueryOptions options)
     {
         if (!columnConditions.isEmpty())
diff --git a/src/java/org/apache/cassandra/cql3/conditions/Conditions.java b/src/java/org/apache/cassandra/cql3/conditions/Conditions.java
index 1622be0..1a202df 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/Conditions.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/Conditions.java
@@ -98,6 +98,6 @@ public interface Conditions
      * @param options the query options
      */
     public void addConditionsTo(CQL3CasRequest request,
-                                Clustering clustering,
+                                Clustering<?> clustering,
                                 QueryOptions options);
 }
diff --git a/src/java/org/apache/cassandra/cql3/conditions/IfExistsCondition.java b/src/java/org/apache/cassandra/cql3/conditions/IfExistsCondition.java
index fa8822d..51e8dd0 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/IfExistsCondition.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/IfExistsCondition.java
@@ -24,7 +24,7 @@ import org.apache.cassandra.db.Clustering;
 final class IfExistsCondition extends AbstractConditions
 {
     @Override
-    public void addConditionsTo(CQL3CasRequest request, Clustering clustering, QueryOptions options)
+    public void addConditionsTo(CQL3CasRequest request, Clustering<?> clustering, QueryOptions options)
     {
         request.addExist(clustering);
     }
diff --git a/src/java/org/apache/cassandra/cql3/conditions/IfNotExistsCondition.java b/src/java/org/apache/cassandra/cql3/conditions/IfNotExistsCondition.java
index ede9119..ccbe386 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/IfNotExistsCondition.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/IfNotExistsCondition.java
@@ -24,7 +24,7 @@ import org.apache.cassandra.db.Clustering;
 final class IfNotExistsCondition extends AbstractConditions
 {
     @Override
-    public void addConditionsTo(CQL3CasRequest request, Clustering clustering, QueryOptions options)
+    public void addConditionsTo(CQL3CasRequest request, Clustering<?> clustering, QueryOptions options)
     {
         request.addNotExist(clustering);
     }
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
index 265d354..0a252ff 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
@@ -101,7 +101,7 @@ final class ClusteringColumnRestrictions extends RestrictionSetWrapper
         return false;
     }
 
-    public NavigableSet<Clustering> valuesAsClustering(QueryOptions options) throws InvalidRequestException
+    public NavigableSet<Clustering<?>> valuesAsClustering(QueryOptions options) throws InvalidRequestException
     {
         MultiCBuilder builder = MultiCBuilder.create(comparator, hasIN());
         for (SingleRestriction r : restrictions)
@@ -113,7 +113,7 @@ final class ClusteringColumnRestrictions extends RestrictionSetWrapper
         return builder.build();
     }
 
-    public NavigableSet<ClusteringBound> boundsAsClustering(Bound bound, QueryOptions options) throws InvalidRequestException
+    public NavigableSet<ClusteringBound<?>> boundsAsClustering(Bound bound, QueryOptions options) throws InvalidRequestException
     {
         MultiCBuilder builder = MultiCBuilder.create(comparator, hasIN() || hasMultiColumnSlice());
         int keyPosition = 0;
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index a9005d1..328e0a9 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@ -743,7 +743,7 @@ public final class StatementRestrictions
      * @param options the query options
      * @return the requested clustering columns
      */
-    public NavigableSet<Clustering> getClusteringColumns(QueryOptions options)
+    public NavigableSet<Clustering<?>> getClusteringColumns(QueryOptions options)
     {
         return clusteringColumnsRestrictions.valuesAsClustering(options);
     }
@@ -755,7 +755,7 @@ public final class StatementRestrictions
      * @param options the query options
      * @return the bounds (start or end) of the clustering columns
      */
-    public NavigableSet<ClusteringBound> getClusteringColumnsBounds(Bound b, QueryOptions options)
+    public NavigableSet<ClusteringBound<?>> getClusteringColumnsBounds(Bound b, QueryOptions options)
     {
         return clusteringColumnsRestrictions.boundsAsClustering(b, options);
     }
diff --git a/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java b/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java
index 9b7abe1..84e1e84 100644
--- a/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java
+++ b/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java
@@ -84,7 +84,7 @@ public final class ResultSetBuilder
         current.add(v);
     }
 
-    public void add(Cell c, int nowInSec)
+    public void add(Cell<?> c, int nowInSec)
     {
         if (c == null)
         {
@@ -101,7 +101,7 @@ public final class ResultSetBuilder
             ttls[current.size() - 1] = remainingTTL(c, nowInSec);
     }
 
-    private int remainingTTL(Cell c, int nowInSec)
+    private int remainingTTL(Cell<?> c, int nowInSec)
     {
         if (!c.isExpiring())
             return -1;
@@ -110,11 +110,11 @@ public final class ResultSetBuilder
         return remaining >= 0 ? remaining : -1;
     }
 
-    private ByteBuffer value(Cell c)
+    private <V> ByteBuffer value(Cell<V> c)
     {
         return c.isCounterCell()
-             ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
-             : c.value();
+             ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value(), c.accessor()))
+             : c.buffer();
     }
 
     /**
@@ -123,7 +123,7 @@ public final class ResultSetBuilder
      * @param partitionKey the partition key of the new row
      * @param clustering the clustering of the new row
      */
-    public void newRow(DecoratedKey partitionKey, Clustering clustering)
+    public void newRow(DecoratedKey partitionKey, Clustering<?> clustering)
     {
         // The groupMaker needs to be called for each row
         boolean isNewAggregate = groupMaker == null || groupMaker.isNewGroup(partitionKey, clustering);
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index c165969..2cf2660 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -503,7 +503,7 @@ public class BatchStatement implements CQLStatement
             }
             else
             {
-                Clustering clustering = Iterables.getOnlyElement(statement.createClustering(statementOptions));
+                Clustering<?> clustering = Iterables.getOnlyElement(statement.createClustering(statementOptions));
                 if (statement.hasConditions())
                 {
                     statement.addConditions(clustering, casRequest, statementOptions);
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
index ed985db..d61381d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
@@ -56,7 +56,7 @@ public class CQL3CasRequest implements CASRequest
     // We index RowCondition by the clustering of the row they applied to for 2 reasons:
     //   1) this allows to keep things sorted to build the read command below
     //   2) this allows to detect when contradictory conditions are set (not exists with some other conditions on the same row)
-    private final TreeMap<Clustering, RowCondition> conditions;
+    private final TreeMap<Clustering<?>, RowCondition> conditions;
 
     private final List<RowUpdate> updates = new ArrayList<>();
     private final List<RangeDeletion> rangeDeletions = new ArrayList<>();
@@ -75,7 +75,7 @@ public class CQL3CasRequest implements CASRequest
         this.updatesStaticRow = updatesStaticRow;
     }
 
-    void addRowUpdate(Clustering clustering, ModificationStatement stmt, QueryOptions options, long timestamp, int nowInSeconds)
+    void addRowUpdate(Clustering<?> clustering, ModificationStatement stmt, QueryOptions options, long timestamp, int nowInSeconds)
     {
         updates.add(new RowUpdate(clustering, stmt, options, timestamp, nowInSeconds));
     }
@@ -85,17 +85,17 @@ public class CQL3CasRequest implements CASRequest
         rangeDeletions.add(new RangeDeletion(slice, stmt, options, timestamp, nowInSeconds));
     }
 
-    public void addNotExist(Clustering clustering) throws InvalidRequestException
+    public void addNotExist(Clustering<?> clustering) throws InvalidRequestException
     {
         addExistsCondition(clustering, new NotExistCondition(clustering), true);
     }
 
-    public void addExist(Clustering clustering) throws InvalidRequestException
+    public void addExist(Clustering<?> clustering) throws InvalidRequestException
     {
         addExistsCondition(clustering, new ExistCondition(clustering), false);
     }
 
-    private void addExistsCondition(Clustering clustering, RowCondition condition, boolean isNotExist)
+    private void addExistsCondition(Clustering<?> clustering, RowCondition condition, boolean isNotExist)
     {
         assert condition instanceof ExistCondition || condition instanceof NotExistCondition;
         RowCondition previous = getConditionsForRow(clustering);
@@ -128,7 +128,7 @@ public class CQL3CasRequest implements CASRequest
         hasExists = true;
     }
 
-    public void addConditions(Clustering clustering, Collection<ColumnCondition> conds, QueryOptions options) throws InvalidRequestException
+    public void addConditions(Clustering<?> clustering, Collection<ColumnCondition> conds, QueryOptions options) throws InvalidRequestException
     {
         RowCondition condition = getConditionsForRow(clustering);
         if (condition == null)
@@ -143,12 +143,12 @@ public class CQL3CasRequest implements CASRequest
         ((ColumnsConditions)condition).addConditions(conds, options);
     }
 
-    private RowCondition getConditionsForRow(Clustering clustering)
+    private RowCondition getConditionsForRow(Clustering<?> clustering)
     {
         return clustering == Clustering.STATIC_CLUSTERING ? staticConditions : conditions.get(clustering);
     }
 
-    private void setConditionsForRow(Clustering clustering, RowCondition condition)
+    private void setConditionsForRow(Clustering<?> clustering, RowCondition condition)
     {
         if (clustering == Clustering.STATIC_CLUSTERING)
         {
@@ -255,13 +255,13 @@ public class CQL3CasRequest implements CASRequest
      */
     private class RowUpdate
     {
-        private final Clustering clustering;
+        private final Clustering<?> clustering;
         private final ModificationStatement stmt;
         private final QueryOptions options;
         private final long timestamp;
         private final int nowInSeconds;
 
-        private RowUpdate(Clustering clustering, ModificationStatement stmt, QueryOptions options, long timestamp, int nowInSeconds)
+        private RowUpdate(Clustering<?> clustering, ModificationStatement stmt, QueryOptions options, long timestamp, int nowInSeconds)
         {
             this.clustering = clustering;
             this.stmt = stmt;
@@ -320,9 +320,9 @@ public class CQL3CasRequest implements CASRequest
 
     private static abstract class RowCondition
     {
-        public final Clustering clustering;
+        public final Clustering<?> clustering;
 
-        protected RowCondition(Clustering clustering)
+        protected RowCondition(Clustering<?> clustering)
         {
             this.clustering = clustering;
         }
@@ -332,7 +332,7 @@ public class CQL3CasRequest implements CASRequest
 
     private static class NotExistCondition extends RowCondition
     {
-        private NotExistCondition(Clustering clustering)
+        private NotExistCondition(Clustering<?> clustering)
         {
             super(clustering);
         }
@@ -345,7 +345,7 @@ public class CQL3CasRequest implements CASRequest
 
     private static class ExistCondition extends RowCondition
     {
-        private ExistCondition(Clustering clustering)
+        private ExistCondition(Clustering<?> clustering)
         {
             super(clustering);
         }
@@ -360,7 +360,7 @@ public class CQL3CasRequest implements CASRequest
     {
         private final Multimap<Pair<ColumnIdentifier, ByteBuffer>, ColumnCondition.Bound> conditions = HashMultimap.create();
 
-        private ColumnsConditions(Clustering clustering)
+        private ColumnsConditions(Clustering<?> clustering)
         {
             super(clustering);
         }
diff --git a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index cbba82b..1a92196 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -54,7 +54,7 @@ public class DeleteStatement extends ModificationStatement
     }
 
     @Override
-    public void addUpdateForKey(PartitionUpdate.Builder updateBuilder, Clustering clustering, UpdateParameters params)
+    public void addUpdateForKey(PartitionUpdate.Builder updateBuilder, Clustering<?> clustering, UpdateParameters params)
     throws InvalidRequestException
     {
         TableMetadata metadata = metadata();
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 6ff536d..c55a75e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -26,6 +26,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
@@ -180,7 +181,7 @@ public abstract class ModificationStatement implements CQLStatement
         return restrictions;
     }
 
-    public abstract void addUpdateForKey(PartitionUpdate.Builder updateBuilder, Clustering clustering, UpdateParameters params);
+    public abstract void addUpdateForKey(PartitionUpdate.Builder updateBuilder, Clustering<?> clustering, UpdateParameters params);
 
     public abstract void addUpdateForKey(PartitionUpdate.Builder updateBuilder, Slice slice, UpdateParameters params);
 
@@ -323,7 +324,7 @@ public abstract class ModificationStatement implements CQLStatement
         return partitionKeys;
     }
 
-    public NavigableSet<Clustering> createClustering(QueryOptions options)
+    public NavigableSet<Clustering<?>> createClustering(QueryOptions options)
     throws InvalidRequestException
     {
         if (appliesOnlyToStaticColumns() && !restrictions.hasClusteringColumnsRestrictions())
@@ -503,7 +504,7 @@ public abstract class ModificationStatement implements CQLStatement
                    "IN on the clustering key columns is not supported with conditional %s",
                     type.isUpdate()? "updates" : "deletions");
 
-        Clustering clustering = Iterables.getOnlyElement(createClustering(options));
+        Clustering<?> clustering = Iterables.getOnlyElement(createClustering(options));
         CQL3CasRequest request = new CQL3CasRequest(metadata(), key, conditionColumns(), updatesRegularRows(), updatesStaticRow());
 
         addConditions(clustering, request, options);
@@ -512,7 +513,7 @@ public abstract class ModificationStatement implements CQLStatement
         return request;
     }
 
-    public void addConditions(Clustering clustering, CQL3CasRequest request, QueryOptions options) throws InvalidRequestException
+    public void addConditions(Clustering<?> clustering, CQL3CasRequest request, QueryOptions options) throws InvalidRequestException
     {
         conditions.addConditionsTo(request, clustering, options);
     }
@@ -717,7 +718,7 @@ public abstract class ModificationStatement implements CQLStatement
         }
         else
         {
-            NavigableSet<Clustering> clusterings = createClustering(options);
+            NavigableSet<Clustering<?>> clusterings = createClustering(options);
 
             // If some of the restrictions were unspecified (e.g. empty IN restrictions) we do not need to do anything.
             if (restrictions.hasClusteringColumnsRestrictions() && clusterings.isEmpty())
@@ -738,15 +739,9 @@ public abstract class ModificationStatement implements CQLStatement
                 }
                 else
                 {
-                    for (Clustering clustering : clusterings)
+                    for (Clustering<?> clustering : clusterings)
                     {
-                        for (ByteBuffer c : clustering.getRawValues())
-                        {
-                            if (c != null && c.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
-                                throw new InvalidRequestException(String.format("Key length of %d is longer than maximum of %d",
-                                                                                clustering.dataSize(),
-                                                                                FBUtilities.MAX_UNSIGNED_SHORT));
-                        }
+                        validateClustering(clustering);
                         addUpdateForKey(updateBuilder, clustering, params);
                     }
                 }
@@ -754,16 +749,28 @@ public abstract class ModificationStatement implements CQLStatement
         }
     }
 
+    private <V> void validateClustering(Clustering<V> clustering)
+    {
+        ValueAccessor<V> accessor = clustering.accessor();
+        for (V v : clustering.getRawValues())
+        {
+            if (v != null && accessor.size(v) > FBUtilities.MAX_UNSIGNED_SHORT)
+                throw new InvalidRequestException(String.format("Key length of %d is longer than maximum of %d",
+                                                                clustering.dataSize(),
+                                                                FBUtilities.MAX_UNSIGNED_SHORT));
+        }
+    }
+
     Slices createSlices(QueryOptions options)
     {
-        SortedSet<ClusteringBound> startBounds = restrictions.getClusteringColumnsBounds(Bound.START, options);
-        SortedSet<ClusteringBound> endBounds = restrictions.getClusteringColumnsBounds(Bound.END, options);
+        SortedSet<ClusteringBound<?>> startBounds = restrictions.getClusteringColumnsBounds(Bound.START, options);
+        SortedSet<ClusteringBound<?>> endBounds = restrictions.getClusteringColumnsBounds(Bound.END, options);
 
         return toSlices(startBounds, endBounds);
     }
 
     private UpdateParameters makeUpdateParameters(Collection<ByteBuffer> keys,
-                                                  NavigableSet<Clustering> clusterings,
+                                                  NavigableSet<Clustering<?>> clusterings,
                                                   QueryOptions options,
                                                   boolean local,
                                                   long timestamp,
@@ -818,14 +825,14 @@ public abstract class ModificationStatement implements CQLStatement
                                     lists);
     }
 
-    private Slices toSlices(SortedSet<ClusteringBound> startBounds, SortedSet<ClusteringBound> endBounds)
+    private Slices toSlices(SortedSet<ClusteringBound<?>> startBounds, SortedSet<ClusteringBound<?>> endBounds)
     {
         assert startBounds.size() == endBounds.size();
 
         Slices.Builder builder = new Slices.Builder(metadata().comparator);
 
-        Iterator<ClusteringBound> starts = startBounds.iterator();
-        Iterator<ClusteringBound> ends = endBounds.iterator();
+        Iterator<ClusteringBound<?>> starts = startBounds.iterator();
+        Iterator<ClusteringBound<?>> ends = endBounds.iterator();
 
         while (starts.hasNext())
         {
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index ca80ff6..1ddb29d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -544,7 +544,7 @@ public class SelectStatement implements CQLStatement
             return ((ClusteringIndexSliceFilter)filter).requestedSlices();
 
         Slices.Builder builder = new Slices.Builder(table.comparator);
-        for (Clustering clustering: ((ClusteringIndexNamesFilter)filter).requestedRows())
+        for (Clustering<?> clustering: ((ClusteringIndexNamesFilter)filter).requestedRows())
             builder.add(Slice.make(clustering));
         return builder.build();
     }
@@ -616,7 +616,7 @@ public class SelectStatement implements CQLStatement
             return new ClusteringIndexSliceFilter(slices, isReversed);
         }
 
-        NavigableSet<Clustering> clusterings = getRequestedRows(options);
+        NavigableSet<Clustering<?>> clusterings = getRequestedRows(options);
         // We can have no clusterings if either we're only selecting the static columns, or if we have
         // a 'IN ()' for clusterings. In that case, we still want to query if some static columns are
         // queried. But we're fine otherwise.
@@ -630,27 +630,27 @@ public class SelectStatement implements CQLStatement
     public Slices makeSlices(QueryOptions options)
     throws InvalidRequestException
     {
-        SortedSet<ClusteringBound> startBounds = restrictions.getClusteringColumnsBounds(Bound.START, options);
-        SortedSet<ClusteringBound> endBounds = restrictions.getClusteringColumnsBounds(Bound.END, options);
+        SortedSet<ClusteringBound<?>> startBounds = restrictions.getClusteringColumnsBounds(Bound.START, options);
+        SortedSet<ClusteringBound<?>> endBounds = restrictions.getClusteringColumnsBounds(Bound.END, options);
         assert startBounds.size() == endBounds.size();
 
         // The case where startBounds == 1 is common enough that it's worth optimizing
         if (startBounds.size() == 1)
         {
-            ClusteringBound start = startBounds.first();
-            ClusteringBound end = endBounds.first();
+            ClusteringBound<?> start = startBounds.first();
+            ClusteringBound<?> end = endBounds.first();
             return Slice.isEmpty(table.comparator, start, end)
                  ? Slices.NONE
                  : Slices.with(table.comparator, Slice.make(start, end));
         }
 
         Slices.Builder builder = new Slices.Builder(table.comparator, startBounds.size());
-        Iterator<ClusteringBound> startIter = startBounds.iterator();
-        Iterator<ClusteringBound> endIter = endBounds.iterator();
+        Iterator<ClusteringBound<?>> startIter = startBounds.iterator();
+        Iterator<ClusteringBound<?>> endIter = endBounds.iterator();
         while (startIter.hasNext() && endIter.hasNext())
         {
-            ClusteringBound start = startIter.next();
-            ClusteringBound end = endIter.next();
+            ClusteringBound<?> start = startIter.next();
+            ClusteringBound<?> end = endIter.next();
 
             // Ignore slices that are nonsensical
             if (Slice.isEmpty(table.comparator, start, end))
@@ -748,7 +748,7 @@ public class SelectStatement implements CQLStatement
         return userLimit;
     }
 
-    private NavigableSet<Clustering> getRequestedRows(QueryOptions options) throws InvalidRequestException
+    private NavigableSet<Clustering<?>> getRequestedRows(QueryOptions options) throws InvalidRequestException
     {
         // Note: getRequestedColumns don't handle static columns, but due to CASSANDRA-5762
         // we always do a slice for CQL3 tables, so it's ok to ignore them here
@@ -860,7 +860,7 @@ public class SelectStatement implements CQLStatement
                         result.add(keyComponents[def.position()]);
                         break;
                     case CLUSTERING:
-                        result.add(row.clustering().get(def.position()));
+                        result.add(row.clustering().bufferAt(def.position()));
                         break;
                     case REGULAR:
                         addValue(result, def, row, nowInSec, protocolVersion);
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 57d7691..34b1b99 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -57,7 +57,7 @@ public class UpdateStatement extends ModificationStatement
     }
 
     @Override
-    public void addUpdateForKey(PartitionUpdate.Builder updateBuilder, Clustering clustering, UpdateParameters params)
+    public void addUpdateForKey(PartitionUpdate.Builder updateBuilder, Clustering<?> clustering, UpdateParameters params)
     {
         if (updatesRegularRows())
         {
diff --git a/src/java/org/apache/cassandra/serializers/EmptySerializer.java b/src/java/org/apache/cassandra/db/AbstractArrayClusteringPrefix.java
similarity index 53%
copy from src/java/org/apache/cassandra/serializers/EmptySerializer.java
copy to src/java/org/apache/cassandra/db/AbstractArrayClusteringPrefix.java
index 733e179..b698f5d 100644
--- a/src/java/org/apache/cassandra/serializers/EmptySerializer.java
+++ b/src/java/org/apache/cassandra/db/AbstractArrayClusteringPrefix.java
@@ -16,40 +16,38 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.serializers;
+package org.apache.cassandra.db;
 
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 import java.nio.ByteBuffer;
 
-public class EmptySerializer implements TypeSerializer<Void>
-{
-    public static final EmptySerializer instance = new EmptySerializer();
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
+import org.apache.cassandra.db.marshal.ValueAccessor;
 
-    public Void deserialize(ByteBuffer bytes)
-    {
-        validate(bytes);
-        return null;
-    }
+public abstract class AbstractArrayClusteringPrefix extends AbstractOnHeapClusteringPrefix<byte[]>
+{
+    public static final byte[][] EMPTY_VALUES_ARRAY = new byte[0][];
 
-    public ByteBuffer serialize(Void value)
+    public AbstractArrayClusteringPrefix(Kind kind, byte[][] values)
     {
-        return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+        super(kind, values);
     }
 
-    public void validate(ByteBuffer bytes) throws MarshalException
+    public ValueAccessor<byte[]> accessor()
     {
-        if (bytes.remaining() > 0)
-            throw new MarshalException("EmptyType only accept empty values");
+        return ByteArrayAccessor.instance;
     }
 
-    public String toString(Void value)
+    public ByteBuffer[] getBufferArray()
     {
-        return "";
+        ByteBuffer[] out = new ByteBuffer[values.length];
+        for (int i=0; i<values.length; i++)
+            out[i] = ByteBuffer.wrap(values[i]);
+        return out;
     }
 
-    public Class<Void> getType()
+    public ClusteringPrefix<byte[]> minimize()
     {
-        return Void.class;
+        return this;
     }
 }
diff --git a/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java b/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java
index dd4a095..457d0c4 100644
--- a/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java
+++ b/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java
@@ -19,53 +19,33 @@ package org.apache.cassandra.db;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.db.marshal.ValueAccessor;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-public abstract class AbstractBufferClusteringPrefix extends AbstractClusteringPrefix
+public abstract class AbstractBufferClusteringPrefix extends AbstractOnHeapClusteringPrefix<ByteBuffer>
 {
     public static final ByteBuffer[] EMPTY_VALUES_ARRAY = new ByteBuffer[0];
 
-    protected final Kind kind;
-    protected final ByteBuffer[] values;
-
     protected AbstractBufferClusteringPrefix(Kind kind, ByteBuffer[] values)
     {
-        this.kind = kind;
-        this.values = values;
-    }
-
-    public Kind kind()
-    {
-        return kind;
-    }
-
-    public ClusteringPrefix clustering()
-    {
-        return this;
-    }
-
-    public int size()
-    {
-        return values.length;
-    }
-
-    public ByteBuffer get(int i)
-    {
-        return values[i];
+        super(kind, values);
     }
 
-    public ByteBuffer[] getRawValues()
+    public ValueAccessor<ByteBuffer> accessor()
     {
-        return values;
+        return ByteBufferAccessor.instance;
     }
 
-    public long unsharedHeapSize()
+    public ByteBuffer[] getBufferArray()
     {
-        return Clustering.EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(values);
+        return getRawValues();
     }
 
-    public long unsharedHeapSizeExcludingData()
+    public ClusteringPrefix<ByteBuffer> minimize()
     {
-        return Clustering.EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(values);
+        if (!ByteBufferUtil.canMinimize(values))
+            return this;
+        return new BufferClustering(ByteBufferUtil.minimizeBuffers(values));
     }
 }
diff --git a/src/java/org/apache/cassandra/db/AbstractClusteringPrefix.java b/src/java/org/apache/cassandra/db/AbstractClusteringPrefix.java
deleted file mode 100644
index 8714936..0000000
--- a/src/java/org/apache/cassandra/db/AbstractClusteringPrefix.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.nio.ByteBuffer;
-import java.util.Objects;
-
-public abstract class AbstractClusteringPrefix implements ClusteringPrefix
-{
-    public ClusteringPrefix clustering()
-    {
-        return this;
-    }
-
-    public int dataSize()
-    {
-        int size = 0;
-        for (int i = 0; i < size(); i++)
-        {
-            ByteBuffer bb = get(i);
-            size += bb == null ? 0 : bb.remaining();
-        }
-        return size;
-    }
-
-    public void digest(Digest digest)
-    {
-        for (int i = 0; i < size(); i++)
-        {
-            ByteBuffer bb = get(i);
-            if (bb != null)
-                digest.update(bb);
-        }
-        digest.updateWithByte(kind().ordinal());
-    }
-
-    @Override
-    public final int hashCode()
-    {
-        int result = 31;
-        for (int i = 0; i < size(); i++)
-            result += 31 * Objects.hashCode(get(i));
-        return 31 * result + Objects.hashCode(kind());
-    }
-
-    @Override
-    public final boolean equals(Object o)
-    {
-        if(!(o instanceof ClusteringPrefix))
-            return false;
-
-        ClusteringPrefix that = (ClusteringPrefix)o;
-        if (this.kind() != that.kind() || this.size() != that.size())
-            return false;
-
-        for (int i = 0; i < size(); i++)
-            if (!Objects.equals(this.get(i), that.get(i)))
-                return false;
-
-        return true;
-    }
-}
diff --git a/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java b/src/java/org/apache/cassandra/db/AbstractOnHeapClusteringPrefix.java
similarity index 62%
copy from src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java
copy to src/java/org/apache/cassandra/db/AbstractOnHeapClusteringPrefix.java
index dd4a095..3653bc5 100644
--- a/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java
+++ b/src/java/org/apache/cassandra/db/AbstractOnHeapClusteringPrefix.java
@@ -17,18 +17,12 @@
  */
 package org.apache.cassandra.db;
 
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.utils.ObjectSizes;
-
-public abstract class AbstractBufferClusteringPrefix extends AbstractClusteringPrefix
+public abstract class AbstractOnHeapClusteringPrefix<V> implements ClusteringPrefix<V>
 {
-    public static final ByteBuffer[] EMPTY_VALUES_ARRAY = new ByteBuffer[0];
-
     protected final Kind kind;
-    protected final ByteBuffer[] values;
+    protected final V[] values;
 
-    protected AbstractBufferClusteringPrefix(Kind kind, ByteBuffer[] values)
+    public AbstractOnHeapClusteringPrefix(Kind kind, V[] values)
     {
         this.kind = kind;
         this.values = values;
@@ -39,7 +33,7 @@ public abstract class AbstractBufferClusteringPrefix extends AbstractClusteringP
         return kind;
     }
 
-    public ClusteringPrefix clustering()
+    public ClusteringPrefix<V> clustering()
     {
         return this;
     }
@@ -49,23 +43,25 @@ public abstract class AbstractBufferClusteringPrefix extends AbstractClusteringP
         return values.length;
     }
 
-    public ByteBuffer get(int i)
+    public V get(int i)
     {
         return values[i];
     }
 
-    public ByteBuffer[] getRawValues()
+    public V[] getRawValues()
     {
         return values;
     }
 
-    public long unsharedHeapSize()
+    @Override
+    public int hashCode()
     {
-        return Clustering.EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(values);
+        return ClusteringPrefix.hashCode(this);
     }
 
-    public long unsharedHeapSizeExcludingData()
+    @Override
+    public boolean equals(Object o)
     {
-        return Clustering.EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(values);
+        return ClusteringPrefix.equals(this, o);
     }
 }
diff --git a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
index 2bfaf0c..55c5fc5 100644
--- a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
+++ b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
@@ -45,10 +45,10 @@ public abstract class AbstractReadCommandBuilder
     protected Set<ColumnIdentifier> columns;
     protected final RowFilter filter = RowFilter.create();
 
-    private ClusteringBound lowerClusteringBound;
-    private ClusteringBound upperClusteringBound;
+    private ClusteringBound<?> lowerClusteringBound;
+    private ClusteringBound<?> upperClusteringBound;
 
-    private NavigableSet<Clustering> clusterings;
+    private NavigableSet<Clustering<?>> clusterings;
 
     // Use Util.cmd() instead of this ctor directly
     AbstractReadCommandBuilder(ColumnFamilyStore cfs)
@@ -197,8 +197,8 @@ public abstract class AbstractReadCommandBuilder
         }
         else
         {
-            Slice slice = Slice.make(lowerClusteringBound == null ? ClusteringBound.BOTTOM : lowerClusteringBound,
-                                     upperClusteringBound == null ? ClusteringBound.TOP : upperClusteringBound);
+            Slice slice = Slice.make(lowerClusteringBound == null ? BufferClusteringBound.BOTTOM : lowerClusteringBound,
+                                     upperClusteringBound == null ? BufferClusteringBound.TOP : upperClusteringBound);
             return new ClusteringIndexSliceFilter(Slices.with(cfs.metadata().comparator, slice), reversed);
         }
     }
@@ -230,37 +230,6 @@ public abstract class AbstractReadCommandBuilder
         }
     }
 
-    public static class SinglePartitionSliceBuilder extends AbstractReadCommandBuilder
-    {
-        private final DecoratedKey partitionKey;
-        private Slices.Builder sliceBuilder;
-
-        public SinglePartitionSliceBuilder(ColumnFamilyStore cfs, DecoratedKey key)
-        {
-            super(cfs);
-            this.partitionKey = key;
-            sliceBuilder = new Slices.Builder(cfs.getComparator());
-        }
-
-        public SinglePartitionSliceBuilder addSlice(Slice slice)
-        {
-            sliceBuilder.add(slice);
-            return this;
-        }
-
-        @Override
-        protected ClusteringIndexFilter makeFilter()
-        {
-            return new ClusteringIndexSliceFilter(sliceBuilder.build(), reversed);
-        }
-
-        @Override
-        public ReadCommand build()
-        {
-            return SinglePartitionReadCommand.create(cfs.metadata(), nowInSeconds, makeColumnFilter(), filter, makeLimits(), partitionKey, makeFilter());
-        }
-    }
-
     public static class PartitionRangeBuilder extends AbstractReadCommandBuilder
     {
         private DecoratedKey startKey;
diff --git a/src/java/org/apache/cassandra/db/Clusterable.java b/src/java/org/apache/cassandra/db/ArrayClustering.java
similarity index 54%
copy from src/java/org/apache/cassandra/db/Clusterable.java
copy to src/java/org/apache/cassandra/db/ArrayClustering.java
index 62ab9dc..0d3de49 100644
--- a/src/java/org/apache/cassandra/db/Clusterable.java
+++ b/src/java/org/apache/cassandra/db/ArrayClustering.java
@@ -15,13 +15,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.cassandra.db;
 
-/**
- * Common class for objects that are identified by a clustering prefix, and can be thus sorted by a
- * {@link ClusteringComparator}.
- */
-public interface Clusterable
+import org.apache.cassandra.utils.ObjectSizes;
+
+public class ArrayClustering extends AbstractArrayClusteringPrefix implements Clustering<byte[]>
 {
-    public ClusteringPrefix clustering();
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new ArrayClustering(EMPTY_VALUES_ARRAY));
+
+    public ArrayClustering(byte[]... values)
+    {
+        super(Kind.CLUSTERING, values);
+    }
+
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOfArray(values) + values.length;
+    }
+
+    public long unsharedHeapSizeExcludingData()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOfArray(values);
+    }
+
+    public static ArrayClustering make(byte[]... values)
+    {
+        return new ArrayClustering(values);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/ArrayClusteringBound.java b/src/java/org/apache/cassandra/db/ArrayClusteringBound.java
new file mode 100644
index 0000000..3ec4e8c
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/ArrayClusteringBound.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db;
+
+import java.nio.ByteBuffer;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+
+public class ArrayClusteringBound extends ArrayClusteringBoundOrBoundary implements ClusteringBound<byte[]>
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new ArrayClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, EMPTY_VALUES_ARRAY));
+
+    public ArrayClusteringBound(Kind kind, byte[][] values)
+    {
+        super(kind, values);
+    }
+
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOfArray(values) + values.length;
+    }
+
+    @Override
+    public ClusteringBound<byte[]> invert()
+    {
+        return create(kind().invert(), values);
+    }
+
+    public ClusteringBound<ByteBuffer> copy(AbstractAllocator allocator)
+    {
+        return (ClusteringBound<ByteBuffer>) super.copy(allocator);
+    }
+
+    public static ArrayClusteringBound create(ClusteringPrefix.Kind kind, byte[][] values)
+    {
+        Preconditions.checkArgument(!kind.isBoundary(), "Expected bound clustering kind, got %s", kind);
+        return new ArrayClusteringBound(kind, values);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/Clusterable.java b/src/java/org/apache/cassandra/db/ArrayClusteringBoundOrBoundary.java
similarity index 53%
copy from src/java/org/apache/cassandra/db/Clusterable.java
copy to src/java/org/apache/cassandra/db/ArrayClusteringBoundOrBoundary.java
index 62ab9dc..4fde35f 100644
--- a/src/java/org/apache/cassandra/db/Clusterable.java
+++ b/src/java/org/apache/cassandra/db/ArrayClusteringBoundOrBoundary.java
@@ -15,13 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.cassandra.db;
 
-/**
- * Common class for objects that are identified by a clustering prefix, and can be thus sorted by a
- * {@link ClusteringComparator}.
- */
-public interface Clusterable
+import com.google.common.base.Preconditions;
+
+public abstract class ArrayClusteringBoundOrBoundary extends AbstractArrayClusteringPrefix implements ClusteringBoundOrBoundary<byte[]>
 {
-    public ClusteringPrefix clustering();
+    public ArrayClusteringBoundOrBoundary(Kind kind, byte[][] values)
+    {
+        super(kind, values);
+        Preconditions.checkArgument(values.length > 0 || !kind.isBoundary(),
+                                    "Cannot create bounds/boundary objects without clustering values");
+    }
+    public static ClusteringBoundOrBoundary<byte[]> create(Kind kind, byte[][] values)
+    {
+        return kind.isBoundary()
+               ? new ArrayClusteringBoundary(kind, values)
+               : new ArrayClusteringBound(kind, values);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/ArrayClusteringBoundary.java b/src/java/org/apache/cassandra/db/ArrayClusteringBoundary.java
new file mode 100644
index 0000000..b8c0d62
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/ArrayClusteringBoundary.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.utils.ObjectSizes;
+
+public class ArrayClusteringBoundary extends ArrayClusteringBoundOrBoundary implements ClusteringBoundary<byte[]>
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new ArrayClusteringBoundary(ClusteringPrefix.Kind.INCL_START_BOUND, EMPTY_VALUES_ARRAY));
+
+    public ArrayClusteringBoundary(Kind kind, byte[][] values)
+    {
+        super(kind, values);
+    }
+
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOfArray(values) + values.length;
+    }
+
+    public static ClusteringBoundary<byte[]> create(Kind kind, byte[][] values)
+    {
+        Preconditions.checkArgument(kind.isBoundary(), "Expected boundary clustering kind, got %s", kind);
+        return new ArrayClusteringBoundary(kind, values);
+    }
+
+    @Override
+    public ClusteringBoundary<byte[]> invert()
+    {
+        return create(kind().invert(), values);
+    }
+
+    public ClusteringBound<byte[]> openBound(boolean reversed)
+    {
+        return ArrayClusteringBound.create(kind.openBoundOfBoundary(reversed), values);
+    }
+
+    public ClusteringBound<byte[]> closeBound(boolean reversed)
+    {
+        return ArrayClusteringBound.create(kind.closeBoundOfBoundary(reversed), values);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/BufferClustering.java b/src/java/org/apache/cassandra/db/BufferClustering.java
index fc635ab..a6dcd1b 100644
--- a/src/java/org/apache/cassandra/db/BufferClustering.java
+++ b/src/java/org/apache/cassandra/db/BufferClustering.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
 
 /**
  * The clustering column values for a row.
@@ -32,17 +33,27 @@ import org.apache.cassandra.utils.ByteBufferUtil;
  * {@code null} values (this is currently only allowed in COMPACT table for historical reasons, but we
  * could imagine lifting that limitation if we decide it make sense from a CQL point of view).
  */
-public class BufferClustering extends AbstractBufferClusteringPrefix implements Clustering
+public class BufferClustering extends AbstractBufferClusteringPrefix implements Clustering<ByteBuffer>
 {
-    BufferClustering(ByteBuffer... values)
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferClustering(EMPTY_VALUES_ARRAY));
+
+    public BufferClustering(ByteBuffer... values)
     {
         super(Kind.CLUSTERING, values);
     }
 
-    public ClusteringPrefix minimize()
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(values);
+    }
+
+    public long unsharedHeapSizeExcludingData()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(values);
+    }
+
+    public static BufferClustering make(ByteBuffer... values)
     {
-        if (!ByteBufferUtil.canMinimize(values))
-            return this;
-        return new BufferClustering(ByteBufferUtil.minimizeBuffers(values));
+        return new BufferClustering(values);
     }
 }
diff --git a/src/java/org/apache/cassandra/db/BufferClusteringBound.java b/src/java/org/apache/cassandra/db/BufferClusteringBound.java
new file mode 100644
index 0000000..9b511fa
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/BufferClusteringBound.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db;
+
+import java.nio.ByteBuffer;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+
+public class BufferClusteringBound extends BufferClusteringBoundOrBoundary implements ClusteringBound<ByteBuffer>
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferClusteringBound(Kind.INCL_START_BOUND, EMPTY_VALUES_ARRAY));
+
+    public BufferClusteringBound(ClusteringPrefix.Kind kind, ByteBuffer[] values)
+    {
+        super(kind, values);
+    }
+
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(values);
+    }
+
+    @Override
+    public ClusteringBound<ByteBuffer> invert()
+    {
+        return create(kind().invert(), values);
+    }
+
+    public ClusteringBound<ByteBuffer> copy(AbstractAllocator allocator)
+    {
+        return (ClusteringBound<ByteBuffer>) super.copy(allocator);
+    }
+
+    public static BufferClusteringBound create(ClusteringPrefix.Kind kind, ByteBuffer[] values)
+    {
+        Preconditions.checkArgument(!kind.isBoundary(), "Expected bound clustering kind, got %s", kind);
+        return new BufferClusteringBound(kind, values);
+    }
+
+    public static BufferClusteringBound inclusiveStartOf(ByteBuffer... values)
+    {
+        return create(ClusteringPrefix.Kind.INCL_START_BOUND, values);
+    }
+
+    public static BufferClusteringBound inclusiveEndOf(ByteBuffer... values)
+    {
+        return create(ClusteringPrefix.Kind.INCL_END_BOUND, values);
+    }
+
+    public static BufferClusteringBound exclusiveStartOf(ByteBuffer... values)
+    {
+        return create(ClusteringPrefix.Kind.EXCL_START_BOUND, values);
+    }
+
+    public static BufferClusteringBound exclusiveEndOf(ByteBuffer... values)
+    {
+        return create(ClusteringPrefix.Kind.EXCL_END_BOUND, values);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/Clusterable.java b/src/java/org/apache/cassandra/db/BufferClusteringBoundOrBoundary.java
similarity index 62%
copy from src/java/org/apache/cassandra/db/Clusterable.java
copy to src/java/org/apache/cassandra/db/BufferClusteringBoundOrBoundary.java
index 62ab9dc..84c5652 100644
--- a/src/java/org/apache/cassandra/db/Clusterable.java
+++ b/src/java/org/apache/cassandra/db/BufferClusteringBoundOrBoundary.java
@@ -15,13 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.cassandra.db;
 
-/**
- * Common class for objects that are identified by a clustering prefix, and can be thus sorted by a
- * {@link ClusteringComparator}.
- */
-public interface Clusterable
+import java.nio.ByteBuffer;
+
+import com.google.common.base.Preconditions;
+
+public abstract class BufferClusteringBoundOrBoundary extends AbstractBufferClusteringPrefix implements ClusteringBoundOrBoundary<ByteBuffer>
 {
-    public ClusteringPrefix clustering();
+    BufferClusteringBoundOrBoundary(Kind kind, ByteBuffer[] values)
+    {
+        super(kind, values);
+        Preconditions.checkArgument(values.length > 0 || !kind.isBoundary(),
+                                    "Cannot create bounds/boundary objects without clustering values");
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/BufferClusteringBoundary.java b/src/java/org/apache/cassandra/db/BufferClusteringBoundary.java
new file mode 100644
index 0000000..cad4d87
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/BufferClusteringBoundary.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db;
+
+import java.nio.ByteBuffer;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+
+public class BufferClusteringBoundary extends BufferClusteringBoundOrBoundary implements ClusteringBoundary<ByteBuffer>
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferClusteringBoundary(Kind.INCL_START_BOUND, EMPTY_VALUES_ARRAY));
+
+    public BufferClusteringBoundary(Kind kind, ByteBuffer[] values)
+    {
+        super(kind, values);
+    }
+
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(values);
+    }
+
+    public static ClusteringBoundary<ByteBuffer> create(Kind kind, ByteBuffer[] values)
+    {
+        Preconditions.checkArgument(kind.isBoundary(), "Expected boundary clustering kind, got %s", kind);
+        return new BufferClusteringBoundary(kind, values);
+    }
+
+    @Override
+    public ClusteringBoundary<ByteBuffer> invert()
+    {
+        return create(kind().invert(), values);
+    }
+
+    @Override
+    public ClusteringBoundary<ByteBuffer> copy(AbstractAllocator allocator)
+    {
+        return (ClusteringBoundary<ByteBuffer>) super.copy(allocator);
+    }
+
+    public ClusteringBound<ByteBuffer> openBound(boolean reversed)
+    {
+        return BufferClusteringBound.create(kind.openBoundOfBoundary(reversed), values);
+    }
+
+    public ClusteringBound<ByteBuffer> closeBound(boolean reversed)
+    {
+        return BufferClusteringBound.create(kind.closeBoundOfBoundary(reversed), values);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/CBuilder.java b/src/java/org/apache/cassandra/db/CBuilder.java
index be56394..7b28684 100644
--- a/src/java/org/apache/cassandra/db/CBuilder.java
+++ b/src/java/org/apache/cassandra/db/CBuilder.java
@@ -22,6 +22,8 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.db.marshal.ValueAccessor;
 
 /**
  * Allows to build ClusteringPrefixes, either Clustering or ClusteringBound.
@@ -45,7 +47,7 @@ public abstract class CBuilder
             throw new UnsupportedOperationException();
         }
 
-        public CBuilder add(ByteBuffer value)
+        public <T> CBuilder add(T value, ValueAccessor<T> accessor)
         {
             throw new UnsupportedOperationException();
         }
@@ -55,37 +57,22 @@ public abstract class CBuilder
             throw new UnsupportedOperationException();
         }
 
-        public Clustering build()
+        public Clustering<?> build()
         {
             return Clustering.STATIC_CLUSTERING;
         }
 
-        public ClusteringBound buildBound(boolean isStart, boolean isInclusive)
+        public ClusteringBound<?> buildBound(boolean isStart, boolean isInclusive)
         {
             throw new UnsupportedOperationException();
         }
 
-        public Slice buildSlice()
+        public Clustering<?> buildWith(List<ByteBuffer> newValues)
         {
             throw new UnsupportedOperationException();
         }
 
-        public Clustering buildWith(ByteBuffer value)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public Clustering buildWith(List<ByteBuffer> newValues)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public ClusteringBound buildBoundWith(ByteBuffer value, boolean isStart, boolean isInclusive)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public ClusteringBound buildBoundWith(List<ByteBuffer> newValues, boolean isStart, boolean isInclusive)
+        public ClusteringBound<?> buildBoundWith(List<ByteBuffer> newValues, boolean isStart, boolean isInclusive)
         {
             throw new UnsupportedOperationException();
         }
@@ -99,15 +86,20 @@ public abstract class CBuilder
     public abstract int count();
     public abstract int remainingCount();
     public abstract ClusteringComparator comparator();
-    public abstract CBuilder add(ByteBuffer value);
+    public final CBuilder add(ByteBuffer value)
+    {
+        return add(value, ByteBufferAccessor.instance);
+    }
+    public final <V> CBuilder add(ClusteringPrefix<V> prefix, int i)
+    {
+        return add(prefix.get(i), prefix.accessor());
+    }
+    public abstract <V> CBuilder add(V value, ValueAccessor<V> accessor);
     public abstract CBuilder add(Object value);
-    public abstract Clustering build();
-    public abstract ClusteringBound buildBound(boolean isStart, boolean isInclusive);
-    public abstract Slice buildSlice();
-    public abstract Clustering buildWith(ByteBuffer value);
-    public abstract Clustering buildWith(List<ByteBuffer> newValues);
-    public abstract ClusteringBound buildBoundWith(ByteBuffer value, boolean isStart, boolean isInclusive);
-    public abstract ClusteringBound buildBoundWith(List<ByteBuffer> newValues, boolean isStart, boolean isInclusive);
+    public abstract Clustering<?> build();
+    public abstract ClusteringBound<?> buildBound(boolean isStart, boolean isInclusive);
+    public abstract Clustering<?> buildWith(List<ByteBuffer> newValues);
+    public abstract ClusteringBound<?> buildBoundWith(List<ByteBuffer> newValues, boolean isStart, boolean isInclusive);
 
     private static class ArrayBackedBuilder extends CBuilder
     {
@@ -137,11 +129,11 @@ public abstract class CBuilder
             return type;
         }
 
-        public CBuilder add(ByteBuffer value)
+        public <V> CBuilder add(V value, ValueAccessor<V> accessor)
         {
             if (isDone())
                 throw new IllegalStateException();
-            values[size++] = value;
+            values[size++] = accessor.toBuffer(value);
             return this;
         }
 
@@ -155,7 +147,7 @@ public abstract class CBuilder
             return remainingCount() == 0 || built;
         }
 
-        public Clustering build()
+        public Clustering<?> build()
         {
             // We don't allow to add more element to a builder that has been built so
             // that we don't have to copy values.
@@ -165,41 +157,20 @@ public abstract class CBuilder
             return size == 0 ? Clustering.EMPTY : Clustering.make(values);
         }
 
-        public ClusteringBound buildBound(boolean isStart, boolean isInclusive)
+        public ClusteringBound<?> buildBound(boolean isStart, boolean isInclusive)
         {
             // We don't allow to add more element to a builder that has been built so
             // that we don't have to copy values (even though we have to do it in most cases).
             built = true;
 
             if (size == 0)
-                return isStart ? ClusteringBound.BOTTOM : ClusteringBound.TOP;
+                return isStart ? BufferClusteringBound.BOTTOM : BufferClusteringBound.TOP;
 
-            return ClusteringBound.create(ClusteringBound.boundKind(isStart, isInclusive),
+            return BufferClusteringBound.create(ClusteringBound.boundKind(isStart, isInclusive),
                                 size == values.length ? values : Arrays.copyOfRange(values, 0, size));
         }
 
-        public Slice buildSlice()
-        {
-            // We don't allow to add more element to a builder that has been built so
-            // that we don't have to copy values.
-            built = true;
-
-            if (size == 0)
-                return Slice.ALL;
-
-            return Slice.make(buildBound(true, true), buildBound(false, true));
-        }
-
-        public Clustering buildWith(ByteBuffer value)
-        {
-            assert size+1 <= type.size();
-
-            ByteBuffer[] newValues = Arrays.copyOf(values, type.size());
-            newValues[size] = value;
-            return Clustering.make(newValues);
-        }
-
-        public Clustering buildWith(List<ByteBuffer> newValues)
+        public Clustering<?> buildWith(List<ByteBuffer> newValues)
         {
             assert size + newValues.size() <= type.size();
             ByteBuffer[] buffers = Arrays.copyOf(values, type.size());
@@ -210,21 +181,14 @@ public abstract class CBuilder
             return Clustering.make(buffers);
         }
 
-        public ClusteringBound buildBoundWith(ByteBuffer value, boolean isStart, boolean isInclusive)
-        {
-            ByteBuffer[] newValues = Arrays.copyOf(values, size+1);
-            newValues[size] = value;
-            return ClusteringBound.create(ClusteringBound.boundKind(isStart, isInclusive), newValues);
-        }
-
-        public ClusteringBound buildBoundWith(List<ByteBuffer> newValues, boolean isStart, boolean isInclusive)
+        public ClusteringBound<?> buildBoundWith(List<ByteBuffer> newValues, boolean isStart, boolean isInclusive)
         {
             ByteBuffer[] buffers = Arrays.copyOf(values, size + newValues.size());
             int newSize = size;
             for (ByteBuffer value : newValues)
                 buffers[newSize++] = value;
 
-            return ClusteringBound.create(ClusteringBound.boundKind(isStart, isInclusive), buffers);
+            return BufferClusteringBound.create(ClusteringBound.boundKind(isStart, isInclusive), buffers);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/db/Clusterable.java b/src/java/org/apache/cassandra/db/Clusterable.java
index 62ab9dc..118b272 100644
--- a/src/java/org/apache/cassandra/db/Clusterable.java
+++ b/src/java/org/apache/cassandra/db/Clusterable.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.db;
  * Common class for objects that are identified by a clustering prefix, and can be thus sorted by a
  * {@link ClusteringComparator}.
  */
-public interface Clusterable
+public interface Clusterable<T>
 {
-    public ClusteringPrefix clustering();
+    public ClusteringPrefix<T> clustering();
 }
diff --git a/src/java/org/apache/cassandra/db/Clustering.java b/src/java/org/apache/cassandra/db/Clustering.java
index 451a087..c685638 100644
--- a/src/java/org/apache/cassandra/db/Clustering.java
+++ b/src/java/org/apache/cassandra/db/Clustering.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -29,20 +30,17 @@ import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 import static org.apache.cassandra.db.AbstractBufferClusteringPrefix.EMPTY_VALUES_ARRAY;
 
-public interface Clustering extends ClusteringPrefix
+public interface Clustering<V> extends ClusteringPrefix<V>
 {
-    static final long EMPTY_SIZE = ObjectSizes.measure(new BufferClustering(EMPTY_VALUES_ARRAY));
-
     public static final Serializer serializer = new Serializer();
 
     public long unsharedHeapSizeExcludingData();
 
-    public default Clustering copy(AbstractAllocator allocator)
+    public default Clustering<?> copy(AbstractAllocator allocator)
     {
         // Important for STATIC_CLUSTERING (but must copy empty native clustering types).
         if (size() == 0)
@@ -51,7 +49,7 @@ public interface Clustering extends ClusteringPrefix
         ByteBuffer[] newValues = new ByteBuffer[size()];
         for (int i = 0; i < size(); i++)
         {
-            ByteBuffer val = get(i);
+            ByteBuffer val = accessor().toBuffer(get(i));
             newValues[i] = val == null ? null : allocator.clone(val);
         }
         return new BufferClustering(newValues);
@@ -63,7 +61,7 @@ public interface Clustering extends ClusteringPrefix
         for (int i = 0; i < size(); i++)
         {
             ColumnMetadata c = metadata.clusteringColumns().get(i);
-            sb.append(i == 0 ? "" : ", ").append(c.name).append('=').append(get(i) == null ? "null" : c.type.getString(get(i)));
+            sb.append(i == 0 ? "" : ", ").append(c.name).append('=').append(get(i) == null ? "null" : c.type.getString(get(i), accessor()));
         }
         return sb.toString();
     }
@@ -74,12 +72,12 @@ public interface Clustering extends ClusteringPrefix
         for (int i = 0; i < size(); i++)
         {
             ColumnMetadata c = metadata.clusteringColumns().get(i);
-            sb.append(i == 0 ? "" : ", ").append(c.type.getString(get(i)));
+            sb.append(i == 0 ? "" : ", ").append(c.type.getString(get(i), accessor()));
         }
         return sb.toString();
     }
 
-    public static Clustering make(ByteBuffer... values)
+    public static Clustering<ByteBuffer> make(ByteBuffer... values)
     {
         return values.length == 0 ? EMPTY : new BufferClustering(values);
     }
@@ -88,7 +86,7 @@ public interface Clustering extends ClusteringPrefix
      * The special cased clustering used by all static rows. It is a special case in the
      * sense that it's always empty, no matter how many clustering columns the table has.
      */
-    public static final Clustering STATIC_CLUSTERING = new BufferClustering(EMPTY_VALUES_ARRAY)
+    public static final Clustering<ByteBuffer> STATIC_CLUSTERING = new BufferClustering(EMPTY_VALUES_ARRAY)
     {
         @Override
         public Kind kind()
@@ -110,7 +108,7 @@ public interface Clustering extends ClusteringPrefix
     };
 
     /** Empty clustering for tables having no clustering columns. */
-    public static final Clustering EMPTY = new BufferClustering(EMPTY_VALUES_ARRAY)
+    public static final Clustering<ByteBuffer> EMPTY = new BufferClustering(EMPTY_VALUES_ARRAY)
     {
         @Override
         public String toString(TableMetadata metadata)
@@ -127,14 +125,14 @@ public interface Clustering extends ClusteringPrefix
      */
     public static class Serializer
     {
-        public void serialize(Clustering clustering, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
+        public void serialize(Clustering<?> clustering, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
         {
             assert clustering != STATIC_CLUSTERING : "We should never serialize a static clustering";
             assert clustering.size() == types.size() : "Invalid clustering for the table: " + clustering;
             ClusteringPrefix.serializer.serializeValuesWithoutSize(clustering, out, version, types);
         }
 
-        public ByteBuffer serialize(Clustering clustering, int version, List<AbstractType<?>> types)
+        public ByteBuffer serialize(Clustering<?> clustering, int version, List<AbstractType<?>> types)
         {
             try (DataOutputBuffer buffer = new DataOutputBuffer((int)serializedSize(clustering, version, types)))
             {
@@ -147,7 +145,7 @@ public interface Clustering extends ClusteringPrefix
             }
         }
 
-        public long serializedSize(Clustering clustering, int version, List<AbstractType<?>> types)
+        public long serializedSize(Clustering<?> clustering, int version, List<AbstractType<?>> types)
         {
             return ClusteringPrefix.serializer.valuesWithoutSizeSerializedSize(clustering, version, types);
         }
@@ -158,16 +156,16 @@ public interface Clustering extends ClusteringPrefix
                 ClusteringPrefix.serializer.skipValuesWithoutSize(in, types.size(), version, types);
         }
 
-        public Clustering deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
+        public Clustering<byte[]> deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
         {
             if (types.isEmpty())
-                return EMPTY;
+                return ByteArrayAccessor.factory.clustering();
 
-            ByteBuffer[] values = ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, types.size(), version, types);
-            return new BufferClustering(values);
+            byte[][] values = ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, types.size(), version, types);
+            return ByteArrayAccessor.factory.clustering(values);
         }
 
-        public Clustering deserialize(ByteBuffer in, int version, List<AbstractType<?>> types)
+        public Clustering<byte[]> deserialize(ByteBuffer in, int version, List<AbstractType<?>> types)
         {
             try (DataInputBuffer buffer = new DataInputBuffer(in, true))
             {
diff --git a/src/java/org/apache/cassandra/db/ClusteringBound.java b/src/java/org/apache/cassandra/db/ClusteringBound.java
index 6ae0816..364856f 100644
--- a/src/java/org/apache/cassandra/db/ClusteringBound.java
+++ b/src/java/org/apache/cassandra/db/ClusteringBound.java
@@ -23,135 +23,55 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * The start or end of a range of clusterings, either inclusive or exclusive.
  */
-public class ClusteringBound extends ClusteringBoundOrBoundary
+public interface ClusteringBound<V> extends ClusteringBoundOrBoundary<V>
 {
     /** The smallest start bound, i.e. the one that starts before any row. */
-    public static final ClusteringBound BOTTOM = new ClusteringBound(Kind.INCL_START_BOUND, EMPTY_VALUES_ARRAY);
+    public static final ClusteringBound<?> BOTTOM = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, BufferClusteringBound.EMPTY_VALUES_ARRAY);
     /** The biggest end bound, i.e. the one that ends after any row. */
-    public static final ClusteringBound TOP = new ClusteringBound(Kind.INCL_END_BOUND, EMPTY_VALUES_ARRAY);
+    public static final ClusteringBound<?> TOP = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, BufferClusteringBound.EMPTY_VALUES_ARRAY);
 
-    protected ClusteringBound(Kind kind, ByteBuffer[] values)
-    {
-        super(kind, values);
-    }
-
-    public ClusteringPrefix minimize()
-    {
-        if (!ByteBufferUtil.canMinimize(values))
-            return this;
-        return new ClusteringBound(kind, ByteBufferUtil.minimizeBuffers(values));
-    }
-
-    public static ClusteringBound create(Kind kind, ByteBuffer[] values)
-    {
-        assert !kind.isBoundary();
-        return new ClusteringBound(kind, values);
-    }
-
-    public static Kind boundKind(boolean isStart, boolean isInclusive)
+    public static ClusteringPrefix.Kind boundKind(boolean isStart, boolean isInclusive)
     {
         return isStart
-             ? (isInclusive ? Kind.INCL_START_BOUND : Kind.EXCL_START_BOUND)
-             : (isInclusive ? Kind.INCL_END_BOUND : Kind.EXCL_END_BOUND);
-    }
-
-    public static ClusteringBound inclusiveStartOf(ByteBuffer... values)
-    {
-        return create(Kind.INCL_START_BOUND, values);
-    }
-
-    public static ClusteringBound inclusiveEndOf(ByteBuffer... values)
-    {
-        return create(Kind.INCL_END_BOUND, values);
-    }
-
-    public static ClusteringBound exclusiveStartOf(ByteBuffer... values)
-    {
-        return create(Kind.EXCL_START_BOUND, values);
-    }
-
-    public static ClusteringBound exclusiveEndOf(ByteBuffer... values)
-    {
-        return create(Kind.EXCL_END_BOUND, values);
-    }
-
-    public static ClusteringBound inclusiveStartOf(ClusteringPrefix prefix)
-    {
-        ByteBuffer[] values = new ByteBuffer[prefix.size()];
-        for (int i = 0; i < prefix.size(); i++)
-            values[i] = prefix.get(i);
-        return inclusiveStartOf(values);
-    }
-
-    public static ClusteringBound exclusiveStartOf(ClusteringPrefix prefix)
-    {
-        ByteBuffer[] values = new ByteBuffer[prefix.size()];
-        for (int i = 0; i < prefix.size(); i++)
-            values[i] = prefix.get(i);
-        return exclusiveStartOf(values);
-    }
-
-    public static ClusteringBound inclusiveEndOf(ClusteringPrefix prefix)
-    {
-        ByteBuffer[] values = new ByteBuffer[prefix.size()];
-        for (int i = 0; i < prefix.size(); i++)
-            values[i] = prefix.get(i);
-        return inclusiveEndOf(values);
-    }
-
-    public static ClusteringBound create(ClusteringComparator comparator, boolean isStart, boolean isInclusive, Object... values)
-    {
-        CBuilder builder = CBuilder.create(comparator);
-        for (Object val : values)
-        {
-            if (val instanceof ByteBuffer)
-                builder.add((ByteBuffer) val);
-            else
-                builder.add(val);
-        }
-        return builder.buildBound(isStart, isInclusive);
+               ? (isInclusive ? ClusteringPrefix.Kind.INCL_START_BOUND : ClusteringPrefix.Kind.EXCL_START_BOUND)
+               : (isInclusive ? ClusteringPrefix.Kind.INCL_END_BOUND : ClusteringPrefix.Kind.EXCL_END_BOUND);
     }
 
     @Override
-    public ClusteringBound invert()
-    {
-        return create(kind().invert(), values);
-    }
+    ClusteringBound<V> invert();
 
-    public ClusteringBound copy(AbstractAllocator allocator)
-    {
-        return (ClusteringBound) super.copy(allocator);
-    }
+    @Override
+    ClusteringBound<ByteBuffer> copy(AbstractAllocator allocator);
 
-    public boolean isStart()
+    default boolean isStart()
     {
         return kind().isStart();
     }
 
-    public boolean isEnd()
+    default boolean isEnd()
     {
         return !isStart();
     }
 
-    public boolean isInclusive()
+    default boolean isInclusive()
     {
-        return kind == Kind.INCL_START_BOUND || kind == Kind.INCL_END_BOUND;
+        return kind() == Kind.INCL_START_BOUND || kind() == Kind.INCL_END_BOUND;
     }
 
-    public boolean isExclusive()
+    default boolean isExclusive()
     {
-        return kind == Kind.EXCL_START_BOUND || kind == Kind.EXCL_END_BOUND;
+        return kind() == Kind.EXCL_START_BOUND || kind() == Kind.EXCL_END_BOUND;
     }
 
     // For use by intersects, it's called with the sstable bound opposite to the slice bound
     // (so if the slice bound is a start, it's call with the max sstable bound)
-    int compareTo(ClusteringComparator comparator, List<ByteBuffer> sstableBound)
+    default int compareTo(ClusteringComparator comparator, List<ByteBuffer> sstableBound)
     {
         for (int i = 0; i < sstableBound.size(); i++)
         {
@@ -162,7 +82,7 @@ public class ClusteringBound extends ClusteringBoundOrBoundary
             if (i >= size())
                 return isStart() ? -1 : 1;
 
-            int cmp = comparator.compareComponent(i, get(i), sstableBound.get(i));
+            int cmp = comparator.compareComponent(i, get(i), accessor(), sstableBound.get(i), ByteBufferAccessor.instance);
             if (cmp != 0)
                 return cmp;
         }
@@ -176,4 +96,42 @@ public class ClusteringBound extends ClusteringBoundOrBoundary
         // The slice bound is equal to the sstable bound. Results depends on whether the slice is inclusive or not
         return isInclusive() ? 0 : (isStart() ? 1 : -1);
     }
+
+    static <V> ClusteringBound<V> create(ClusteringPrefix.Kind kind, ClusteringPrefix<V> from)
+    {
+        return from.accessor().factory().bound(kind, from.getRawValues());
+    }
+
+    public static ClusteringBound<?> inclusiveStartOf(ClusteringPrefix<?> from)
+    {
+        return create(ClusteringPrefix.Kind.INCL_START_BOUND, from);
+    }
+
+    public static ClusteringBound<?> inclusiveEndOf(ClusteringPrefix<?> from)
+    {
+        return create(ClusteringPrefix.Kind.INCL_END_BOUND, from);
+    }
+
+    public static ClusteringBound<?> exclusiveStartOf(ClusteringPrefix<?> from)
+    {
+        return create(ClusteringPrefix.Kind.EXCL_START_BOUND, from);
+    }
+
+    public static ClusteringBound<?> exclusiveEndOf(ClusteringPrefix<?> from)
+    {
+        return create(ClusteringPrefix.Kind.EXCL_END_BOUND, from);
+    }
+
+    public static ClusteringBound<?> create(ClusteringComparator comparator, boolean isStart, boolean isInclusive, Object... values)
+    {
+        CBuilder builder = CBuilder.create(comparator);
+        for (Object val : values)
+        {
+            if (val instanceof ByteBuffer)
+                builder.add((ByteBuffer) val);
+            else
+                builder.add(val);
+        }
+        return builder.buildBound(isStart, isInclusive);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/ClusteringBoundOrBoundary.java b/src/java/org/apache/cassandra/db/ClusteringBoundOrBoundary.java
index 84a9e30..7be9ed2 100644
--- a/src/java/org/apache/cassandra/db/ClusteringBoundOrBoundary.java
+++ b/src/java/org/apache/cassandra/db/ClusteringBoundOrBoundary.java
@@ -24,6 +24,8 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -41,82 +43,39 @@ import org.apache.cassandra.utils.memory.AbstractAllocator;
  *      iterator. See this comment for more details: https://goo.gl/yyB5mR.
  *   2) This saves some storage space.
  */
-public abstract class ClusteringBoundOrBoundary extends AbstractBufferClusteringPrefix
+public interface ClusteringBoundOrBoundary<V> extends ClusteringPrefix<V>
 {
     public static final ClusteringBoundOrBoundary.Serializer serializer = new Serializer();
 
-    protected ClusteringBoundOrBoundary(Kind kind, ByteBuffer[] values)
+    default boolean isBoundary()
     {
-        super(kind, values);
-        assert values.length > 0 || !kind.isBoundary();
+        return kind().isBoundary();
     }
 
-    public static ClusteringBoundOrBoundary create(Kind kind, ByteBuffer[] values)
+    default boolean isOpen(boolean reversed)
     {
-        return kind.isBoundary()
-                ? new ClusteringBoundary(kind, values)
-                : new ClusteringBound(kind, values);
+        return kind().isOpen(reversed);
     }
 
-    public boolean isBoundary()
+    default boolean isClose(boolean reversed)
     {
-        return kind.isBoundary();
+        return kind().isClose(reversed);
     }
 
-    public boolean isOpen(boolean reversed)
-    {
-        return kind.isOpen(reversed);
-    }
-
-    public boolean isClose(boolean reversed)
-    {
-        return kind.isClose(reversed);
-    }
-
-    public static ClusteringBound inclusiveOpen(boolean reversed, ByteBuffer[] boundValues)
-    {
-        return new ClusteringBound(reversed ? Kind.INCL_END_BOUND : Kind.INCL_START_BOUND, boundValues);
-    }
-
-    public static ClusteringBound exclusiveOpen(boolean reversed, ByteBuffer[] boundValues)
-    {
-        return new ClusteringBound(reversed ? Kind.EXCL_END_BOUND : Kind.EXCL_START_BOUND, boundValues);
-    }
-
-    public static ClusteringBound inclusiveClose(boolean reversed, ByteBuffer[] boundValues)
-    {
-        return new ClusteringBound(reversed ? Kind.INCL_START_BOUND : Kind.INCL_END_BOUND, boundValues);
-    }
-
-    public static ClusteringBound exclusiveClose(boolean reversed, ByteBuffer[] boundValues)
-    {
-        return new ClusteringBound(reversed ? Kind.EXCL_START_BOUND : Kind.EXCL_END_BOUND, boundValues);
-    }
-
-    public static ClusteringBoundary inclusiveCloseExclusiveOpen(boolean reversed, ByteBuffer[] boundValues)
-    {
-        return new ClusteringBoundary(reversed ? Kind.EXCL_END_INCL_START_BOUNDARY : Kind.INCL_END_EXCL_START_BOUNDARY, boundValues);
-    }
-
-    public static ClusteringBoundary exclusiveCloseInclusiveOpen(boolean reversed, ByteBuffer[] boundValues)
-    {
-        return new ClusteringBoundary(reversed ? Kind.INCL_END_EXCL_START_BOUNDARY : Kind.EXCL_END_INCL_START_BOUNDARY, boundValues);
-    }
-
-    public ClusteringBoundOrBoundary copy(AbstractAllocator allocator)
+    default ClusteringBoundOrBoundary<ByteBuffer> copy(AbstractAllocator allocator)
     {
         ByteBuffer[] newValues = new ByteBuffer[size()];
         for (int i = 0; i < size(); i++)
-            newValues[i] = allocator.clone(get(i));
-        return create(kind(), newValues);
+            newValues[i] = allocator.clone(get(i), accessor());
+        return ByteBufferAccessor.instance.factory().boundOrBoundary(kind(), newValues);
     }
 
-    public String toString(TableMetadata metadata)
+    default String toString(TableMetadata metadata)
     {
         return toString(metadata.comparator);
     }
 
-    public String toString(ClusteringComparator comparator)
+    default String toString(ClusteringComparator comparator)
     {
         StringBuilder sb = new StringBuilder();
         sb.append(kind()).append('(');
@@ -124,7 +83,7 @@ public abstract class ClusteringBoundOrBoundary extends AbstractBufferClustering
         {
             if (i > 0)
                 sb.append(", ");
-            sb.append(comparator.subtype(i).getString(get(i)));
+            sb.append(comparator.subtype(i).getString(get(i), accessor()));
         }
         return sb.append(')').toString();
     }
@@ -137,25 +96,25 @@ public abstract class ClusteringBoundOrBoundary extends AbstractBufferClustering
      * @return the invert of this bound. For instance, if this bound is an exlusive start, this return
      * an inclusive end with the same values.
      */
-    public abstract ClusteringBoundOrBoundary invert();
+    public abstract ClusteringBoundOrBoundary<V> invert();
 
     public static class Serializer
     {
-        public void serialize(ClusteringBoundOrBoundary bound, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
+        public <T> void serialize(ClusteringBoundOrBoundary<T> bound, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
         {
             out.writeByte(bound.kind().ordinal());
             out.writeShort(bound.size());
             ClusteringPrefix.serializer.serializeValuesWithoutSize(bound, out, version, types);
         }
 
-        public long serializedSize(ClusteringBoundOrBoundary bound, int version, List<AbstractType<?>> types)
+        public <T> long serializedSize(ClusteringBoundOrBoundary<T> bound, int version, List<AbstractType<?>> types)
         {
             return 1 // kind ordinal
                  + TypeSizes.sizeof((short)bound.size())
                  + ClusteringPrefix.serializer.valuesWithoutSizeSerializedSize(bound, version, types);
         }
 
-        public ClusteringBoundOrBoundary deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
+        public ClusteringBoundOrBoundary<byte[]> deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
         {
             Kind kind = Kind.values()[in.readByte()];
             return deserializeValues(in, kind, version, types);
@@ -170,14 +129,14 @@ public abstract class ClusteringBoundOrBoundary extends AbstractBufferClustering
             ClusteringPrefix.serializer.skipValuesWithoutSize(in, size, version, types);
         }
 
-        public ClusteringBoundOrBoundary deserializeValues(DataInputPlus in, Kind kind, int version, List<AbstractType<?>> types) throws IOException
+        public ClusteringBoundOrBoundary<byte[]> deserializeValues(DataInputPlus in, Kind kind, int version, List<AbstractType<?>> types) throws IOException
         {
             int size = in.readUnsignedShort();
             if (size == 0)
-                return kind.isStart() ? ClusteringBound.BOTTOM : ClusteringBound.TOP;
+                return ByteArrayAccessor.factory.bound(kind);
 
-            ByteBuffer[] values = ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, size, version, types);
-            return create(kind, values);
+            byte[][] values = ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, size, version, types);
+            return ByteArrayAccessor.factory.boundOrBoundary(kind, values);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/db/ClusteringBoundary.java b/src/java/org/apache/cassandra/db/ClusteringBoundary.java
index b1dcdb6..3e50f52 100644
--- a/src/java/org/apache/cassandra/db/ClusteringBoundary.java
+++ b/src/java/org/apache/cassandra/db/ClusteringBoundary.java
@@ -20,54 +20,21 @@
  */
 package org.apache.cassandra.db;
 
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-
 /**
  * The threshold between two different ranges, i.e. a shortcut for the combination of two ClusteringBounds -- one
  * specifying the end of one of the ranges, and its (implicit) complement specifying the beginning of the other.
  */
-public class ClusteringBoundary extends ClusteringBoundOrBoundary
+public interface ClusteringBoundary<V> extends ClusteringBoundOrBoundary<V>
 {
-    protected ClusteringBoundary(Kind kind, ByteBuffer[] values)
-    {
-        super(kind, values);
-    }
-
-    public ClusteringPrefix minimize()
-    {
-        if (!ByteBufferUtil.canMinimize(values))
-            return this;
-        return new ClusteringBoundary(kind, ByteBufferUtil.minimizeBuffers(values));
-    }
-
-    public static ClusteringBoundary create(Kind kind, ByteBuffer[] values)
-    {
-        assert kind.isBoundary();
-        return new ClusteringBoundary(kind, values);
-    }
-
     @Override
-    public ClusteringBoundary invert()
-    {
-        return create(kind().invert(), values);
-    }
+    public ClusteringBoundary<V> invert();
 
-    @Override
-    public ClusteringBoundary copy(AbstractAllocator allocator)
-    {
-        return (ClusteringBoundary) super.copy(allocator);
-    }
+    public ClusteringBound<V> openBound(boolean reversed);
 
-    public ClusteringBound openBound(boolean reversed)
-    {
-        return ClusteringBound.create(kind.openBoundOfBoundary(reversed), values);
-    }
+    public ClusteringBound<V> closeBound(boolean reversed);
 
-    public ClusteringBound closeBound(boolean reversed)
+    public static <V> ClusteringBoundary<V> create(ClusteringBound.Kind kind, ClusteringPrefix<V> from)
     {
-        return ClusteringBound.create(kind.closeBoundOfBoundary(reversed), values);
+        return from.accessor().factory().boundary(kind, from.getRawValues());
     }
 }
diff --git a/src/java/org/apache/cassandra/db/ClusteringComparator.java b/src/java/org/apache/cassandra/db/ClusteringComparator.java
index 50cf5bf..fdc4508 100644
--- a/src/java/org/apache/cassandra/db/ClusteringComparator.java
+++ b/src/java/org/apache/cassandra/db/ClusteringComparator.java
@@ -25,6 +25,7 @@ import java.util.Objects;
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
 
+import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.serializers.MarshalException;
@@ -46,7 +47,8 @@ public class ClusteringComparator implements Comparator<Clusterable>
     private final Comparator<IndexInfo> indexReverseComparator;
     private final Comparator<Clusterable> reverseComparator;
 
-    private final Comparator<Row> rowComparator = (r1, r2) -> compare(r1.clustering(), r2.clustering());
+    private final Comparator<Row> rowComparator = (r1, r2) -> compare((ClusteringPrefix<?>) r1.clustering(),
+                                                                      (ClusteringPrefix<?>) r2.clustering());
 
     public ClusteringComparator(AbstractType<?>... clusteringTypes)
     {
@@ -58,8 +60,10 @@ public class ClusteringComparator implements Comparator<Clusterable>
         // copy the list to ensure despatch is monomorphic
         this.clusteringTypes = ImmutableList.copyOf(clusteringTypes);
 
-        this.indexComparator = (o1, o2) -> ClusteringComparator.this.compare(o1.lastName, o2.lastName);
-        this.indexReverseComparator = (o1, o2) -> ClusteringComparator.this.compare(o1.firstName, o2.firstName);
+        this.indexComparator = (o1, o2) -> ClusteringComparator.this.compare((ClusteringPrefix<?>) o1.lastName,
+                                                                             (ClusteringPrefix<?>) o2.lastName);
+        this.indexReverseComparator = (o1, o2) -> ClusteringComparator.this.compare((ClusteringPrefix<?>) o1.firstName,
+                                                                                    (ClusteringPrefix<?>) o2.firstName);
         this.reverseComparator = (c1, c2) -> ClusteringComparator.this.compare(c2, c1);
         for (AbstractType<?> type : clusteringTypes)
             type.checkComparable(); // this should already be enforced by TableMetadata.Builder.addColumn, but we check again for other constructors
@@ -102,7 +106,7 @@ public class ClusteringComparator implements Comparator<Clusterable>
      *
      * @return the newly created clustering.
      */
-    public Clustering make(Object... values)
+    public Clustering<?> make(Object... values)
     {
         if (values.length != size())
             throw new IllegalArgumentException(String.format("Invalid number of components, expecting %d but got %d", size(), values.length));
@@ -120,10 +124,10 @@ public class ClusteringComparator implements Comparator<Clusterable>
 
     public int compare(Clusterable c1, Clusterable c2)
     {
-        return compare(c1.clustering(), c2.clustering());
+        return compare((ClusteringPrefix<?>) c1.clustering(), (ClusteringPrefix<?>) c2.clustering());
     }
 
-    public int compare(ClusteringPrefix c1, ClusteringPrefix c2)
+    public <V1, V2> int compare(ClusteringPrefix<V1> c1, ClusteringPrefix<V2> c2)
     {
         int s1 = c1.size();
         int s2 = c2.size();
@@ -131,7 +135,7 @@ public class ClusteringComparator implements Comparator<Clusterable>
 
         for (int i = 0; i < minSize; i++)
         {
-            int cmp = compareComponent(i, c1.get(i), c2.get(i));
+            int cmp = compareComponent(i, c1.get(i), c1.accessor(), c2.get(i), c2.accessor());
             if (cmp != 0)
                 return cmp;
         }
@@ -142,7 +146,7 @@ public class ClusteringComparator implements Comparator<Clusterable>
         return s1 < s2 ? c1.kind().comparedToClustering : -c2.kind().comparedToClustering;
     }
 
-    public int compare(Clustering c1, Clustering c2)
+    public <V1, V2> int compare(Clustering<V1> c1, Clustering<V2> c2)
     {
         return compare(c1, c2, size());
     }
@@ -156,25 +160,30 @@ public class ClusteringComparator implements Comparator<Clusterable>
      * @return a negative integer, zero, or a positive integer as the first argument is less than,
      * equal to, or greater than the second.
      */
-    public int compare(Clustering c1, Clustering c2, int size)
+    public <V1, V2> int compare(Clustering<V1> c1, Clustering<V2> c2, int size)
     {
         for (int i = 0; i < size; i++)
         {
-            int cmp = compareComponent(i, c1.get(i), c2.get(i));
+            int cmp = compareComponent(i, c1.get(i), c1.accessor(), c2.get(i), c2.accessor());
             if (cmp != 0)
                 return cmp;
         }
         return 0;
     }
 
-    public int compareComponent(int i, ByteBuffer v1, ByteBuffer v2)
+    public <V1, V2> int compareComponent(int i, V1 v1, ValueAccessor<V1> accessor1, V2 v2, ValueAccessor<V2> accessor2)
     {
         if (v1 == null)
             return v2 == null ? 0 : -1;
         if (v2 == null)
             return 1;
 
-        return clusteringTypes.get(i).compare(v1, v2);
+        return clusteringTypes.get(i).compare(v1, accessor1, v2, accessor2);
+    }
+
+    public <V1, V2> int compareComponent(int i, ClusteringPrefix<V1> v1, ClusteringPrefix<V2> v2)
+    {
+        return compareComponent(i, v1.get(i), v1.accessor(), v2.get(i), v2.accessor());
     }
 
     /**
@@ -212,13 +221,14 @@ public class ClusteringComparator implements Comparator<Clusterable>
      *
      * @throws MarshalException if {@code clustering} contains some invalid data.
      */
-    public void validate(ClusteringPrefix clustering)
+    public <T> void validate(ClusteringPrefix<T> clustering)
     {
+        ValueAccessor<T> accessor = clustering.accessor();
         for (int i = 0; i < clustering.size(); i++)
         {
-            ByteBuffer value = clustering.get(i);
+            T value = clustering.get(i);
             if (value != null)
-                subtype(i).validate(value);
+                subtype(i).validate(value, accessor);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/db/ClusteringPrefix.java b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
index 357d746..a1291c8 100644
--- a/src/java/org/apache/cassandra/db/ClusteringPrefix.java
+++ b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
@@ -23,12 +23,16 @@ import java.util.*;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.util.DataInputPlus;
 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;
 
 /**
@@ -44,7 +48,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
  *   3) {@code ClusteringBoundary} represents the threshold between two adjacent range tombstones.
  * See those classes for more details.
  */
-public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
+public interface ClusteringPrefix<V> extends IMeasurableMemory, Clusterable<V>
 {
     public static final Serializer serializer = new Serializer();
 
@@ -195,6 +199,16 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
         }
     }
 
+    default boolean isBottom()
+    {
+        return kind() == Kind.INCL_START_BOUND && size() == 0;
+    }
+
+    default boolean isTop()
+    {
+        return kind() == Kind.INCL_END_BOUND && size() == 0;
+    }
+
     public Kind kind();
 
     /**
@@ -206,6 +220,11 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
      */
     public int size();
 
+    default boolean isEmpty()
+    {
+        return size() == 0;
+    }
+
     /**
      * Retrieves the ith value of this prefix.
      *
@@ -213,14 +232,40 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
      *
      * @return the ith value of this prefix. Note that a value can be {@code null}.
      */
-    public ByteBuffer get(int i);
+    public V get(int i);
+
+    public ValueAccessor<V> accessor();
+
+    default ByteBuffer bufferAt(int i)
+    {
+        return accessor().toBuffer(get(i));
+    }
+
+    default String stringAt(int i, ClusteringComparator comparator)
+    {
+        return comparator.subtype(i).getString(get(i), accessor());
+    }
+
+    default void validate(int i, ClusteringComparator comparator)
+    {
+        comparator.subtype(i).validate(get(i), accessor());
+    }
 
     /**
      * Adds the data of this clustering prefix to the provided Digest instance.
      *
      * @param digest the Digest instance to which to add this prefix.
      */
-    public void digest(Digest digest);
+    default void digest(Digest digest)
+    {
+        for (int i = 0; i < size(); i++)
+        {
+            V value = get(i);
+            if (value != null)
+                digest.update(value, accessor());
+        }
+        digest.updateWithByte(kind().ordinal());
+    }
 
     /**
      * The size of the data hold by this prefix.
@@ -228,7 +273,16 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
      * @return the size of the data hold by this prefix (this is not the size of the object in memory, just
      * the size of the data it stores).
      */
-    public int dataSize();
+    default int dataSize()
+    {
+        int size = 0;
+        for (int i = 0; i < size(); i++)
+        {
+            V v = get(i);
+            size += v == null ? 0 : accessor().size(v);
+        }
+        return size;
+    }
 
     /**
      * Generates a proper string representation of the prefix.
@@ -247,12 +301,12 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
     default ByteBuffer serializeAsPartitionKey()
     {
         if (size() == 1)
-            return get(0);
+            return accessor().toBuffer(get(0));
 
         ByteBuffer[] values = new ByteBuffer[size()];
         for (int i = 0; i < size(); i++)
-            values[i] = get(i);
-        return CompositeType.build(values);
+            values[i] = accessor().toBuffer(get(i));
+        return CompositeType.build(ByteBufferAccessor.instance, values);
     }
     /**
      * The values of this prefix as an array.
@@ -263,28 +317,30 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
      *
      * @return the values for this prefix as an array.
      */
-    public ByteBuffer[] getRawValues();
+    public V[] getRawValues();
+
+    public ByteBuffer[] getBufferArray();
 
     /**
      * If the prefix contains byte buffers that can be minimized (see {@link ByteBufferUtil#minimalBufferFor(ByteBuffer)}),
      * this will return a copy of the prefix with minimized values, otherwise it returns itself.
      */
-    public ClusteringPrefix minimize();
+    public ClusteringPrefix<V> minimize();
 
     public static class Serializer
     {
-        public void serialize(ClusteringPrefix clustering, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
+        public void serialize(ClusteringPrefix<?> clustering, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
         {
             // We shouldn't serialize static clusterings
             assert clustering.kind() != Kind.STATIC_CLUSTERING;
             if (clustering.kind() == Kind.CLUSTERING)
             {
                 out.writeByte(clustering.kind().ordinal());
-                Clustering.serializer.serialize((Clustering)clustering, out, version, types);
+                Clustering.serializer.serialize((Clustering<?>)clustering, out, version, types);
             }
             else
             {
-                ClusteringBoundOrBoundary.serializer.serialize((ClusteringBoundOrBoundary)clustering, out, version, types);
+                ClusteringBoundOrBoundary.serializer.serialize((ClusteringBoundOrBoundary<?>)clustering, out, version, types);
             }
         }
 
@@ -299,7 +355,7 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
                 ClusteringBoundOrBoundary.serializer.skipValues(in, kind, version, types);
         }
 
-        public ClusteringPrefix deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
+        public ClusteringPrefix<byte[]> deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
         {
             Kind kind = Kind.values()[in.readByte()];
             // We shouldn't serialize static clusterings
@@ -310,20 +366,21 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
                 return ClusteringBoundOrBoundary.serializer.deserializeValues(in, kind, version, types);
         }
 
-        public long serializedSize(ClusteringPrefix clustering, int version, List<AbstractType<?>> types)
+        public long serializedSize(ClusteringPrefix<?> clustering, int version, List<AbstractType<?>> types)
         {
             // We shouldn't serialize static clusterings
             assert clustering.kind() != Kind.STATIC_CLUSTERING;
             if (clustering.kind() == Kind.CLUSTERING)
-                return 1 + Clustering.serializer.serializedSize((Clustering)clustering, version, types);
+                return 1 + Clustering.serializer.serializedSize((Clustering<?>)clustering, version, types);
             else
-                return ClusteringBoundOrBoundary.serializer.serializedSize((ClusteringBoundOrBoundary)clustering, version, types);
+                return ClusteringBoundOrBoundary.serializer.serializedSize((ClusteringBoundOrBoundary<?>)clustering, version, types);
         }
 
-        void serializeValuesWithoutSize(ClusteringPrefix clustering, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
+        <V> void serializeValuesWithoutSize(ClusteringPrefix<V> clustering, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
         {
             int offset = 0;
             int clusteringSize = clustering.size();
+            ValueAccessor<V> accessor = clustering.accessor();
             // serialize in batches of 32, to avoid garbage when deserializing headers
             while (offset < clusteringSize)
             {
@@ -335,15 +392,15 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
                 out.writeUnsignedVInt(makeHeader(clustering, offset, limit));
                 while (offset < limit)
                 {
-                    ByteBuffer v = clustering.get(offset);
-                    if (v != null && v.hasRemaining())
-                        types.get(offset).writeValue(v, out);
+                    V v = clustering.get(offset);
+                    if (v != null && !accessor.isEmpty(v))
+                        types.get(offset).writeValue(v, accessor, out);
                     offset++;
                 }
             }
         }
 
-        long valuesWithoutSizeSerializedSize(ClusteringPrefix clustering, int version, List<AbstractType<?>> types)
+        <V> long valuesWithoutSizeSerializedSize(ClusteringPrefix<V> clustering, int version, List<AbstractType<?>> types)
         {
             long result = 0;
             int offset = 0;
@@ -354,22 +411,23 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
                 result += TypeSizes.sizeofUnsignedVInt(makeHeader(clustering, offset, limit));
                 offset = limit;
             }
+            ValueAccessor<V> accessor = clustering.accessor();
             for (int i = 0; i < clusteringSize; i++)
             {
-                ByteBuffer v = clustering.get(i);
-                if (v == null || !v.hasRemaining())
+                V v = clustering.get(i);
+                if (v == null || accessor.isEmpty(v))
                     continue; // handled in the header
 
-                result += types.get(i).writtenLength(v);
+                result += types.get(i).writtenLength(v, accessor);
             }
             return result;
         }
 
-        ByteBuffer[] deserializeValuesWithoutSize(DataInputPlus in, int size, int version, List<AbstractType<?>> types) throws IOException
+        byte[][] deserializeValuesWithoutSize(DataInputPlus in, int size, int version, List<AbstractType<?>> types) throws IOException
         {
             // Callers of this method should handle the case where size = 0 (in all case we want to return a special value anyway).
             assert size > 0;
-            ByteBuffer[] values = new ByteBuffer[size];
+            byte[][] values = new byte[size][];
             int offset = 0;
             while (offset < size)
             {
@@ -378,8 +436,9 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
                 while (offset < limit)
                 {
                     values[offset] = isNull(header, offset)
-                                ? null
-                                : (isEmpty(header, offset) ? ByteBufferUtil.EMPTY_BYTE_BUFFER : types.get(offset).readValue(in, DatabaseDescriptor.getMaxValueSize()));
+                                     ? null
+                                     : (isEmpty(header, offset) ? ByteArrayUtil.EMPTY_BYTE_ARRAY
+                                                                : types.get(offset).readArray(in, DatabaseDescriptor.getMaxValueSize()));
                     offset++;
                 }
             }
@@ -410,16 +469,17 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
          * empty values too. So for that, every clustering prefix includes a "header" that contains 2 bits per element in the prefix. For each element,
          * those 2 bits encode whether the element is null, empty, or none of those.
          */
-        private static long makeHeader(ClusteringPrefix clustering, int offset, int limit)
+        private static <V> long makeHeader(ClusteringPrefix<V> clustering, int offset, int limit)
         {
             long header = 0;
+            ValueAccessor<V> accessor = clustering.accessor();
             for (int i = offset ; i < limit ; i++)
             {
-                ByteBuffer v = clustering.get(i);
+                V v = clustering.get(i);
                 // no need to do modulo arithmetic for i, since the left-shift execute on the modulus of RH operand by definition
                 if (v == null)
                     header |= (1L << (i * 2) + 1);
-                else if (!v.hasRemaining())
+                else if (accessor.isEmpty(v))
                     header |= (1L << (i * 2));
             }
             return header;
@@ -461,7 +521,8 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
         private int nextSize;
         private ClusteringPrefix.Kind nextKind;
         private int deserializedSize;
-        private ByteBuffer[] nextValues;
+        private byte[][] nextValues;
+        private final ValueAccessor<byte[]> accessor = ByteArrayAccessor.instance;
 
         public Deserializer(ClusteringComparator comparator, DataInputPlus in, SerializationHeader header)
         {
@@ -486,12 +547,12 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
             // nextValues is of the proper size. Note that the 2nd condition may not hold for range tombstone bounds, but all
             // rows have a fixed size clustering, so we'll still save in the common case.
             if (nextValues == null || nextValues.length != nextSize)
-                this.nextValues = new ByteBuffer[nextSize];
+                this.nextValues = new byte[nextSize][];
         }
 
-        public int compareNextTo(ClusteringBoundOrBoundary bound) throws IOException
+        public <T> int compareNextTo(ClusteringBoundOrBoundary<T> bound) throws IOException
         {
-            if (bound == ClusteringBound.TOP)
+            if (bound.isTop())
                 return -1;
 
             for (int i = 0; i < bound.size(); i++)
@@ -499,7 +560,7 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
                 if (!hasComponent(i))
                     return nextKind.comparedToClustering;
 
-                int cmp = comparator.compareComponent(i, nextValues[i], bound.get(i));
+                int cmp = comparator.compareComponent(i, nextValues[i], accessor, bound.get(i), bound.accessor());
                 if (cmp != 0)
                     return cmp;
             }
@@ -533,7 +594,8 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
             int i = deserializedSize++;
             nextValues[i] = Serializer.isNull(nextHeader, i)
                           ? null
-                          : (Serializer.isEmpty(nextHeader, i) ? ByteBufferUtil.EMPTY_BYTE_BUFFER : serializationHeader.clusteringTypes().get(i).readValue(in, DatabaseDescriptor.getMaxValueSize()));
+                          : (Serializer.isEmpty(nextHeader, i) ? ByteArrayUtil.EMPTY_BYTE_ARRAY
+                                                               : serializationHeader.clusteringTypes().get(i).readArray(in, DatabaseDescriptor.getMaxValueSize()));
             return true;
         }
 
@@ -543,20 +605,20 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
                 continue;
         }
 
-        public ClusteringBoundOrBoundary deserializeNextBound() throws IOException
+        public ClusteringBoundOrBoundary<byte[]> deserializeNextBound() throws IOException
         {
             assert !nextIsRow;
             deserializeAll();
-            ClusteringBoundOrBoundary bound = ClusteringBoundOrBoundary.create(nextKind, nextValues);
+            ClusteringBoundOrBoundary<byte[]> bound = accessor.factory().boundOrBoundary(nextKind, nextValues);
             nextValues = null;
             return bound;
         }
 
-        public Clustering deserializeNextClustering() throws IOException
+        public Clustering<byte[]> deserializeNextClustering() throws IOException
         {
             assert nextIsRow;
             deserializeAll();
-            Clustering clustering = Clustering.make(nextValues);
+            Clustering<byte[]> clustering = accessor.factory().clustering(nextValues);
             nextValues = null;
             return clustering;
         }
@@ -574,4 +636,44 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
             return nextKind;
         }
     }
+
+    public static <V> int hashCode(ClusteringPrefix<V> prefix)
+    {
+        int result = 31;
+        for (int i = 0; i < prefix.size(); i++)
+            result += 31 * prefix.accessor().hashCode(prefix.get(i));
+        return 31 * result + Objects.hashCode(prefix.kind());
+    }
+
+    static <V1, V2> boolean equals(ClusteringPrefix<V1> left, ClusteringPrefix<V2> right)
+    {
+        if (left.kind() != right.kind() || left.size() != right.size())
+            return false;
+
+        for (int i = 0; i < left.size(); i++)
+        {
+            V1 lVal = left.get(i);
+            V2 rVal = right.get(i);
+
+            if (lVal == null && rVal == null)
+                continue;
+
+            if (lVal == null || rVal == null)
+                return false;
+
+            if (!ValueAccessor.equals(lVal, left.accessor(), rVal, right.accessor()))
+                return false;
+        }
+
+        return true;
+    }
+
+    public static boolean equals(ClusteringPrefix<?> prefix, Object o)
+    {
+        if(!(o instanceof ClusteringPrefix))
+            return false;
+
+        return equals(prefix, (ClusteringPrefix<?>) o);
+    }
+
 }
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 824005b..cb6c86b 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -2057,14 +2057,14 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         invalidateCachedPartition(new RowCacheKey(metadata(), key));
     }
 
-    public ClockAndCount getCachedCounter(ByteBuffer partitionKey, Clustering clustering, ColumnMetadata column, CellPath path)
+    public ClockAndCount getCachedCounter(ByteBuffer partitionKey, Clustering<?> clustering, ColumnMetadata column, CellPath path)
     {
         if (CacheService.instance.counterCache.getCapacity() == 0L) // counter cache disabled.
             return null;
         return CacheService.instance.counterCache.get(CounterCacheKey.create(metadata(), partitionKey, clustering, column, path));
     }
 
-    public void putCachedCounter(ByteBuffer partitionKey, Clustering clustering, ColumnMetadata column, CellPath path, ClockAndCount clockAndCount)
+    public void putCachedCounter(ByteBuffer partitionKey, Clustering<?> clustering, ColumnMetadata column, CellPath path, ClockAndCount clockAndCount)
     {
         if (CacheService.instance.counterCache.getCapacity() == 0L) // counter cache disabled.
             return;
diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java b/src/java/org/apache/cassandra/db/ColumnIndex.java
index e11f784..b872300 100644
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@ -65,8 +65,8 @@ public class ColumnIndex
     private int written;
     private long previousRowStart;
 
-    private ClusteringPrefix firstClustering;
-    private ClusteringPrefix lastClustering;
+    private ClusteringPrefix<?> firstClustering;
+    private ClusteringPrefix<?> lastClustering;
 
     private DeletionTime openMarker;
 
diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java
index 722ad73..bc0cd85 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -30,6 +30,7 @@ import com.google.common.collect.PeekingIterator;
 import com.google.common.util.concurrent.Striped;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.partitions.*;
@@ -221,7 +222,7 @@ public class CounterMutation implements IMutation
     private void updateWithCurrentValue(PartitionUpdate.CounterMark mark, ClockAndCount currentValue, ColumnFamilyStore cfs)
     {
         long clock = Math.max(FBUtilities.timestampMicros(), currentValue.clock + 1L);
-        long count = currentValue.count + CounterContext.instance().total(mark.value());
+        long count = currentValue.count + CounterContext.instance().total(mark.value(), ByteBufferAccessor.instance);
 
         mark.setValue(CounterContext.instance().createGlobal(CounterId.getLocalId(), clock, count));
 
@@ -249,7 +250,7 @@ public class CounterMutation implements IMutation
     private void updateWithCurrentValuesFromCFS(List<PartitionUpdate.CounterMark> marks, ColumnFamilyStore cfs)
     {
         ColumnFilter.Builder builder = ColumnFilter.selectionBuilder();
-        BTreeSet.Builder<Clustering> names = BTreeSet.builder(cfs.metadata().comparator);
+        BTreeSet.Builder<Clustering<?>> names = BTreeSet.builder(cfs.metadata().comparator);
         for (PartitionUpdate.CounterMark mark : marks)
         {
             if (mark.clustering() != Clustering.STATIC_CLUSTERING)
@@ -279,7 +280,7 @@ public class CounterMutation implements IMutation
         }
     }
 
-    private int compare(Clustering c1, Clustering c2, ColumnFamilyStore cfs)
+    private int compare(Clustering<?> c1, Clustering<?> c2, ColumnFamilyStore cfs)
     {
         if (c1 == Clustering.STATIC_CLUSTERING)
             return c2 == Clustering.STATIC_CLUSTERING ? 0 : -1;
@@ -302,10 +303,10 @@ public class CounterMutation implements IMutation
         while (cmp == 0)
         {
             PartitionUpdate.CounterMark mark = markIter.next();
-            Cell cell = mark.path() == null ? row.getCell(mark.column()) : row.getCell(mark.column(), mark.path());
+            Cell<?> cell = mark.path() == null ? row.getCell(mark.column()) : row.getCell(mark.column(), mark.path());
             if (cell != null)
             {
-                updateWithCurrentValue(mark, CounterContext.instance().getLocalClockAndCount(cell.value()), cfs);
+                updateWithCurrentValue(mark, CounterContext.instance().getLocalClockAndCount(cell.buffer()), cfs);
                 markIter.remove();
             }
             if (!markIter.hasNext())
diff --git a/src/java/org/apache/cassandra/db/DataRange.java b/src/java/org/apache/cassandra/db/DataRange.java
index aa23f3d..91a62b3 100644
--- a/src/java/org/apache/cassandra/db/DataRange.java
+++ b/src/java/org/apache/cassandra/db/DataRange.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.filter.*;
@@ -232,7 +233,7 @@ public class DataRange
      *
      * @return a new {@code DataRange} suitable for paging {@code this} range given the {@code lastRetuned} result of the previous page.
      */
-    public DataRange forPaging(AbstractBounds<PartitionPosition> range, ClusteringComparator comparator, Clustering lastReturned, boolean inclusive)
+    public DataRange forPaging(AbstractBounds<PartitionPosition> range, ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive)
     {
         return new Paging(range, clusteringIndexFilter, comparator, lastReturned, inclusive);
     }
@@ -338,13 +339,13 @@ public class DataRange
     public static class Paging extends DataRange
     {
         private final ClusteringComparator comparator;
-        private final Clustering lastReturned;
+        private final Clustering<?> lastReturned;
         private final boolean inclusive;
 
         private Paging(AbstractBounds<PartitionPosition> range,
                        ClusteringIndexFilter filter,
                        ClusteringComparator comparator,
-                       Clustering lastReturned,
+                       Clustering<?> lastReturned,
                        boolean inclusive)
         {
             super(range, filter);
@@ -380,7 +381,7 @@ public class DataRange
         /**
          * @return the last Clustering that was returned (in the previous page)
          */
-        public Clustering getLastReturned()
+        public Clustering<?> getLastReturned()
         {
             return lastReturned;
         }
@@ -430,7 +431,7 @@ public class DataRange
             if (in.readBoolean())
             {
                 ClusteringComparator comparator = metadata.comparator;
-                Clustering lastReturned = Clustering.serializer.deserialize(in, version, comparator.subtypes());
+                Clustering<byte[]> lastReturned = Clustering.serializer.deserialize(in, version, comparator.subtypes());
                 boolean inclusive = in.readBoolean();
                 return new Paging(range, filter, comparator, lastReturned, inclusive);
             }
diff --git a/src/java/org/apache/cassandra/db/DeletionInfo.java b/src/java/org/apache/cassandra/db/DeletionInfo.java
index 5bec812..b8327e9 100644
--- a/src/java/org/apache/cassandra/db/DeletionInfo.java
+++ b/src/java/org/apache/cassandra/db/DeletionInfo.java
@@ -52,7 +52,7 @@ public interface DeletionInfo extends IMeasurableMemory
 
     public Iterator<RangeTombstone> rangeIterator(Slice slice, boolean reversed);
 
-    public RangeTombstone rangeCovering(Clustering name);
+    public RangeTombstone rangeCovering(Clustering<?> name);
 
     public void collectStats(EncodingStats.Collector collector);
 
diff --git a/src/java/org/apache/cassandra/db/DeletionTime.java b/src/java/org/apache/cassandra/db/DeletionTime.java
index b2d9343..f1471fd 100644
--- a/src/java/org/apache/cassandra/db/DeletionTime.java
+++ b/src/java/org/apache/cassandra/db/DeletionTime.java
@@ -140,7 +140,7 @@ public class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory
         return deletes(info.timestamp());
     }
 
-    public boolean deletes(Cell cell)
+    public boolean deletes(Cell<?> cell)
     {
         return deletes(cell.timestamp());
     }
diff --git a/src/java/org/apache/cassandra/db/Digest.java b/src/java/org/apache/cassandra/db/Digest.java
index bac6386..6a4ecd8 100644
--- a/src/java/org/apache/cassandra/db/Digest.java
+++ b/src/java/org/apache/cassandra/db/Digest.java
@@ -24,6 +24,7 @@ import com.google.common.hash.Hasher;
 import com.google.common.hash.Hashing;
 
 import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.utils.FastByteOperations;
 
 public class Digest
@@ -63,15 +64,15 @@ public class Digest
         return new Digest(Hashing.crc32c().newHasher())
         {
             @Override
-            public Digest updateWithCounterContext(ByteBuffer context)
+            public <V> Digest updateWithCounterContext(V context, ValueAccessor<V> accessor)
             {
                 // for the purposes of repaired data tracking on the read path, exclude
                 // contexts with legacy shards as these may be irrevocably different on
                 // different replicas
-                if (CounterContext.instance().hasLegacyShards(context))
+                if (CounterContext.instance().hasLegacyShards(context, accessor))
                     return this;
 
-                return super.updateWithCounterContext(context);
+                return super.updateWithCounterContext(context, accessor);
             }
         };
     }
@@ -88,6 +89,12 @@ public class Digest
         return this;
     }
 
+    public <V> Digest update(V input, ValueAccessor<V> accessor)
+    {
+        accessor.digest(input, this);
+        return this;
+    }
+
     /**
      * Update the digest with the bytes from the supplied buffer. This does
      * not modify the position of the supplied buffer, so callers are not
@@ -103,7 +110,7 @@ public class Digest
      * not modify the position of the supplied buffer, so callers are not
      * required to duplicate() the source buffer before calling
      */
-    private Digest update(ByteBuffer input, int pos, int len)
+    public Digest update(ByteBuffer input, int pos, int len)
     {
         if (len <= 0)
             return this;
@@ -138,15 +145,15 @@ public class Digest
      * nodes. This means in particular that we always have:
      *  updateDigest(ctx) == updateDigest(clearAllLocal(ctx))
      */
-    public Digest updateWithCounterContext(ByteBuffer context)
+    public <V> Digest updateWithCounterContext(V context, ValueAccessor<V> accessor)
     {
         // context can be empty due to the optimization from CASSANDRA-10657
-        if (!context.hasRemaining())
+        if (accessor.isEmpty(context))
             return this;
 
-        int pos = context.position() + CounterContext.headerLength(context);
-        int len = context.limit() - pos;
-        update(context, pos, len);
+        int pos = CounterContext.headerLength(context, accessor);
+        int len = accessor.size(context) - pos;
+        accessor.digest(context, pos, len, this);
         return this;
     }
 
diff --git a/src/java/org/apache/cassandra/db/MultiCBuilder.java b/src/java/org/apache/cassandra/db/MultiCBuilder.java
index c4cff02..0b5625b 100644
--- a/src/java/org/apache/cassandra/db/MultiCBuilder.java
+++ b/src/java/org/apache/cassandra/db/MultiCBuilder.java
@@ -163,7 +163,7 @@ public abstract class MultiCBuilder
      *
      * @return the clusterings
      */
-    public abstract NavigableSet<Clustering> build();
+    public abstract NavigableSet<Clustering<?>> build();
 
     /**
      * Builds the <code>ClusteringBound</code>s for slice restrictions.
@@ -174,10 +174,10 @@ public abstract class MultiCBuilder
      * @param columnDefs the columns of the slice restriction
      * @return the <code>ClusteringBound</code>s
      */
-    public abstract NavigableSet<ClusteringBound> buildBoundForSlice(boolean isStart,
-                                                                 boolean isInclusive,
-                                                                 boolean isOtherBoundInclusive,
-                                                                 List<ColumnMetadata> columnDefs);
+    public abstract NavigableSet<ClusteringBound<?>> buildBoundForSlice(boolean isStart,
+                                                                        boolean isInclusive,
+                                                                        boolean isOtherBoundInclusive,
+                                                                        List<ColumnMetadata> columnDefs);
 
     /**
      * Builds the <code>ClusteringBound</code>s
@@ -186,7 +186,7 @@ public abstract class MultiCBuilder
      * @param isInclusive specify if the bound is inclusive or not
      * @return the <code>ClusteringBound</code>s
      */
-    public abstract NavigableSet<ClusteringBound> buildBound(boolean isStart, boolean isInclusive);
+    public abstract NavigableSet<ClusteringBound<?>> buildBound(boolean isStart, boolean isInclusive);
 
     /**
      * Checks if some elements can still be added to the clusterings.
@@ -252,7 +252,7 @@ public abstract class MultiCBuilder
             return addEachElementToAll(values.get(0));
         }
 
-        public NavigableSet<Clustering> build()
+        public NavigableSet<Clustering<?>> build()
         {
             built = true;
 
@@ -263,15 +263,15 @@ public abstract class MultiCBuilder
         }
 
         @Override
-        public NavigableSet<ClusteringBound> buildBoundForSlice(boolean isStart,
-                                                                boolean isInclusive,
-                                                                boolean isOtherBoundInclusive,
-                                                                List<ColumnMetadata> columnDefs)
+        public NavigableSet<ClusteringBound<?>> buildBoundForSlice(boolean isStart,
+                                                                   boolean isInclusive,
+                                                                   boolean isOtherBoundInclusive,
+                                                                   List<ColumnMetadata> columnDefs)
         {
             return buildBound(isStart, columnDefs.get(0).isReversedType() ? isOtherBoundInclusive : isInclusive);
         }
 
-        public NavigableSet<ClusteringBound> buildBound(boolean isStart, boolean isInclusive)
+        public NavigableSet<ClusteringBound<?>> buildBound(boolean isStart, boolean isInclusive)
         {
             built = true;
 
@@ -279,13 +279,13 @@ public abstract class MultiCBuilder
                 return BTreeSet.empty(comparator);
 
             if (size == 0)
-                return BTreeSet.of(comparator, isStart ? ClusteringBound.BOTTOM : ClusteringBound.TOP);
+                return BTreeSet.of(comparator, isStart ? BufferClusteringBound.BOTTOM : BufferClusteringBound.TOP);
 
             ByteBuffer[] newValues = size == elements.length
                                    ? elements
                                    : Arrays.copyOf(elements, size);
 
-            return BTreeSet.of(comparator, ClusteringBound.create(ClusteringBound.boundKind(isStart, isInclusive), newValues));
+            return BTreeSet.of(comparator, BufferClusteringBound.create(ClusteringBound.boundKind(isStart, isInclusive), newValues));
         }
     }
 
@@ -397,7 +397,7 @@ public abstract class MultiCBuilder
             return this;
         }
 
-        public NavigableSet<Clustering> build()
+        public NavigableSet<Clustering<?>> build()
         {
             built = true;
 
@@ -409,7 +409,7 @@ public abstract class MultiCBuilder
             if (elementsList.isEmpty())
                 return BTreeSet.of(builder.comparator(), builder.build());
 
-            BTreeSet.Builder<Clustering> set = BTreeSet.builder(builder.comparator());
+            BTreeSet.Builder<Clustering<?>> set = BTreeSet.builder(builder.comparator());
             for (int i = 0, m = elementsList.size(); i < m; i++)
             {
                 List<ByteBuffer> elements = elementsList.get(i);
@@ -418,10 +418,10 @@ public abstract class MultiCBuilder
             return set.build();
         }
 
-        public NavigableSet<ClusteringBound> buildBoundForSlice(boolean isStart,
-                                                            boolean isInclusive,
-                                                            boolean isOtherBoundInclusive,
-                                                            List<ColumnMetadata> columnDefs)
+        public NavigableSet<ClusteringBound<?>> buildBoundForSlice(boolean isStart,
+                                                                   boolean isInclusive,
+                                                                   boolean isOtherBoundInclusive,
+                                                                   List<ColumnMetadata> columnDefs)
         {
             built = true;
 
@@ -434,7 +434,7 @@ public abstract class MultiCBuilder
                 return BTreeSet.of(comparator, builder.buildBound(isStart, isInclusive));
 
             // Use a TreeSet to sort and eliminate duplicates
-            BTreeSet.Builder<ClusteringBound> set = BTreeSet.builder(comparator);
+            BTreeSet.Builder<ClusteringBound<?>> set = BTreeSet.builder(comparator);
 
             // The first column of the slice might not be the first clustering column (e.g. clustering_0 = ? AND (clustering_1, clustering_2) >= (?, ?)
             int offset = columnDefs.get(0).position();
@@ -469,7 +469,7 @@ public abstract class MultiCBuilder
             return set.build();
         }
 
-        public NavigableSet<ClusteringBound> buildBound(boolean isStart, boolean isInclusive)
+        public NavigableSet<ClusteringBound<?>> buildBound(boolean isStart, boolean isInclusive)
         {
             built = true;
 
@@ -482,7 +482,7 @@ public abstract class MultiCBuilder
                 return BTreeSet.of(comparator, builder.buildBound(isStart, isInclusive));
 
             // Use a TreeSet to sort and eliminate duplicates
-            BTreeSet.Builder<ClusteringBound> set = BTreeSet.builder(comparator);
+            BTreeSet.Builder<ClusteringBound<?>> set = BTreeSet.builder(comparator);
 
             for (int i = 0, m = elementsList.size(); i < m; i++)
             {
diff --git a/src/java/org/apache/cassandra/db/MutableDeletionInfo.java b/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
index 356d763..8544b78 100644
--- a/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
+++ b/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
@@ -160,7 +160,7 @@ public class MutableDeletionInfo implements DeletionInfo
         return ranges == null ? Collections.emptyIterator() : ranges.iterator(slice, reversed);
     }
 
-    public RangeTombstone rangeCovering(Clustering name)
+    public RangeTombstone rangeCovering(Clustering<?> name)
     {
         return ranges == null ? null : ranges.search(name);
     }
@@ -291,8 +291,8 @@ public class MutableDeletionInfo implements DeletionInfo
                 DeletionTime openDeletion = openMarker.openDeletionTime(reversed);
                 assert marker.closeDeletionTime(reversed).equals(openDeletion);
 
-                ClusteringBound open = openMarker.openBound(reversed);
-                ClusteringBound close = marker.closeBound(reversed);
+                ClusteringBound<?> open = openMarker.openBound(reversed);
+                ClusteringBound<?> close = marker.closeBound(reversed);
 
                 Slice slice = reversed ? Slice.make(close, open) : Slice.make(open, close);
                 deletion.add(new RangeTombstone(slice, openDeletion), comparator);
diff --git a/src/java/org/apache/cassandra/db/NativeClustering.java b/src/java/org/apache/cassandra/db/NativeClustering.java
index 780f6a4..0e4c19d 100644
--- a/src/java/org/apache/cassandra/db/NativeClustering.java
+++ b/src/java/org/apache/cassandra/db/NativeClustering.java
@@ -21,12 +21,14 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.MemoryUtil;
 import org.apache.cassandra.utils.memory.NativeAllocator;
 
-public class NativeClustering extends AbstractClusteringPrefix implements Clustering
+public class NativeClustering implements Clustering<ByteBuffer>
 {
     private static final long EMPTY_SIZE = ObjectSizes.measure(new NativeClustering());
 
@@ -34,12 +36,12 @@ public class NativeClustering extends AbstractClusteringPrefix implements Cluste
 
     private NativeClustering() { peer = 0; }
 
-    public ClusteringPrefix minimize()
+    public ClusteringPrefix<ByteBuffer> minimize()
     {
         return this;
     }
 
-    public NativeClustering(NativeAllocator allocator, OpOrder.Group writeOp, Clustering clustering)
+    public NativeClustering(NativeAllocator allocator, OpOrder.Group writeOp, Clustering<?> clustering)
     {
         int count = clustering.size();
         int metadataSize = (count * 2) + 4;
@@ -61,7 +63,7 @@ public class NativeClustering extends AbstractClusteringPrefix implements Cluste
         {
             MemoryUtil.setShort(peer + 2 + i * 2, (short) dataOffset);
 
-            ByteBuffer value = clustering.get(i);
+            ByteBuffer value = clustering.bufferAt(i);
             if (value == null)
             {
                 long boffset = bitmapStart + (i >>> 3);
@@ -84,6 +86,11 @@ public class NativeClustering extends AbstractClusteringPrefix implements Cluste
         return Kind.CLUSTERING;
     }
 
+    public ClusteringPrefix<ByteBuffer> clustering()
+    {
+        return this;
+    }
+
     public int size()
     {
         return MemoryUtil.getShort(peer);
@@ -118,6 +125,17 @@ public class NativeClustering extends AbstractClusteringPrefix implements Cluste
         return values;
     }
 
+    public ByteBuffer[] getBufferArray()
+    {
+        return getRawValues();
+    }
+
+    public ValueAccessor<ByteBuffer> accessor()
+    {
+        // TODO: add a native accessor
+        return ByteBufferAccessor.instance;
+    }
+
     public long unsharedHeapSize()
     {
         return EMPTY_SIZE;
@@ -127,4 +145,16 @@ public class NativeClustering extends AbstractClusteringPrefix implements Cluste
     {
         return EMPTY_SIZE;
     }
+
+    @Override
+    public final int hashCode()
+    {
+        return ClusteringPrefix.hashCode(this);
+    }
+
+    @Override
+    public final boolean equals(Object o)
+    {
+        return ClusteringPrefix.equals(this, o);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadQuery.java b/src/java/org/apache/cassandra/db/PartitionRangeReadQuery.java
index 86ee4c0..816b783 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadQuery.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadQuery.java
@@ -75,7 +75,7 @@ public interface PartitionRangeReadQuery extends ReadQuery
         return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().partitionKeyType);
     }
 
-    default boolean selectsClustering(DecoratedKey key, Clustering clustering)
+    default boolean selectsClustering(DecoratedKey key, Clustering<?> clustering)
     {
         if (clustering == Clustering.STATIC_CLUSTERING)
             return !columnFilter().fetchedColumns().statics.isEmpty();
diff --git a/src/java/org/apache/cassandra/db/RangeTombstoneList.java b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
index acc5f17..293596e 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstoneList.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
@@ -55,15 +55,15 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
 
     // Note: we don't want to use a List for the markedAts and delTimes to avoid boxing. We could
     // use a List for starts and ends, but having arrays everywhere is almost simpler.
-    private ClusteringBound[] starts;
-    private ClusteringBound[] ends;
+    private ClusteringBound<?>[] starts;
+    private ClusteringBound<?>[] ends;
     private long[] markedAts;
     private int[] delTimes;
 
     private long boundaryHeapSize;
     private int size;
 
-    private RangeTombstoneList(ClusteringComparator comparator, ClusteringBound[] starts, ClusteringBound[] ends, long[] markedAts, int[] delTimes, long boundaryHeapSize, int size)
+    private RangeTombstoneList(ClusteringComparator comparator, ClusteringBound<?>[] starts, ClusteringBound<?>[] ends, long[] markedAts, int[] delTimes, long boundaryHeapSize, int size)
     {
         assert starts.length == ends.length && starts.length == markedAts.length && starts.length == delTimes.length;
         this.comparator = comparator;
@@ -77,7 +77,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
 
     public RangeTombstoneList(ClusteringComparator comparator, int capacity)
     {
-        this(comparator, new ClusteringBound[capacity], new ClusteringBound[capacity], new long[capacity], new int[capacity], 0, 0);
+        this(comparator, new ClusteringBound<?>[capacity], new ClusteringBound<?>[capacity], new long[capacity], new int[capacity], 0, 0);
     }
 
     public boolean isEmpty()
@@ -108,8 +108,8 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
     public RangeTombstoneList copy(AbstractAllocator allocator)
     {
         RangeTombstoneList copy =  new RangeTombstoneList(comparator,
-                                                          new ClusteringBound[size],
-                                                          new ClusteringBound[size],
+                                                          new ClusteringBound<?>[size],
+                                                          new ClusteringBound<?>[size],
                                                           Arrays.copyOf(markedAts, size),
                                                           Arrays.copyOf(delTimes, size),
                                                           boundaryHeapSize, size);
@@ -124,12 +124,12 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
         return copy;
     }
 
-    private static ClusteringBound clone(ClusteringBound bound, AbstractAllocator allocator)
+    private static <T> ClusteringBound<ByteBuffer> clone(ClusteringBound<T> bound, AbstractAllocator allocator)
     {
         ByteBuffer[] values = new ByteBuffer[bound.size()];
         for (int i = 0; i < values.length; i++)
-            values[i] = allocator.clone(bound.get(i));
-        return new ClusteringBound(bound.kind(), values);
+            values[i] = allocator.clone(bound.get(i), bound.accessor());
+        return new BufferClusteringBound(bound.kind(), values);
     }
 
     public void add(RangeTombstone tombstone)
@@ -146,7 +146,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
      * This method will be faster if the new tombstone sort after all the currently existing ones (this is a common use case),
      * but it doesn't assume it.
      */
-    public void add(ClusteringBound start, ClusteringBound end, long markedAt, int delTime)
+    public void add(ClusteringBound<?> start, ClusteringBound<?> end, long markedAt, int delTime)
     {
         if (isEmpty())
         {
@@ -233,7 +233,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
      * Returns whether the given name/timestamp pair is deleted by one of the tombstone
      * of this RangeTombstoneList.
      */
-    public boolean isDeleted(Clustering clustering, Cell cell)
+    public boolean isDeleted(Clustering<?> clustering, Cell<?> cell)
     {
         int idx = searchInternal(clustering, 0, size);
         // No matter what the counter cell's timestamp is, a tombstone always takes precedence. See CASSANDRA-7346.
@@ -244,13 +244,13 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
      * Returns the DeletionTime for the tombstone overlapping {@code name} (there can't be more than one),
      * or null if {@code name} is not covered by any tombstone.
      */
-    public DeletionTime searchDeletionTime(Clustering name)
+    public DeletionTime searchDeletionTime(Clustering<?> name)
     {
         int idx = searchInternal(name, 0, size);
         return idx < 0 ? null : new DeletionTime(markedAts[idx], delTimes[idx]);
     }
 
-    public RangeTombstone search(Clustering name)
+    public RangeTombstone search(Clustering<?> name)
     {
         int idx = searchInternal(name, 0, size);
         return idx < 0 ? null : rangeTombstone(idx);
@@ -262,7 +262,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
      *
      * Note that bounds are not in the range if they fall on its boundary.
      */
-    private int searchInternal(ClusteringPrefix name, int startIdx, int endIdx)
+    private int searchInternal(ClusteringPrefix<?> name, int startIdx, int endIdx)
     {
         if (isEmpty())
             return -1;
@@ -325,17 +325,17 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
         return new RangeTombstone(Slice.make(starts[idx], ends[idx]), new DeletionTime(markedAts[idx], delTimes[idx]));
     }
 
-    private RangeTombstone rangeTombstoneWithNewStart(int idx, ClusteringBound newStart)
+    private RangeTombstone rangeTombstoneWithNewStart(int idx, ClusteringBound<?> newStart)
     {
         return new RangeTombstone(Slice.make(newStart, ends[idx]), new DeletionTime(markedAts[idx], delTimes[idx]));
     }
 
-    private RangeTombstone rangeTombstoneWithNewEnd(int idx, ClusteringBound newEnd)
+    private RangeTombstone rangeTombstoneWithNewEnd(int idx, ClusteringBound<?> newEnd)
     {
         return new RangeTombstone(Slice.make(starts[idx], newEnd), new DeletionTime(markedAts[idx], delTimes[idx]));
     }
 
-    private RangeTombstone rangeTombstoneWithNewBounds(int idx, ClusteringBound newStart, ClusteringBound newEnd)
+    private RangeTombstone rangeTombstoneWithNewBounds(int idx, ClusteringBound<?> newStart, ClusteringBound<?> newEnd)
     {
         return new RangeTombstone(Slice.make(newStart, newEnd), new DeletionTime(markedAts[idx], delTimes[idx]));
     }
@@ -382,13 +382,13 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
 
     private Iterator<RangeTombstone> forwardIterator(final Slice slice)
     {
-        int startIdx = slice.start() == ClusteringBound.BOTTOM ? 0 : searchInternal(slice.start(), 0, size);
+        int startIdx = slice.start().isBottom() ? 0 : searchInternal(slice.start(), 0, size);
         final int start = startIdx < 0 ? -startIdx-1 : startIdx;
 
         if (start >= size)
             return Collections.emptyIterator();
 
-        int finishIdx = slice.end() == ClusteringBound.TOP ? size - 1 : searchInternal(slice.end(), start, size);
+        int finishIdx = slice.end().isTop() ? size - 1 : searchInternal(slice.end(), start, size);
         // if stopIdx is the first range after 'slice.end()' we care only until the previous range
         final int finish = finishIdx < 0 ? -finishIdx-2 : finishIdx;
 
@@ -399,8 +399,8 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
         {
             // We want to make sure the range are stricly included within the queried slice as this
             // make it easier to combine things when iterating over successive slices.
-            ClusteringBound s = comparator.compare(starts[start], slice.start()) < 0 ? slice.start() : starts[start];
-            ClusteringBound e = comparator.compare(slice.end(), ends[start]) < 0 ? slice.end() : ends[start];
+            ClusteringBound<?> s = comparator.compare(starts[start], slice.start()) < 0 ? slice.start() : starts[start];
+            ClusteringBound<?> e = comparator.compare(slice.end(), ends[start]) < 0 ? slice.end() : ends[start];
             if (Slice.isEmpty(comparator, s, e))
                 return Collections.emptyIterator();
             return Iterators.<RangeTombstone>singletonIterator(rangeTombstoneWithNewBounds(start, s, e));
@@ -429,14 +429,14 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
 
     private Iterator<RangeTombstone> reverseIterator(final Slice slice)
     {
-        int startIdx = slice.end() == ClusteringBound.TOP ? size - 1 : searchInternal(slice.end(), 0, size);
+        int startIdx = slice.end().isTop() ? size - 1 : searchInternal(slice.end(), 0, size);
         // if startIdx is the first range after 'slice.end()' we care only until the previous range
         final int start = startIdx < 0 ? -startIdx-2 : startIdx;
 
         if (start < 0)
             return Collections.emptyIterator();
 
-        int finishIdx = slice.start() == ClusteringBound.BOTTOM ? 0 : searchInternal(slice.start(), 0, start + 1);  // include same as finish
+        int finishIdx = slice.start().isBottom() ? 0 : searchInternal(slice.start(), 0, start + 1);  // include same as finish
         // if stopIdx is the first range after 'slice.end()' we care only until the previous range
         final int finish = finishIdx < 0 ? -finishIdx-1 : finishIdx;
 
@@ -447,8 +447,8 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
         {
             // We want to make sure the range are stricly included within the queried slice as this
             // make it easier to combine things when iterator over successive slices.
-            ClusteringBound s = comparator.compare(starts[start], slice.start()) < 0 ? slice.start() : starts[start];
-            ClusteringBound e = comparator.compare(slice.end(), ends[start]) < 0 ? slice.end() : ends[start];
+            ClusteringBound<?> s = comparator.compare(starts[start], slice.start()) < 0 ? slice.start() : starts[start];
+            ClusteringBound<?> e = comparator.compare(slice.end(), ends[start]) < 0 ? slice.end() : ends[start];
             if (Slice.isEmpty(comparator, s, e))
                 return Collections.emptyIterator();
             return Iterators.<RangeTombstone>singletonIterator(rangeTombstoneWithNewBounds(start, s, e));
@@ -533,7 +533,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
      *   - e_i <= s_i+1
      * Basically, range are non overlapping and in order.
      */
-    private void insertFrom(int i, ClusteringBound start, ClusteringBound end, long markedAt, int delTime)
+    private void insertFrom(int i, ClusteringBound<?> start, ClusteringBound<?> end, long markedAt, int delTime)
     {
         while (i < size)
         {
@@ -552,7 +552,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
                 // First deal with what might come before the newly added one.
                 if (comparator.compare(starts[i], start) < 0)
                 {
-                    ClusteringBound newEnd = start.invert();
+                    ClusteringBound<?> newEnd = start.invert();
                     if (!Slice.isEmpty(comparator, starts[i], newEnd))
                     {
                         addInternal(i, starts[i], newEnd, markedAts[i], delTimes[i]);
@@ -600,7 +600,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
                     // one to reflect the not overwritten parts. We're then done.
                     addInternal(i, start, end, markedAt, delTime);
                     i++;
-                    ClusteringBound newStart = end.invert();
+                    ClusteringBound<?> newStart = end.invert();
                     if (!Slice.isEmpty(comparator, newStart, ends[i]))
                     {
                         setInternal(i, newStart, ends[i], markedAts[i], delTimes[i]);
@@ -622,7 +622,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
                         addInternal(i, start, end, markedAt, delTime);
                         return;
                     }
-                    ClusteringBound newEnd = starts[i].invert();
+                    ClusteringBound<?> newEnd = starts[i].invert();
                     if (!Slice.isEmpty(comparator, start, newEnd))
                     {
                         addInternal(i, start, newEnd, markedAt, delTime);
@@ -654,7 +654,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
     /*
      * Adds the new tombstone at index i, growing and/or moving elements to make room for it.
      */
-    private void addInternal(int i, ClusteringBound start, ClusteringBound end, long markedAt, int delTime)
+    private void addInternal(int i, ClusteringBound<?> start, ClusteringBound<?> end, long markedAt, int delTime)
     {
         assert i >= 0;
 
@@ -698,12 +698,12 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
         delTimes = grow(delTimes, size, newLength, i);
     }
 
-    private static ClusteringBound[] grow(ClusteringBound[] a, int size, int newLength, int i)
+    private static ClusteringBound<?>[] grow(ClusteringBound<?>[] a, int size, int newLength, int i)
     {
         if (i < 0 || i >= size)
             return Arrays.copyOf(a, newLength);
 
-        ClusteringBound[] newA = new ClusteringBound[newLength];
+        ClusteringBound<?>[] newA = new ClusteringBound<?>[newLength];
         System.arraycopy(a, 0, newA, 0, i);
         System.arraycopy(a, i, newA, i+1, size - i);
         return newA;
@@ -748,7 +748,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
         starts[i] = null;
     }
 
-    private void setInternal(int i, ClusteringBound start, ClusteringBound end, long markedAt, int delTime)
+    private void setInternal(int i, ClusteringBound<?> start, ClusteringBound<?> end, long markedAt, int delTime)
     {
         if (starts[i] != null)
             boundaryHeapSize -= starts[i].unsharedHeapSize() + ends[i].unsharedHeapSize();
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 60ddaa9..6b19ed9 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -550,7 +550,7 @@ public abstract class ReadCommand extends AbstractReadQuery
             public Row applyToRow(Row row)
             {
                 boolean hasTombstones = false;
-                for (Cell cell : row.cells())
+                for (Cell<?> cell : row.cells())
                 {
                     if (!cell.isLive(ReadCommand.this.nowInSec()))
                     {
@@ -579,7 +579,7 @@ public abstract class ReadCommand extends AbstractReadQuery
                 return marker;
             }
 
-            private void countTombstone(ClusteringPrefix clustering)
+            private void countTombstone(ClusteringPrefix<?> clustering)
             {
                 ++tombstones;
                 if (tombstones > failureThreshold && respectTombstoneThresholds)
diff --git a/src/java/org/apache/cassandra/db/ReadQuery.java b/src/java/org/apache/cassandra/db/ReadQuery.java
index fd94aa1..bd20c26 100644
--- a/src/java/org/apache/cassandra/db/ReadQuery.java
+++ b/src/java/org/apache/cassandra/db/ReadQuery.java
@@ -81,7 +81,7 @@ public interface ReadQuery
                 return false;
             }
 
-            public boolean selectsClustering(DecoratedKey key, Clustering clustering)
+            public boolean selectsClustering(DecoratedKey key, Clustering<?> clustering)
             {
                 return false;
             }
@@ -192,7 +192,7 @@ public interface ReadQuery
      * @return true if the read query would select the given clustering, including checks against the row filter, if
      * checkRowFilter is true
      */
-    public boolean selectsClustering(DecoratedKey key, Clustering clustering);
+    public boolean selectsClustering(DecoratedKey key, Clustering<?> clustering);
 
     /**
      * The time in seconds to use as "now" for this query.
diff --git a/src/java/org/apache/cassandra/db/RowIndexEntry.java b/src/java/org/apache/cassandra/db/RowIndexEntry.java
index 546c0c8..215768b 100644
--- a/src/java/org/apache/cassandra/db/RowIndexEntry.java
+++ b/src/java/org/apache/cassandra/db/RowIndexEntry.java
@@ -629,7 +629,7 @@ public class RowIndexEntry<T> implements IMeasurableMemory
             this.idxInfoSerializer = idxInfoSerializer;
 
             this.fieldsSerializedSize = serializedSize(deletionTime, headerLength, columnIndexCount);
-            this.offsetsOffset = indexedPartSize + fieldsSerializedSize - columnsIndexCount * TypeSizes.sizeof(0);
+            this.offsetsOffset = indexedPartSize + fieldsSerializedSize - columnsIndexCount * TypeSizes.INT_SIZE;
         }
 
         /**
@@ -650,7 +650,7 @@ public class RowIndexEntry<T> implements IMeasurableMemory
             this.idxInfoSerializer = idxInfoSerializer;
 
             this.fieldsSerializedSize = serializedSize(deletionTime, headerLength, columnsIndexCount);
-            this.offsetsOffset = indexedPartSize + fieldsSerializedSize - columnsIndexCount * TypeSizes.sizeof(0);
+            this.offsetsOffset = indexedPartSize + fieldsSerializedSize - columnsIndexCount * TypeSizes.INT_SIZE;
         }
 
         @Override
@@ -739,7 +739,7 @@ public class RowIndexEntry<T> implements IMeasurableMemory
         IndexInfo fetchIndex(int index) throws IOException
         {
             // seek to position in "offsets to IndexInfo" table
-            indexReader.seek(indexInfoFilePosition + offsetsOffset + index * TypeSizes.sizeof(0));
+            indexReader.seek(indexInfoFilePosition + offsetsOffset + index * TypeSizes.INT_SIZE);
 
             // read offset of IndexInfo
             int indexInfoPos = indexReader.readInt();
diff --git a/src/java/org/apache/cassandra/db/SimpleBuilders.java b/src/java/org/apache/cassandra/db/SimpleBuilders.java
index 98572c5..f1e34f9 100644
--- a/src/java/org/apache/cassandra/db/SimpleBuilders.java
+++ b/src/java/org/apache/cassandra/db/SimpleBuilders.java
@@ -53,10 +53,10 @@ public abstract class SimpleBuilders
         return metadata.partitioner.decorateKey(key);
     }
 
-    private static Clustering makeClustering(TableMetadata metadata, Object... clusteringColumns)
+    private static Clustering<?> makeClustering(TableMetadata metadata, Object... clusteringColumns)
     {
         if (clusteringColumns.length == 1 && clusteringColumns[0] instanceof Clustering)
-            return (Clustering)clusteringColumns[0];
+            return (Clustering<?>)clusteringColumns[0];
 
         if (clusteringColumns.length == 0)
         {
@@ -157,7 +157,7 @@ public abstract class SimpleBuilders
     {
         private final TableMetadata metadata;
         private final DecoratedKey key;
-        private final Map<Clustering, RowBuilder> rowBuilders = new HashMap<>();
+        private final Map<Clustering<?>, RowBuilder> rowBuilders = new HashMap<>();
         private List<RTBuilder> rangeBuilders = null; // We use that rarely, so create lazily
         private List<RangeTombstone> rangeTombstones = null;
 
@@ -176,7 +176,7 @@ public abstract class SimpleBuilders
 
         public Row.SimpleBuilder row(Object... clusteringValues)
         {
-            Clustering clustering = makeClustering(metadata, clusteringValues);
+            Clustering<?> clustering = makeClustering(metadata, clusteringValues);
             RowBuilder builder = rowBuilders.get(clustering);
             if (builder == null)
             {
@@ -303,8 +303,8 @@ public abstract class SimpleBuilders
 
             private RangeTombstone build()
             {
-                ClusteringBound startBound = ClusteringBound.create(comparator, true, startInclusive, start);
-                ClusteringBound endBound = ClusteringBound.create(comparator, false, endInclusive, end);
+                ClusteringBound<?> startBound = ClusteringBound.create(comparator, true, startInclusive, start);
+                ClusteringBound<?> endBound = ClusteringBound.create(comparator, false, endInclusive, end);
                 return new RangeTombstone(Slice.make(startBound, endBound), deletionTime);
             }
         }
@@ -446,7 +446,7 @@ public abstract class SimpleBuilders
             return column;
         }
 
-        private Cell cell(ColumnMetadata column, ByteBuffer value, CellPath path)
+        private Cell<?> cell(ColumnMetadata column, ByteBuffer value, CellPath path)
         {
             if (value == null)
                 return BufferCell.tombstone(column, timestamp, nowInSec, path);
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index a4b028b..62d67f1 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -255,7 +255,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
      * @return a newly created read command that queries the {@code names} in {@code key}. The returned query will
      * query every columns (without limit or row filtering) and be in forward order.
      */
-    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
+    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering<?>> names)
     {
         ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
         return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
@@ -272,7 +272,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
      * @return a newly created read command that queries {@code name} in {@code key}. The returned query will
      * query every columns (without limit or row filtering).
      */
-    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, Clustering name)
+    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, Clustering<?> name)
     {
         return create(metadata, nowInSec, key, FBUtilities.singleton(name, metadata.comparator));
     }
@@ -368,7 +368,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
     }
 
     @Override
-    public SinglePartitionReadCommand forPaging(Clustering lastReturned, DataLimits limits)
+    public SinglePartitionReadCommand forPaging(Clustering<?> lastReturned, DataLimits limits)
     {
         // We shouldn't have set digest yet when reaching that point
         assert !isDigestQuery();
@@ -928,10 +928,10 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
         if (result == null)
             return filter;
 
-        SearchIterator<Clustering, Row> searchIter = result.searchIterator(columnFilter(), false);
+        SearchIterator<Clustering<?>, Row> searchIter = result.searchIterator(columnFilter(), false);
 
         RegularAndStaticColumns columns = columnFilter().fetchedColumns();
-        NavigableSet<Clustering> clusterings = filter.requestedRows();
+        NavigableSet<Clustering<?>> clusterings = filter.requestedRows();
 
         // We want to remove rows for which we have values for all requested columns. We have to deal with both static and regular rows.
         // TODO: we could also remove a selected column if we've found values for every requested row but we'll leave
@@ -944,8 +944,8 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
             removeStatic = staticRow != null && canRemoveRow(staticRow, columns.statics, sstableTimestamp);
         }
 
-        NavigableSet<Clustering> toRemove = null;
-        for (Clustering clustering : clusterings)
+        NavigableSet<Clustering<?>> toRemove = null;
+        for (Clustering<?> clustering : clusterings)
         {
             Row row = searchIter.next(clustering);
             if (row == null || !canRemoveRow(row, columns.regulars, sstableTimestamp))
@@ -967,7 +967,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
 
         if (toRemove != null)
         {
-            BTreeSet.Builder<Clustering> newClusterings = BTreeSet.builder(result.metadata().comparator);
+            BTreeSet.Builder<Clustering<?>> newClusterings = BTreeSet.builder(result.metadata().comparator);
             newClusterings.addAll(Sets.difference(clusterings, toRemove));
             clusterings = newClusterings.build();
         }
@@ -984,7 +984,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
 
         for (ColumnMetadata column : requestedColumns)
         {
-            Cell cell = row.getCell(column);
+            Cell<?> cell = row.getCell(column);
             if (cell == null || cell.timestamp() <= sstableTimestamp)
                 return false;
         }
@@ -1113,7 +1113,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
                                        IndexMetadata index)
         throws IOException
         {
-            DecoratedKey key = metadata.partitioner.decorateKey(metadata.partitionKeyType.readValue(in, DatabaseDescriptor.getMaxValueSize()));
+            DecoratedKey key = metadata.partitioner.decorateKey(metadata.partitionKeyType.readBuffer(in, DatabaseDescriptor.getMaxValueSize()));
             ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata);
             return new SinglePartitionReadCommand(isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, index);
         }
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadQuery.java b/src/java/org/apache/cassandra/db/SinglePartitionReadQuery.java
index f9f0014..755d552 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadQuery.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadQuery.java
@@ -129,7 +129,7 @@ public interface SinglePartitionReadQuery extends ReadQuery
      *
      * @return the newly create query.
      */
-    SinglePartitionReadQuery forPaging(Clustering lastReturned, DataLimits limits);
+    SinglePartitionReadQuery forPaging(Clustering<?> lastReturned, DataLimits limits);
 
     @Override
     default SinglePartitionPager getPager(PagingState pagingState, ProtocolVersion protocolVersion)
@@ -147,7 +147,7 @@ public interface SinglePartitionReadQuery extends ReadQuery
         return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().partitionKeyType);
     }
 
-    default boolean selectsClustering(DecoratedKey key, Clustering clustering)
+    default boolean selectsClustering(DecoratedKey key, Clustering<?> clustering)
     {
         if (clustering == Clustering.STATIC_CLUSTERING)
             return !columnFilter().fetchedColumns().statics.isEmpty();
@@ -260,7 +260,7 @@ public interface SinglePartitionReadQuery extends ReadQuery
             return Iterables.any(queries, c -> c.selectsKey(key));
         }
 
-        public boolean selectsClustering(DecoratedKey key, Clustering clustering)
+        public boolean selectsClustering(DecoratedKey key, Clustering<?> clustering)
         {
             return Iterables.any(queries, c -> c.selectsClustering(key, clustering));
         }
diff --git a/src/java/org/apache/cassandra/db/Slice.java b/src/java/org/apache/cassandra/db/Slice.java
index 384158f..c6f558f 100644
--- a/src/java/org/apache/cassandra/db/Slice.java
+++ b/src/java/org/apache/cassandra/db/Slice.java
@@ -22,9 +22,9 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * A slice represents the selection of a range of rows.
@@ -38,10 +38,10 @@ public class Slice
     public static final Serializer serializer = new Serializer();
 
     /** The slice selecting all rows (of a given partition) */
-    public static final Slice ALL = new Slice(ClusteringBound.BOTTOM, ClusteringBound.TOP)
+    public static final Slice ALL = new Slice(BufferClusteringBound.BOTTOM, BufferClusteringBound.TOP)
     {
         @Override
-        public boolean includes(ClusteringComparator comparator, ClusteringPrefix clustering)
+        public boolean includes(ClusteringComparator comparator, ClusteringPrefix<?> clustering)
         {
             return true;
         }
@@ -59,19 +59,19 @@ public class Slice
         }
     };
 
-    private final ClusteringBound start;
-    private final ClusteringBound end;
+    private final ClusteringBound<?> start;
+    private final ClusteringBound<?> end;
 
-    private Slice(ClusteringBound start, ClusteringBound end)
+    private Slice(ClusteringBound<?> start, ClusteringBound<?> end)
     {
         assert start.isStart() && end.isEnd();
         this.start = start;
         this.end = end;
     }
 
-    public static Slice make(ClusteringBound start, ClusteringBound end)
+    public static Slice make(ClusteringBound<?> start, ClusteringBound<?> end)
     {
-        if (start == ClusteringBound.BOTTOM && end == ClusteringBound.TOP)
+        if (start.isBottom() && end.isTop())
             return ALL;
 
         return new Slice(start, end);
@@ -90,49 +90,37 @@ public class Slice
         return new Slice(builder.buildBound(true, true), builder.buildBound(false, true));
     }
 
-    public static Slice make(Clustering clustering)
+    public static Slice make(Clustering<?> clustering)
     {
         // This doesn't give us what we want with the clustering prefix
         assert clustering != Clustering.STATIC_CLUSTERING;
-        ByteBuffer[] values = extractValues(clustering);
-        return new Slice(ClusteringBound.inclusiveStartOf(values), ClusteringBound.inclusiveEndOf(values));
+        return new Slice(ClusteringBound.inclusiveStartOf(clustering), ClusteringBound.inclusiveEndOf(clustering));
     }
 
-    public static Slice make(Clustering start, Clustering end)
+    public static Slice make(Clustering<?> start, Clustering<?> end)
     {
         // This doesn't give us what we want with the clustering prefix
         assert start != Clustering.STATIC_CLUSTERING && end != Clustering.STATIC_CLUSTERING;
 
-        ByteBuffer[] startValues = extractValues(start);
-        ByteBuffer[] endValues = extractValues(end);
-
-        return new Slice(ClusteringBound.inclusiveStartOf(startValues), ClusteringBound.inclusiveEndOf(endValues));
-    }
-
-    private static ByteBuffer[] extractValues(ClusteringPrefix clustering)
-    {
-        ByteBuffer[] values = new ByteBuffer[clustering.size()];
-        for (int i = 0; i < clustering.size(); i++)
-            values[i] = clustering.get(i);
-        return values;
+        return new Slice(ClusteringBound.inclusiveStartOf(start), ClusteringBound.inclusiveEndOf(end));
     }
 
-    public ClusteringBound start()
+    public ClusteringBound<?> start()
     {
         return start;
     }
 
-    public ClusteringBound end()
+    public ClusteringBound<?> end()
     {
         return end;
     }
 
-    public ClusteringBound open(boolean reversed)
+    public ClusteringBound<?> open(boolean reversed)
     {
         return reversed ? end : start;
     }
 
-    public ClusteringBound close(boolean reversed)
+    public ClusteringBound<?> close(boolean reversed)
     {
         return reversed ? start : end;
     }
@@ -157,7 +145,7 @@ public class Slice
      * @return whether the slice formed by {@code start} and {@code end} is
      * empty or not.
      */
-    public static boolean isEmpty(ClusteringComparator comparator, ClusteringBound start, ClusteringBound end)
+    public static boolean isEmpty(ClusteringComparator comparator, ClusteringBound<?> start, ClusteringBound<?> end)
     {
         assert start.isStart() && end.isEnd();
 
@@ -179,7 +167,7 @@ public class Slice
      *
      * @return whether {@code bound} is within the bounds of this slice.
      */
-    public boolean includes(ClusteringComparator comparator, ClusteringPrefix bound)
+    public boolean includes(ClusteringComparator comparator, ClusteringPrefix<?> bound)
     {
         return comparator.compare(start, bound) <= 0 && comparator.compare(bound, end) <= 0;
     }
@@ -197,7 +185,7 @@ public class Slice
      * @return a new slice that selects results coming after {@code lastReturned}, or {@code null} if paging
      * the resulting slice selects nothing (i.e. if it originally selects nothing coming after {@code lastReturned}).
      */
-    public Slice forPaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive, boolean reversed)
+    public Slice forPaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive, boolean reversed)
     {
         if (lastReturned == null)
             return this;
@@ -212,8 +200,7 @@ public class Slice
             if (cmp < 0 || (inclusive && cmp == 0))
                 return this;
 
-            ByteBuffer[] values = extractValues(lastReturned);
-            return new Slice(start, inclusive ? ClusteringBound.inclusiveEndOf(values) : ClusteringBound.exclusiveEndOf(values));
+            return new Slice(start, inclusive ? ClusteringBound.inclusiveEndOf(lastReturned) : ClusteringBound.exclusiveEndOf(lastReturned));
         }
         else
         {
@@ -225,8 +212,7 @@ public class Slice
             if (cmp < 0 || (inclusive && cmp == 0))
                 return this;
 
-            ByteBuffer[] values = extractValues(lastReturned);
-            return new Slice(inclusive ? ClusteringBound.inclusiveStartOf(values) : ClusteringBound.exclusiveStartOf(values), end);
+            return new Slice(inclusive ? ClusteringBound.inclusiveStartOf(lastReturned) : ClusteringBound.exclusiveStartOf(lastReturned), end);
         }
     }
 
@@ -255,14 +241,14 @@ public class Slice
         {
             if (i > 0)
                 sb.append(':');
-            sb.append(comparator.subtype(i).getString(start.get(i)));
+            sb.append(start.stringAt(i, comparator));
         }
         sb.append(", ");
         for (int i = 0; i < end.size(); i++)
         {
             if (i > 0)
                 sb.append(':');
-            sb.append(comparator.subtype(i).getString(end.get(i)));
+            sb.append(end.stringAt(i, comparator));
         }
         sb.append(end.isInclusive() ? "]" : ")");
         return sb.toString();
@@ -301,8 +287,8 @@ public class Slice
 
         public Slice deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
         {
-            ClusteringBound start = (ClusteringBound) ClusteringBound.serializer.deserialize(in, version, types);
-            ClusteringBound end = (ClusteringBound) ClusteringBound.serializer.deserialize(in, version, types);
+            ClusteringBound<byte[]> start = (ClusteringBound<byte[]>) ClusteringBound.serializer.deserialize(in, version, types);
+            ClusteringBound<byte[]> end = (ClusteringBound<byte[]>) ClusteringBound.serializer.deserialize(in, version, types);
             return new Slice(start, end);
         }
     }
diff --git a/src/java/org/apache/cassandra/db/Slices.java b/src/java/org/apache/cassandra/db/Slices.java
index 3d19fe9..441a5d3 100644
--- a/src/java/org/apache/cassandra/db/Slices.java
+++ b/src/java/org/apache/cassandra/db/Slices.java
@@ -59,7 +59,7 @@ public abstract class Slices implements Iterable<Slice>
      */
     public static Slices with(ClusteringComparator comparator, Slice slice)
     {
-        if (slice.start() == ClusteringBound.BOTTOM && slice.end() == ClusteringBound.TOP)
+        if (slice.start().isBottom() && slice.end().isTop())
             return Slices.ALL;
 
         Preconditions.checkArgument(!slice.isEmpty(comparator));
@@ -106,7 +106,7 @@ public abstract class Slices implements Iterable<Slice>
      *
      * @return new slices that select results coming after {@code lastReturned}.
      */
-    public abstract Slices forPaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive, boolean reversed);
+    public abstract Slices forPaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive, boolean reversed);
 
     /**
      * An object that allows to test whether rows are selected by this {@code Slices} objects assuming those rows
@@ -126,7 +126,7 @@ public abstract class Slices implements Iterable<Slice>
      *
      * @return whether a given clustering (row) is selected by this {@code Slices} object.
      */
-    public abstract boolean selects(Clustering clustering);
+    public abstract boolean selects(Clustering<?> clustering);
 
 
     /**
@@ -159,7 +159,7 @@ public abstract class Slices implements Iterable<Slice>
      */
     public interface InOrderTester
     {
-        public boolean includes(Clustering value);
+        public boolean includes(Clustering<?> value);
         public boolean isDone();
     }
 
@@ -186,7 +186,7 @@ public abstract class Slices implements Iterable<Slice>
             this.slices = new ArrayList<>(initialSize);
         }
 
-        public Builder add(ClusteringBound start, ClusteringBound end)
+        public Builder add(ClusteringBound<?> start, ClusteringBound<?> end)
         {
             return add(Slice.make(start, end));
         }
@@ -335,7 +335,7 @@ public abstract class Slices implements Iterable<Slice>
             for (int i = 0; i < size; i++)
                 slices[i] = Slice.serializer.deserialize(in, version, metadata.comparator.subtypes());
 
-            if (size == 1 && slices[0].start() == ClusteringBound.BOTTOM && slices[0].end() == ClusteringBound.TOP)
+            if (size == 1 && slices[0].start().isBottom() && slices[0].end().isTop())
                 return ALL;
 
             return new ArrayBackedSlices(metadata.comparator, slices);
@@ -377,7 +377,7 @@ public abstract class Slices implements Iterable<Slice>
             return slices[i];
         }
 
-        public boolean selects(Clustering clustering)
+        public boolean selects(Clustering<?> clustering)
         {
             for (int i = 0; i < slices.length; i++)
             {
@@ -396,12 +396,12 @@ public abstract class Slices implements Iterable<Slice>
             return reversed ? new InReverseOrderTester() : new InForwardOrderTester();
         }
 
-        public Slices forPaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive, boolean reversed)
+        public Slices forPaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive, boolean reversed)
         {
             return reversed ? forReversePaging(comparator, lastReturned, inclusive) : forForwardPaging(comparator, lastReturned, inclusive);
         }
 
-        private Slices forForwardPaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive)
+        private Slices forForwardPaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive)
         {
             for (int i = 0; i < slices.length; i++)
             {
@@ -420,7 +420,7 @@ public abstract class Slices implements Iterable<Slice>
             return Slices.NONE;
         }
 
-        private Slices forReversePaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive)
+        private Slices forReversePaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive)
         {
             for (int i = slices.length - 1; i >= 0; i--)
             {
@@ -459,7 +459,7 @@ public abstract class Slices implements Iterable<Slice>
             private int idx;
             private boolean inSlice;
 
-            public boolean includes(Clustering value)
+            public boolean includes(Clustering<?> value)
             {
                 while (idx < slices.length)
                 {
@@ -502,7 +502,7 @@ public abstract class Slices implements Iterable<Slice>
                 this.idx = slices.length - 1;
             }
 
-            public boolean includes(Clustering value)
+            public boolean includes(Clustering<?> value)
             {
                 while (idx >= 0)
                 {
@@ -665,8 +665,8 @@ public abstract class Slices implements Iterable<Slice>
 
             public static ComponentOfSlice fromSlice(int component, Slice slice)
             {
-                ClusteringBound start = slice.start();
-                ClusteringBound end = slice.end();
+                ClusteringBound<?> start = slice.start();
+                ClusteringBound<?> end = slice.end();
 
                 if (component >= start.size() && component >= end.size())
                     return null;
@@ -676,12 +676,12 @@ public abstract class Slices implements Iterable<Slice>
                 if (component < start.size())
                 {
                     startInclusive = start.isInclusive();
-                    startValue = start.get(component);
+                    startValue = start.bufferAt(component);
                 }
                 if (component < end.size())
                 {
                     endInclusive = end.isInclusive();
-                    endValue = end.get(component);
+                    endValue = end.bufferAt(component);
                 }
                 return new ComponentOfSlice(startInclusive, startValue, endInclusive, endValue);
             }
@@ -702,7 +702,7 @@ public abstract class Slices implements Iterable<Slice>
     {
         private static final InOrderTester trivialTester = new InOrderTester()
         {
-            public boolean includes(Clustering value)
+            public boolean includes(Clustering<?> value)
             {
                 return true;
             }
@@ -733,12 +733,12 @@ public abstract class Slices implements Iterable<Slice>
             return false;
         }
 
-        public boolean selects(Clustering clustering)
+        public boolean selects(Clustering<?> clustering)
         {
             return true;
         }
 
-        public Slices forPaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive, boolean reversed)
+        public Slices forPaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive, boolean reversed)
         {
             return new ArrayBackedSlices(comparator, new Slice[]{ Slice.ALL.forPaging(comparator, lastReturned, inclusive, reversed) });
         }
@@ -777,7 +777,7 @@ public abstract class Slices implements Iterable<Slice>
     {
         private static final InOrderTester trivialTester = new InOrderTester()
         {
-            public boolean includes(Clustering value)
+            public boolean includes(Clustering<?> value)
             {
                 return false;
             }
@@ -808,12 +808,12 @@ public abstract class Slices implements Iterable<Slice>
             return false;
         }
 
-        public Slices forPaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive, boolean reversed)
+        public Slices forPaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive, boolean reversed)
         {
             return this;
         }
 
-        public boolean selects(Clustering clustering)
+        public boolean selects(Clustering<?> clustering)
         {
             return false;
         }
diff --git a/src/java/org/apache/cassandra/db/TypeSizes.java b/src/java/org/apache/cassandra/db/TypeSizes.java
index b47e300..aed18a6 100644
--- a/src/java/org/apache/cassandra/db/TypeSizes.java
+++ b/src/java/org/apache/cassandra/db/TypeSizes.java
@@ -27,12 +27,14 @@ public final class TypeSizes
 
     private TypeSizes(){}
 
-    private static final int BOOL_SIZE = 1;
-    private static final int BYTE_SIZE = 1;
-    private static final int SHORT_SIZE = 2;
-    private static final int INT_SIZE = 4;
-    private static final int LONG_SIZE = 8;
-    private static final int UUID_SIZE = 16;
+    public static final int BOOL_SIZE = 1;
+    public static final int BYTE_SIZE = 1;
+    public static final int SHORT_SIZE = 2;
+    public static final int INT_SIZE = 4;
+    public static final int LONG_SIZE = 8;
+    public static final int FLOAT_SIZE = 4;
+    public static final int DOUBLE_SIZE = 8;
+    public static final int UUID_SIZE = 16;
 
     /** assumes UTF8 */
     public static int sizeof(String value)
@@ -74,31 +76,49 @@ public final class TypeSizes
         return sizeofUnsignedVInt(value.remaining()) + value.remaining();
     }
 
+    @SuppressWarnings("unused")
     public static int sizeof(boolean value)
     {
         return BOOL_SIZE;
     }
 
+    @SuppressWarnings("unused")
     public static int sizeof(byte value)
     {
         return BYTE_SIZE;
     }
 
+    @SuppressWarnings("unused")
     public static int sizeof(short value)
     {
         return SHORT_SIZE;
     }
 
+    @SuppressWarnings("unused")
     public static int sizeof(int value)
     {
         return INT_SIZE;
     }
 
+    @SuppressWarnings("unused")
     public static int sizeof(long value)
     {
         return LONG_SIZE;
     }
 
+    @SuppressWarnings("unused")
+    public static int sizeof(float value)
+    {
+        return FLOAT_SIZE;
+    }
+
+    @SuppressWarnings("unused")
+    public static int sizeof(double value)
+    {
+        return DOUBLE_SIZE;
+    }
+
+    @SuppressWarnings("unused")
     public static int sizeof(UUID value)
     {
         return UUID_SIZE;
diff --git a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
index f9ff1d7..8430541 100644
--- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
+++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.db;
 
 import java.io.IOException;
 
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -105,7 +106,7 @@ public class UnfilteredDeserializer
      * comparison. Whenever we know what to do with this atom (read it or skip it),
      * readNext or skipNext should be called.
      */
-    public int compareNextTo(ClusteringBound bound) throws IOException
+    public int compareNextTo(ClusteringBound<?> bound) throws IOException
     {
         if (!isReady)
             prepareNext();
@@ -134,7 +135,7 @@ public class UnfilteredDeserializer
         isReady = false;
         if (UnfilteredSerializer.kind(nextFlags) == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER)
         {
-            ClusteringBoundOrBoundary bound = clusteringDeserializer.deserializeNextBound();
+            ClusteringBoundOrBoundary<byte[]> bound = clusteringDeserializer.deserializeNextBound();
             return UnfilteredSerializer.serializer.deserializeMarkerBody(in, header, bound);
         }
         else
diff --git a/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java b/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java
index 11f1f77..ba9441a 100644
--- a/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java
+++ b/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java
@@ -122,7 +122,7 @@ public class VirtualTableSinglePartitionReadQuery extends VirtualTableReadQuery
     }
 
     @Override
-    public SinglePartitionReadQuery forPaging(Clustering lastReturned, DataLimits limits)
+    public SinglePartitionReadQuery forPaging(Clustering<?> lastReturned, DataLimits limits)
     {
         return new VirtualTableSinglePartitionReadQuery(metadata(),
                                                         nowInSec(),
diff --git a/src/java/org/apache/cassandra/db/aggregation/GroupMaker.java b/src/java/org/apache/cassandra/db/aggregation/GroupMaker.java
index ff0ecaf..90cdab2 100644
--- a/src/java/org/apache/cassandra/db/aggregation/GroupMaker.java
+++ b/src/java/org/apache/cassandra/db/aggregation/GroupMaker.java
@@ -33,7 +33,7 @@ public abstract class GroupMaker
      */
     public static final GroupMaker GROUP_EVERYTHING = new GroupMaker()
     {
-        public boolean isNewGroup(DecoratedKey partitionKey, Clustering clustering)
+        public boolean isNewGroup(DecoratedKey partitionKey, Clustering<?> clustering)
         {
             return false;
         }
@@ -62,7 +62,7 @@ public abstract class GroupMaker
      * @return <code>true</code> if the row belongs to the same group that the previous one, <code>false</code>
      * otherwise.
      */
-    public abstract boolean isNewGroup(DecoratedKey partitionKey, Clustering clustering);
+    public abstract boolean isNewGroup(DecoratedKey partitionKey, Clustering<?> clustering);
 
     /**
      * Specify if at least one row must be returned. If the selection is performing some aggregations on all the rows,
@@ -95,7 +95,7 @@ public abstract class GroupMaker
         /**
          * The last clustering seen
          */
-        private Clustering lastClustering;
+        private Clustering<?> lastClustering;
 
         public PkPrefixGroupMaker(ClusteringComparator comparator, int clusteringPrefixSize, GroupingState state)
         {
@@ -111,7 +111,7 @@ public abstract class GroupMaker
         }
 
         @Override
-        public boolean isNewGroup(DecoratedKey partitionKey, Clustering clustering)
+        public boolean isNewGroup(DecoratedKey partitionKey, Clustering<?> clustering)
         {
             boolean isNew = false;
 
diff --git a/src/java/org/apache/cassandra/db/aggregation/GroupingState.java b/src/java/org/apache/cassandra/db/aggregation/GroupingState.java
index ba5ae28..2e522c4 100644
--- a/src/java/org/apache/cassandra/db/aggregation/GroupingState.java
+++ b/src/java/org/apache/cassandra/db/aggregation/GroupingState.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.ClusteringComparator;
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -55,9 +56,9 @@ public final class GroupingState
     /**
      * The last row clustering
      */
-    final Clustering clustering;
+    final Clustering<?> clustering;
 
-    public GroupingState(ByteBuffer partitionKey, Clustering clustering)
+    public GroupingState(ByteBuffer partitionKey, Clustering<?> clustering)
     {
         this.partitionKey = partitionKey;
         this.clustering = clustering;
@@ -78,7 +79,7 @@ public final class GroupingState
      * @return he last row clustering or <code>null</code> if either no rows has been processed yet or the last
      * row was a static row
      */
-    public Clustering clustering()
+    public Clustering<?> clustering()
     {
         return clustering;
     }
@@ -115,7 +116,7 @@ public final class GroupingState
                 return GroupingState.EMPTY_STATE;
 
             ByteBuffer partitionKey = ByteBufferUtil.readWithVIntLength(in);
-            Clustering clustering = null;
+            Clustering<byte[]> clustering = null;
             if (in.readBoolean())
                 clustering = Clustering.serializer.deserialize(in, version, comparator.subtypes());
 
diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index c631f1c..fee45c2 100644
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@ -530,17 +530,17 @@ public abstract class AbstractSSTableIterator implements UnfilteredRowIterator
 
         // Finds the index of the first block containing the provided bound, starting at the provided index.
         // Will be -1 if the bound is before any block, and blocksCount() if it is after every block.
-        public int findBlockIndex(ClusteringBound bound, int fromIdx) throws IOException
+        public int findBlockIndex(ClusteringBound<?> bound, int fromIdx) throws IOException
         {
-            if (bound == ClusteringBound.BOTTOM)
+            if (bound.isBottom())
                 return -1;
-            if (bound == ClusteringBound.TOP)
+            if (bound.isTop())
                 return blocksCount();
 
             return indexFor(bound, fromIdx);
         }
 
-        public int indexFor(ClusteringPrefix name, int lastIndex) throws IOException
+        public int indexFor(ClusteringPrefix<?> name, int lastIndex) throws IOException
         {
             IndexInfo target = new IndexInfo(name, name, 0, 0, null);
             /*
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
index 9346345..d4362f7 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
@@ -75,9 +75,9 @@ public class SSTableIterator extends AbstractSSTableIterator
     private class ForwardReader extends Reader
     {
         // The start of the current slice. This will be null as soon as we know we've passed that bound.
-        protected ClusteringBound start;
+        protected ClusteringBound<?> start;
         // The end of the current slice. Will never be null.
-        protected ClusteringBound end = ClusteringBound.TOP;
+        protected ClusteringBound<?> end = BufferClusteringBound.TOP;
 
         protected Unfiltered next; // the next element to return: this is computed by hasNextInternal().
 
@@ -91,7 +91,7 @@ public class SSTableIterator extends AbstractSSTableIterator
 
         public void setForSlice(Slice slice) throws IOException
         {
-            start = slice.start() == ClusteringBound.BOTTOM ? null : slice.start();
+            start = slice.start().isBottom() ? null : slice.start();
             end = slice.end();
 
             sliceDone = false;
@@ -119,7 +119,7 @@ public class SSTableIterator extends AbstractSSTableIterator
                     updateOpenMarker((RangeTombstoneMarker)deserializer.readNext());
             }
 
-            ClusteringBound sliceStart = start;
+            ClusteringBound<?> sliceStart = start;
             start = null;
 
             // We've reached the beginning of our queried slice. If we have an open marker
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 1e1030c..a60aafa 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -172,8 +172,8 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
 
         // Reads the unfiltered from disk and load them into the reader buffer. It stops reading when either the partition
         // is fully read, or when stopReadingDisk() returns true.
-        protected void loadFromDisk(ClusteringBound start,
-                                    ClusteringBound end,
+        protected void loadFromDisk(ClusteringBound<?> start,
+                                    ClusteringBound<?> end,
                                     boolean hasPreviousBlock,
                                     boolean hasNextBlock) throws IOException
         {
@@ -209,7 +209,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
                 // want to "return" it just yet, we'll wait until we reach it in the next blocks. That's why we trigger
                 // skipLastIteratedItem in that case (this is first item of the block, but we're iterating in reverse order
                 // so it will be last returned by the iterator).
-                ClusteringBound markerStart = start == null ? ClusteringBound.BOTTOM : start;
+                ClusteringBound<?> markerStart = start == null ? BufferClusteringBound.BOTTOM : start;
                 buffer.add(new RangeTombstoneBoundMarker(markerStart, openMarker));
                 if (hasNextBlock)
                     skipLastIteratedItem = true;
@@ -243,7 +243,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
                 // not breaking ImmutableBTreePartition, we should skip it when returning from the iterator, hence the
                 // skipFirstIteratedItem (this is the last item of the block, but we're iterating in reverse order so it will
                 // be the first returned by the iterator).
-                ClusteringBound markerEnd = end == null ? ClusteringBound.TOP : end;
+                ClusteringBound<?> markerEnd = end == null ? BufferClusteringBound.TOP : end;
                 buffer.add(new RangeTombstoneBoundMarker(markerEnd, openMarker));
                 if (hasPreviousBlock)
                     skipFirstIteratedItem = true;
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 92cff54..681a833 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -765,14 +765,14 @@ public class Scrubber implements Closeable
             {
                 if (cd.column().isSimple())
                 {
-                    Cell cell = (Cell)cd;
+                    Cell<?> cell = (Cell<?>)cd;
                     if (cell.isExpiring() && cell.localDeletionTime() < 0)
                         return true;
                 }
                 else
                 {
                     ComplexColumnData complexData = (ComplexColumnData)cd;
-                    for (Cell cell : complexData)
+                    for (Cell<?> cell : complexData)
                     {
                         if (cell.isExpiring() && cell.localDeletionTime() < 0)
                             return true;
@@ -795,14 +795,14 @@ public class Scrubber implements Closeable
             {
                 if (cd.column().isSimple())
                 {
-                    Cell cell = (Cell)cd;
+                    Cell<?> cell = (Cell<?>)cd;
                     builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
                 }
                 else
                 {
                     ComplexColumnData complexData = (ComplexColumnData)cd;
                     builder.addComplexDeletion(complexData.column(), complexData.complexDeletion());
-                    for (Cell cell : complexData)
+                    for (Cell<?> cell : complexData)
                     {
                         builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
                     }
diff --git a/src/java/org/apache/cassandra/db/context/CounterContext.java b/src/java/org/apache/cassandra/db/context/CounterContext.java
index 6a618ca..7db8192 100644
--- a/src/java/org/apache/cassandra/db/context/CounterContext.java
+++ b/src/java/org/apache/cassandra/db/context/CounterContext.java
@@ -28,6 +28,9 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.ClockAndCount;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.db.marshal.ValueAccessor;
+import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.*;
 
@@ -90,7 +93,7 @@ public class CounterContext
 
     private static final Logger logger = LoggerFactory.getLogger(CounterContext.class);
 
-    public static enum Relationship
+    public enum Relationship
     {
         EQUAL, GREATER_THAN, LESS_THAN, DISJOINT
     }
@@ -167,9 +170,9 @@ public class CounterContext
         return state.context;
     }
 
-    public static int headerLength(ByteBuffer context)
+    public static <V> int headerLength(V context, ValueAccessor<V> accessor)
     {
-        return HEADER_SIZE_LENGTH + Math.abs(context.getShort(context.position())) * HEADER_ELT_LENGTH;
+        return HEADER_SIZE_LENGTH + Math.abs(accessor.getShort(context, 0)) * HEADER_ELT_LENGTH;
     }
 
     private static int compareId(ByteBuffer bb1, int pos1, ByteBuffer bb2, int pos2)
@@ -566,34 +569,39 @@ public class CounterContext
      * @param context a counter context
      * @return the aggregated count represented by {@code context}
      */
-    public long total(ByteBuffer context)
+    public <V> long total(V context, ValueAccessor<V> accessor)
     {
         long total = 0L;
         // we could use a ContextState but it is easy enough that we avoid the object creation
-        for (int offset = context.position() + headerLength(context); offset < context.limit(); offset += STEP_LENGTH)
-            total += context.getLong(offset + CounterId.LENGTH + CLOCK_LENGTH);
+        for (int offset = headerLength(context, accessor), size=accessor.size(context); offset < size; offset += STEP_LENGTH)
+            total += accessor.getLong(context, offset + CounterId.LENGTH + CLOCK_LENGTH);
         return total;
     }
 
-    public boolean shouldClearLocal(ByteBuffer context)
+    public <V> long total(Cell<V> cell)
+    {
+        return total(cell.value(), cell.accessor());
+    }
+
+    public <V> boolean shouldClearLocal(V context, ValueAccessor<V> accessor)
     {
         // #elt being negative means we have to clean local shards.
-        return context.getShort(context.position()) < 0;
+        return accessor.getShort(context, 0) < 0;
     }
 
     /**
      * Detects whether or not the context has any legacy (local or remote) shards in it.
      */
-    public boolean hasLegacyShards(ByteBuffer context)
+    public <V>  boolean hasLegacyShards(V context, ValueAccessor<V> accessor)
     {
-        int totalCount = (context.remaining() - headerLength(context)) / STEP_LENGTH;
-        int localAndGlobalCount = Math.abs(context.getShort(context.position()));
+        int totalCount = (accessor.size(context) - headerLength(context, accessor)) / STEP_LENGTH;
+        int localAndGlobalCount = Math.abs(accessor.getShort(context, 0));
 
         if (localAndGlobalCount < totalCount)
             return true; // remote shard(s) present
 
         for (int i = 0; i < localAndGlobalCount; i++)
-            if (context.getShort(context.position() + HEADER_SIZE_LENGTH + i * HEADER_ELT_LENGTH) >= 0)
+            if (accessor.getShort(context, HEADER_SIZE_LENGTH + i * HEADER_ELT_LENGTH) >= 0)
                 return true; // found a local shard
 
         return false;
@@ -636,22 +644,16 @@ public class CounterContext
         return marked;
     }
 
-    /**
-     * Remove all the local of a context (but keep global).
-     *
-     * @param context a counter context
-     * @return a version of {@code context} where no shards are local.
-     */
-    public ByteBuffer clearAllLocal(ByteBuffer context)
+    public <V> V clearAllLocal(V context, ValueAccessor<V> accessor)
     {
-        int count = Math.abs(context.getShort(context.position()));
+        int count = Math.abs(accessor.getShort(context, 0));
         if (count == 0)
             return context; // no local or global shards present.
 
         List<Short> globalShardIndexes = new ArrayList<>(count);
         for (int i = 0; i < count; i++)
         {
-            short elt = context.getShort(context.position() + HEADER_SIZE_LENGTH + i * HEADER_ELT_LENGTH);
+            short elt = accessor.getShort(context, HEADER_SIZE_LENGTH + i * HEADER_ELT_LENGTH);
             if (elt < 0)
                 globalShardIndexes.add(elt);
         }
@@ -660,25 +662,26 @@ public class CounterContext
             return context; // no local shards detected.
 
         // allocate a smaller BB for the cleared context - with no local header elts.
-        ByteBuffer cleared = ByteBuffer.allocate(context.remaining() - (count - globalShardIndexes.size()) * HEADER_ELT_LENGTH);
+        V cleared = accessor.allocate(accessor.size(context) - (count - globalShardIndexes.size()) * HEADER_ELT_LENGTH);
 
-        cleared.putShort(cleared.position(), (short) globalShardIndexes.size());
+        accessor.putShort(cleared, 0, (short) globalShardIndexes.size());
         for (int i = 0; i < globalShardIndexes.size(); i++)
-            cleared.putShort(cleared.position() + HEADER_SIZE_LENGTH + i * HEADER_ELT_LENGTH, globalShardIndexes.get(i));
+            accessor.putShort(cleared, HEADER_SIZE_LENGTH + i * HEADER_ELT_LENGTH, globalShardIndexes.get(i));
 
-        int origHeaderLength = headerLength(context);
-        ByteBufferUtil.copyBytes(context,
-                                 context.position() + origHeaderLength,
-                                 cleared,
-                                 cleared.position() + headerLength(cleared),
-                                 context.remaining() - origHeaderLength);
+        int origHeaderLength = headerLength(context, accessor);
+        accessor.copyTo(context,
+                        origHeaderLength,
+                        cleared,
+                        accessor,
+                        headerLength(cleared, accessor),
+                        accessor.size(context) - origHeaderLength);
 
         return cleared;
     }
 
-    public void validateContext(ByteBuffer context) throws MarshalException
+    public <V> void validateContext(V context, ValueAccessor<V> accessor) throws MarshalException
     {
-        if ((context.remaining() - headerLength(context)) % STEP_LENGTH != 0)
+        if ((accessor.size(context) - headerLength(context, accessor)) % STEP_LENGTH != 0)
             throw new MarshalException("Invalid size for a counter context");
     }
 
@@ -719,7 +722,7 @@ public class CounterContext
     @VisibleForTesting
     public int findPositionOf(ByteBuffer context, CounterId id)
     {
-        int headerLength = headerLength(context);
+        int headerLength = headerLength(context, ByteBufferAccessor.instance);
         int offset = context.position() + headerLength;
 
         int left = 0;
@@ -764,7 +767,7 @@ public class CounterContext
         private ContextState(ByteBuffer context)
         {
             this.context = context;
-            this.headerLength = this.bodyOffset = headerLength(context);
+            this.headerLength = this.bodyOffset = headerLength(context, ByteBufferAccessor.instance);
             this.headerOffset = HEADER_SIZE_LENGTH;
             updateIsGlobalOrLocal();
         }
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
index cdb61c9..d3c6826 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
@@ -77,7 +77,7 @@ public interface ClusteringIndexFilter
      *
      * @return a new filter that selects results coming after {@code lastReturned}.
      */
-    public ClusteringIndexFilter forPaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive);
+    public ClusteringIndexFilter forPaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive);
 
     /**
      * Returns whether we can guarantee that a given cached partition contains all the data selected by this filter.
@@ -109,7 +109,7 @@ public interface ClusteringIndexFilter
      *
      * @return whether the row with clustering {@code clustering} is selected by this filter.
      */
-    public boolean selects(Clustering clustering);
+    public boolean selects(Clustering<?> clustering);
 
     /**
      * Returns an iterator that only returns the rows of the provided iterator that this filter selects.
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
index 63815a1..ff63aca 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
@@ -42,14 +42,14 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
 
     // This could be empty if selectedColumns only has static columns (in which case the filter still
     // selects the static row)
-    private final NavigableSet<Clustering> clusterings;
+    private final NavigableSet<Clustering<?>> clusterings;
 
     // clusterings is always in clustering order (because we need it that way in some methods), but we also
     // sometimes need those clustering in "query" order (i.e. in reverse clustering order if the query is
     // reversed), so we keep that too for simplicity.
-    private final NavigableSet<Clustering> clusteringsInQueryOrder;
+    private final NavigableSet<Clustering<?>> clusteringsInQueryOrder;
 
-    public ClusteringIndexNamesFilter(NavigableSet<Clustering> clusterings, boolean reversed)
+    public ClusteringIndexNamesFilter(NavigableSet<Clustering<?>> clusterings, boolean reversed)
     {
         super(reversed);
         assert !clusterings.contains(Clustering.STATIC_CLUSTERING);
@@ -65,7 +65,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
      * @return the set of requested clustering in clustering order (note that
      * this is always in clustering order even if the query is reversed).
      */
-    public NavigableSet<Clustering> requestedRows()
+    public NavigableSet<Clustering<?>> requestedRows()
     {
         return clusterings;
     }
@@ -77,16 +77,16 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
         return clusterings.isEmpty();
     }
 
-    public boolean selects(Clustering clustering)
+    public boolean selects(Clustering<?> clustering)
     {
         return clusterings.contains(clustering);
     }
 
-    public ClusteringIndexNamesFilter forPaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive)
+    public ClusteringIndexNamesFilter forPaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive)
     {
-        NavigableSet<Clustering> newClusterings = reversed ?
-                                                  clusterings.headSet(lastReturned, inclusive) :
-                                                  clusterings.tailSet(lastReturned, inclusive);
+        NavigableSet<Clustering<?>> newClusterings = reversed ?
+                                                     clusterings.headSet(lastReturned, inclusive) :
+                                                     clusterings.tailSet(lastReturned, inclusive);
 
         return new ClusteringIndexNamesFilter(newClusterings, reversed);
     }
@@ -131,15 +131,15 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
     public Slices getSlices(TableMetadata metadata)
     {
         Slices.Builder builder = new Slices.Builder(metadata.comparator, clusteringsInQueryOrder.size());
-        for (Clustering clustering : clusteringsInQueryOrder)
+        for (Clustering<?> clustering : clusteringsInQueryOrder)
             builder.add(Slice.make(clustering));
         return builder.build();
     }
 
     public UnfilteredRowIterator getUnfilteredRowIterator(final ColumnFilter columnFilter, final Partition partition)
     {
-        final Iterator<Clustering> clusteringIter = clusteringsInQueryOrder.iterator();
-        final SearchIterator<Clustering, Row> searcher = partition.searchIterator(columnFilter, reversed);
+        final Iterator<Clustering<?>> clusteringIter = clusteringsInQueryOrder.iterator();
+        final SearchIterator<Clustering<?>, Row> searcher = partition.searchIterator(columnFilter, reversed);
 
         return new AbstractUnfilteredRowIterator(partition.metadata(),
                                                  partition.partitionKey(),
@@ -169,7 +169,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
         List<ByteBuffer> maxClusteringValues = sstable.getSSTableMetadata().maxClusteringValues;
 
         // If any of the requested clustering is within the bounds covered by the sstable, we need to include the sstable
-        for (Clustering clustering : clusterings)
+        for (Clustering<?> clustering : clusterings)
         {
             if (Slice.make(clustering).intersects(comparator, minClusteringValues, maxClusteringValues))
                 return true;
@@ -182,7 +182,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
         StringBuilder sb = new StringBuilder();
         sb.append("names(");
         int i = 0;
-        for (Clustering clustering : clusterings)
+        for (Clustering<?> clustering : clusterings)
             sb.append(i++ == 0 ? "" : ", ").append(clustering.toString(metadata));
         if (reversed)
             sb.append(", reversed");
@@ -198,7 +198,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
         sb.append('(').append(ColumnMetadata.toCQLString(metadata.clusteringColumns())).append(')');
         sb.append(clusterings.size() == 1 ? " = " : " IN (");
         int i = 0;
-        for (Clustering clustering : clusterings)
+        for (Clustering<?> clustering : clusterings)
             sb.append(i++ == 0 ? "" : ", ").append('(').append(clustering.toCQLString(metadata)).append(')');
         sb.append(clusterings.size() == 1 ? "" : ")");
 
@@ -229,7 +229,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
     {
         ClusteringComparator comparator = (ClusteringComparator)clusterings.comparator();
         out.writeUnsignedVInt(clusterings.size());
-        for (Clustering clustering : clusterings)
+        for (Clustering<?> clustering : clusterings)
             Clustering.serializer.serialize(clustering, out, version, comparator.subtypes());
     }
 
@@ -237,7 +237,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
     {
         ClusteringComparator comparator = (ClusteringComparator)clusterings.comparator();
         long size = TypeSizes.sizeofUnsignedVInt(clusterings.size());
-        for (Clustering clustering : clusterings)
+        for (Clustering<?> clustering : clusterings)
             size += Clustering.serializer.serializedSize(clustering, version, comparator.subtypes());
         return size;
     }
@@ -247,7 +247,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
         public ClusteringIndexFilter deserialize(DataInputPlus in, int version, TableMetadata metadata, boolean reversed) throws IOException
         {
             ClusteringComparator comparator = metadata.comparator;
-            BTreeSet.Builder<Clustering> clusterings = BTreeSet.builder(comparator);
+            BTreeSet.Builder<Clustering<?>> clusterings = BTreeSet.builder(comparator);
             int size = (int)in.readUnsignedVInt();
             for (int i = 0; i < size; i++)
                 clusterings.add(Clustering.serializer.deserialize(in, version, comparator.subtypes()));
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
index 9490adf..6711325 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
@@ -56,12 +56,12 @@ public class ClusteringIndexSliceFilter extends AbstractClusteringIndexFilter
         return slices.size() == 1 && !slices.hasLowerBound() && !slices.hasUpperBound();
     }
 
-    public boolean selects(Clustering clustering)
+    public boolean selects(Clustering<?> clustering)
     {
         return slices.selects(clustering);
     }
 
-    public ClusteringIndexSliceFilter forPaging(ClusteringComparator comparator, Clustering lastReturned, boolean inclusive)
+    public ClusteringIndexSliceFilter forPaging(ClusteringComparator comparator, Clustering<?> lastReturned, boolean inclusive)
     {
         Slices newSlices = slices.forPaging(comparator, lastReturned, inclusive, reversed);
         return slices == newSlices
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
index 20a1656..30c3ed7 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
@@ -276,7 +276,7 @@ public class ColumnFilter
      * @param cells the cells to filter.
      * @return a filtered iterator that only include the cells from {@code cells} that are included by this filter.
      */
-    public Iterator<Cell> filterComplexCells(ColumnMetadata column, Iterator<Cell> cells)
+    public Iterator<Cell<?>> filterComplexCells(ColumnMetadata column, Iterator<Cell<?>> cells)
     {
         Tester tester = newTester(column);
         if (tester == null)
diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java
index c8585b1..aebb067 100644
--- a/src/java/org/apache/cassandra/db/filter/RowFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java
@@ -205,14 +205,14 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
      * Returns true if all of the expressions within this filter that apply to the clustering key are satisfied by
      * the given Clustering, false otherwise.
      */
-    public boolean clusteringKeyRestrictionsAreSatisfiedBy(Clustering clustering)
+    public boolean clusteringKeyRestrictionsAreSatisfiedBy(Clustering<?> clustering)
     {
         for (Expression e : expressions)
         {
             if (!e.column.isClusteringColumn())
                 continue;
 
-            if (!e.operator().isSatisfiedBy(e.column.type, clustering.get(e.column.position()), e.value))
+            if (!e.operator().isSatisfiedBy(e.column.type, clustering.bufferAt(e.column.position()), e.value))
             {
                 return false;
             }
@@ -448,10 +448,10 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
                          ? CompositeType.extractComponent(partitionKey.getKey(), column.position())
                          : partitionKey.getKey();
                 case CLUSTERING:
-                    return row.clustering().get(column.position());
+                    return row.clustering().bufferAt(column.position());
                 default:
-                    Cell cell = row.getCell(column);
-                    return cell == null ? null : cell.value();
+                    Cell<?> cell = row.getCell(column);
+                    return cell == null ? null : cell.buffer();
             }
         }
 
@@ -616,7 +616,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
                             if (foundValue == null)
                                 return false;
 
-                            ByteBuffer counterValue = LongType.instance.decompose(CounterContext.instance().total(foundValue));
+                            ByteBuffer counterValue = LongType.instance.decompose(CounterContext.instance().total(foundValue, ByteBufferAccessor.instance));
                             return operator.isSatisfiedBy(LongType.instance, counterValue, value);
                         }
                         else
@@ -645,7 +645,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
                         ComplexColumnData complexData = row.getComplexColumnData(column);
                         if (complexData != null)
                         {
-                            for (Cell cell : complexData)
+                            for (Cell<?> cell : complexData)
                             {
                                 if (type.kind == CollectionType.Kind.SET)
                                 {
@@ -654,7 +654,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
                                 }
                                 else
                                 {
-                                    if (type.valueComparator().compare(cell.value(), value) == 0)
+                                    if (type.valueComparator().compare(cell.buffer(), value) == 0)
                                         return true;
                                 }
                             }
@@ -760,7 +760,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         @Override
         public ByteBuffer getIndexValue()
         {
-            return CompositeType.build(key, value);
+            return CompositeType.build(ByteBufferAccessor.instance, key, value);
         }
 
         public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row)
@@ -776,8 +776,8 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
             MapType<?, ?> mt = (MapType<?, ?>)column.type;
             if (column.isComplex())
             {
-                Cell cell = row.getCell(column, CellPath.create(key));
-                return cell != null && mt.valueComparator().compare(cell.value(), value) == 0;
+                Cell<?> cell = row.getCell(column, CellPath.create(key));
+                return cell != null && mt.valueComparator().compare(cell.buffer(), value) == 0;
             }
             else
             {
diff --git a/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java b/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
index f7371ec..28d49ae 100644
--- a/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
+++ b/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
@@ -26,13 +26,13 @@ import org.apache.cassandra.db.marshal.*;
 
 public class TombstoneOverwhelmingException extends RuntimeException
 {
-    public TombstoneOverwhelmingException(int numTombstones, String query, TableMetadata metadata, DecoratedKey lastPartitionKey, ClusteringPrefix lastClustering)
+    public TombstoneOverwhelmingException(int numTombstones, String query, TableMetadata metadata, DecoratedKey lastPartitionKey, ClusteringPrefix<?> lastClustering)
     {
         super(String.format("Scanned over %d tombstones during query '%s' (last scanned row token was %s and partion key was (%s)); query aborted",
                             numTombstones, query, lastPartitionKey.getToken(), makePKString(metadata, lastPartitionKey.getKey(), lastClustering)));
     }
 
-    private static String makePKString(TableMetadata metadata, ByteBuffer partitionKey, ClusteringPrefix clustering)
+    private static String makePKString(TableMetadata metadata, ByteBuffer partitionKey, ClusteringPrefix<?> clustering)
     {
         StringBuilder sb = new StringBuilder();
 
@@ -61,7 +61,7 @@ public class TombstoneOverwhelmingException extends RuntimeException
             sb.append(")");
 
         for (int i = 0; i < clustering.size(); i++)
-            sb.append(", ").append(metadata.comparator.subtype(i).getString(clustering.get(i)));
+            sb.append(", ").append(clustering.stringAt(i, metadata.comparator));
 
         return sb.toString();
     }
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
index 0248629..24d2834 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
@@ -43,72 +43,81 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         super(ComparisonType.CUSTOM);
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
 
-        ByteBuffer bb1 = o1.duplicate();
-        ByteBuffer bb2 = o2.duplicate();
-
-        boolean isStatic1 = readIsStatic(bb1);
-        boolean isStatic2 = readIsStatic(bb2);
-        if (isStatic1 != isStatic2)
-            return isStatic1 ? -1 : 1;
+        boolean isStaticL = readIsStatic(left, accessorL);
+        boolean isStaticR = readIsStatic(right, accessorR);
+        if (isStaticL != isStaticR)
+            return isStaticL ? -1 : 1;
 
         int i = 0;
 
-        ByteBuffer previous = null;
+        VL previous = null;
+        int offsetL = startingOffset(isStaticL);
+        int offsetR = startingOffset(isStaticR);
 
-        while (bb1.remaining() > 0 && bb2.remaining() > 0)
+        while (!accessorL.isEmptyFromOffset(left, offsetL) && !accessorR.isEmptyFromOffset(right, offsetL))
         {
-            AbstractType<?> comparator = getComparator(i, bb1, bb2);
+            AbstractType<?> comparator = getComparator(i, left, accessorL, right, accessorR, offsetL, offsetR);
+            offsetL += getComparatorSize(i, left, accessorL, offsetL);
+            offsetR += getComparatorSize(i, right, accessorR, offsetR);
 
-            ByteBuffer value1 = ByteBufferUtil.readBytesWithShortLength(bb1);
-            ByteBuffer value2 = ByteBufferUtil.readBytesWithShortLength(bb2);
+            VL value1 = accessorL.sliceWithShortLength(left, offsetL);
+            offsetL += accessorL.sizeWithShortLength(value1);
+            VR value2 = accessorR.sliceWithShortLength(right, offsetR);
+            offsetR += accessorR.sizeWithShortLength(value2);
 
-            int cmp = comparator.compareCollectionMembers(value1, value2, previous);
+            int cmp = comparator.compareCollectionMembers(value1, accessorL, value2, accessorR, previous);
             if (cmp != 0)
                 return cmp;
 
             previous = value1;
 
-            byte b1 = bb1.get();
-            byte b2 = bb2.get();
-            if (b1 != b2)
-                return b1 - b2;
+            byte bL = accessorL.getByte(left, offsetL++);
+            byte bR = accessorR.getByte(right, offsetR++);
+            if (bL != bR)
+                return bL - bR;
 
             ++i;
         }
 
-        if (bb1.remaining() == 0)
-            return bb2.remaining() == 0 ? 0 : -1;
+        if (accessorL.isEmptyFromOffset(left, offsetL))
+            return accessorR.sizeFromOffset(right, offsetR) == 0 ? 0 : -1;
 
-        // bb1.remaining() > 0 && bb2.remaining() == 0
+        // left.remaining() > 0 && right.remaining() == 0
         return 1;
     }
 
     // Check if the provided BB represents a static name and advance the
     // buffer to the real beginning if so.
-    protected abstract boolean readIsStatic(ByteBuffer bb);
+    protected abstract <V> boolean readIsStatic(V value, ValueAccessor<V> accessor);
+
+    protected abstract int startingOffset(boolean isStatic);
 
     /**
      * Split a composite column names into it's components.
      */
-    public ByteBuffer[] split(ByteBuffer name)
+    public ByteBuffer[] split(ByteBuffer bb)
     {
         List<ByteBuffer> l = new ArrayList<ByteBuffer>();
-        ByteBuffer bb = name.duplicate();
-        readIsStatic(bb);
+        boolean isStatic = readIsStatic(bb, ByteBufferAccessor.instance);
+        int offset = startingOffset(isStatic);
+
         int i = 0;
-        while (bb.remaining() > 0)
+        while (!ByteBufferAccessor.instance.isEmptyFromOffset(bb, offset))
         {
-            getComparator(i++, bb);
-            l.add(ByteBufferUtil.readBytesWithShortLength(bb));
-            bb.get(); // skip end-of-component
+            offset += getComparatorSize(i++, bb, ByteBufferAccessor.instance, offset);
+            ByteBuffer value = ByteBufferAccessor.instance.sliceWithShortLength(bb, offset);
+            offset += ByteBufferAccessor.instance.sizeWithShortLength(value);
+            l.add(value);
+            offset++; // skip end-of-component
         }
         return l.toArray(new ByteBuffer[l.size()]);
     }
+
     private static final String COLON = ":";
     private static final Pattern COLON_PAT = Pattern.compile(COLON);
     private static final String ESCAPED_COLON = "\\\\:";
@@ -165,24 +174,27 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         return res;
     }
 
-    public String getString(ByteBuffer bytes)
+    public <V> String getString(V input, ValueAccessor<V> accessor)
     {
         StringBuilder sb = new StringBuilder();
-        ByteBuffer bb = bytes.duplicate();
-        readIsStatic(bb);
+        boolean isStatic  = readIsStatic(input, accessor);
+        int offset = startingOffset(isStatic);
+        int startOffset = offset;
 
         int i = 0;
-        while (bb.remaining() > 0)
+        while (!accessor.isEmptyFromOffset(input, offset))
         {
-            if (bb.remaining() != bytes.remaining())
+            if (offset != startOffset)
                 sb.append(":");
 
-            AbstractType<?> comparator = getAndAppendComparator(i, bb, sb);
-            ByteBuffer value = ByteBufferUtil.readBytesWithShortLength(bb);
+            AbstractType<?> comparator = getAndAppendComparator(i, input, accessor, sb, offset);
+            offset += getComparatorSize(i, input, accessor, offset);
+            V value = accessor.sliceWithShortLength(input, offset);
+            offset += accessor.sizeWithShortLength(value);
 
-            sb.append(escape(comparator.getString(value)));
+            sb.append(escape(comparator.getString(value, accessor)));
 
-            byte b = bb.get();
+            byte b = accessor.getByte(input, offset++);
             if (b != 0)
             {
                 sb.append(b < 0 ? ":_" : ":!");
@@ -258,31 +270,39 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
     }
 
     @Override
-    public void validate(ByteBuffer bytes) throws MarshalException
+    public void validate(ByteBuffer bb) throws MarshalException
+    {
+        validate(bb, ByteBufferAccessor.instance);
+    }
+
+    public  <V> void validate(V input, ValueAccessor<V> accessor)
     {
-        ByteBuffer bb = bytes.duplicate();
-        readIsStatic(bb);
+        boolean isStatic = readIsStatic(input, accessor);
+        int offset = startingOffset(isStatic);
 
         int i = 0;
-        ByteBuffer previous = null;
-        while (bb.remaining() > 0)
+        V previous = null;
+        while (!accessor.isEmptyFromOffset(input, offset))
         {
-            AbstractType<?> comparator = validateComparator(i, bb);
+            AbstractType<?> comparator = validateComparator(i, input, accessor, offset);
+            offset += getComparatorSize(i, input, accessor, offset);
 
-            if (bb.remaining() < 2)
+            if (accessor.sizeFromOffset(input, offset) < 2)
                 throw new MarshalException("Not enough bytes to read value size of component " + i);
-            int length = ByteBufferUtil.readShortLength(bb);
+            int length = accessor.getShort(input, offset);
+            offset += 2;
 
-            if (bb.remaining() < length)
+            if (accessor.sizeFromOffset(input, offset) < length)
                 throw new MarshalException("Not enough bytes to read value of component " + i);
-            ByteBuffer value = ByteBufferUtil.readBytes(bb, length);
+            V value = accessor.slice(input, offset, length);
+            offset += length;
 
-            comparator.validateCollectionMember(value, previous);
+            comparator.validateCollectionMember(value, previous, accessor);
 
-            if (bb.remaining() == 0)
+            if (accessor.isEmptyFromOffset(input, offset))
                 throw new MarshalException("Not enough bytes to read the end-of-component byte of component" + i);
-            byte b = bb.get();
-            if (b != 0 && bb.remaining() != 0)
+            byte b = accessor.getByte(input, offset++);
+            if (b != 0 && !accessor.isEmptyFromOffset(input, offset))
                 throw new MarshalException("Invalid bytes remaining after an end-of-component at component" + i);
 
             previous = value;
@@ -297,28 +317,29 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         return BytesSerializer.instance;
     }
 
+    abstract protected <V> int getComparatorSize(int i, V value, ValueAccessor<V> accessor, int offset);
     /**
      * @return the comparator for the given component. static CompositeType will consult
      * @param i DynamicCompositeType will read the type information from @param bb
-     * @param bb name of type definition
+     * @param value name of type definition
      */
-    abstract protected AbstractType<?> getComparator(int i, ByteBuffer bb);
+    abstract protected <V> AbstractType<?> getComparator(int i, V value, ValueAccessor<V> accessor, int offset);
 
     /**
      * Adds DynamicCompositeType type information from @param bb1 to @param bb2.
      * @param i is ignored.
      */
-    abstract protected AbstractType<?> getComparator(int i, ByteBuffer bb1, ByteBuffer bb2);
+    abstract protected <VL, VR> AbstractType<?> getComparator(int i, VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR, int offsetL, int offsetR);
 
     /**
      * Adds type information from @param bb to @param sb.  @param i is ignored.
      */
-    abstract protected AbstractType<?> getAndAppendComparator(int i, ByteBuffer bb, StringBuilder sb);
+    abstract protected <V> AbstractType<?> getAndAppendComparator(int i, V value, ValueAccessor<V> accessor, StringBuilder sb, int offset);
 
     /**
      * Like getComparator, but validates that @param i does not exceed the defined range
      */
-    abstract protected AbstractType<?> validateComparator(int i, ByteBuffer bb) throws MarshalException;
+    abstract protected <V> AbstractType<?> validateComparator(int i, V value, ValueAccessor<V> accessor, int offset) throws MarshalException;
 
     /**
      * Used by fromString
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 0a34785..0b47644 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -28,14 +28,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.cql3.AssignmentTestable;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -43,7 +39,6 @@ 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.FastByteOperations;
 import org.github.jamm.Unmetered;
 
 import static org.apache.cassandra.db.marshal.AbstractType.ComparisonType.CUSTOM;
@@ -59,8 +54,6 @@ import static org.apache.cassandra.db.marshal.AbstractType.ComparisonType.CUSTOM
 @Unmetered
 public abstract class AbstractType<T> implements Comparator<ByteBuffer>, AssignmentTestable
 {
-    private static final Logger logger = LoggerFactory.getLogger(AbstractType.class);
-
     public final Comparator<ByteBuffer> reverseComparator;
 
     public enum ComparisonType
@@ -83,6 +76,7 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
 
     public final ComparisonType comparisonType;
     public final boolean isByteOrderComparable;
+    public final ValueComparators comparatorSet;
 
     protected AbstractType(ComparisonType comparisonType)
     {
@@ -91,7 +85,7 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
         reverseComparator = (o1, o2) -> AbstractType.this.compare(o2, o1);
         try
         {
-            Method custom = getClass().getMethod("compareCustom", ByteBuffer.class, ByteBuffer.class);
+            Method custom = getClass().getMethod("compareCustom", Object.class, ValueAccessor.class, Object.class, ValueAccessor.class);
             if ((custom.getDeclaringClass() == AbstractType.class) == (comparisonType == CUSTOM))
                 throw new IllegalStateException((comparisonType == CUSTOM ? "compareCustom must be overridden if ComparisonType is CUSTOM"
                                                                          : "compareCustom should not be overridden if ComparisonType is not CUSTOM")
@@ -101,6 +95,17 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
         {
             throw new IllegalStateException();
         }
+
+        comparatorSet = new ValueComparators((l, r) -> compare(l, ByteArrayAccessor.instance, r, ByteArrayAccessor.instance),
+                                             (l, r) -> compare(l, ByteBufferAccessor.instance, r, ByteBufferAccessor.instance));
+    }
+
+    static <VL, VR, T extends Comparable<T>> int compareComposed(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR, AbstractType<T> type)
+    {
+        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
+
+        return type.compose(left, accessorL).compareTo(type.compose(right, accessorR));
     }
 
     public static List<String> asCQLTypeStringList(List<AbstractType<?>> abstractTypes)
@@ -111,26 +116,36 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
         return r;
     }
 
-    public T compose(ByteBuffer bytes)
+    public final T compose(ByteBuffer bytes)
     {
         return getSerializer().deserialize(bytes);
     }
 
+    public <V> T compose(V value, ValueAccessor<V> accessor)
+    {
+        return getSerializer().deserialize(value, accessor);
+    }
+
     public ByteBuffer decompose(T value)
     {
         return getSerializer().serialize(value);
     }
 
     /** get a string representation of the bytes used for various identifier (NOT just for log messages) */
-    public String getString(ByteBuffer bytes)
+    public <V> String getString(V value, ValueAccessor<V> accessor)
     {
-        if (bytes == null)
+        if (value == null)
             return "null";
 
         TypeSerializer<T> serializer = getSerializer();
-        serializer.validate(bytes);
+        serializer.validate(value, accessor);
+
+        return serializer.toString(serializer.deserialize(value, accessor));
+    }
 
-        return serializer.toString(serializer.deserialize(bytes));
+    public final String getString(ByteBuffer bytes)
+    {
+        return getString(bytes, ByteBufferAccessor.instance);
     }
 
     /** get a byte representation of the given string. */
@@ -156,29 +171,46 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
         return '"' + Objects.toString(getSerializer().deserialize(buffer), "") + '"';
     }
 
+    public <V> String toJSONString(V value, ValueAccessor<V> accessor, ProtocolVersion protocolVersion)
+    {
+        return toJSONString(accessor.toBuffer(value), protocolVersion); // FIXME
+    }
+
     /* validate that the byte array is a valid sequence for the type we are supposed to be comparing */
     public void validate(ByteBuffer bytes) throws MarshalException
     {
-        getSerializer().validate(bytes);
+        validate(bytes, ByteBufferAccessor.instance);
+    }
+
+    public <V> void validate(V value, ValueAccessor<V> accessor) throws MarshalException
+    {
+        getSerializer().validate(value, accessor);
     }
 
     public final int compare(ByteBuffer left, ByteBuffer right)
     {
-        return isByteOrderComparable
-               ? FastByteOperations.compareUnsigned(left, right)
-               : compareCustom(left, right);
+        return compare(left, ByteBufferAccessor.instance, right, ByteBufferAccessor.instance);
+    }
+
+    public final <VL, VR> int compare(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
+    {
+        return isByteOrderComparable ? ValueAccessor.compare(left, accessorL, right, accessorR) : compareCustom(left, accessorL, right, accessorR);
     }
 
     /**
      * Implement IFF ComparisonType is CUSTOM
      *
-     * Compares the ByteBuffer representation of two instances of this class,
+     * Compares the byte representation of two instances of this class,
      * for types where this cannot be done by simple in-order comparison of the
      * unsigned bytes
      *
      * Standard Java compare semantics
+     * @param left
+     * @param accessorL
+     * @param right
+     * @param accessorR
      */
-    public int compareCustom(ByteBuffer left, ByteBuffer right)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
         throw new UnsupportedOperationException();
     }
@@ -191,9 +223,9 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
      * @param cellValue ByteBuffer representing cell value
      * @throws MarshalException
      */
-    public void validateCellValue(ByteBuffer cellValue) throws MarshalException
+    public <V> void validateCellValue(V cellValue, ValueAccessor<V> accessor) throws MarshalException
     {
-        validate(cellValue);
+        validate(cellValue, accessor);
     }
 
     /* Most of our internal type should override that. */
@@ -303,19 +335,14 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
      *
      * Unless you're doing something very similar to CollectionsType, you shouldn't override this.
      */
-    public int compareCollectionMembers(ByteBuffer v1, ByteBuffer v2, ByteBuffer collectionName)
+    public <VL, VR> int compareCollectionMembers(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR, VL collectionName)
     {
-        return compare(v1, v2);
+        return compare(left, accessorL, right, accessorR);
     }
 
-    /**
-     * An alternative validation function used by CollectionsType in conjunction with CompositeType.
-     *
-     * This is similar to the compare function above.
-     */
-    public void validateCollectionMember(ByteBuffer bytes, ByteBuffer collectionName) throws MarshalException
+    public <V> void validateCollectionMember(V value, V collectionName, ValueAccessor<V> accessor) throws MarshalException
     {
-        validate(bytes);
+        getSerializer().validate(value, accessor);
     }
 
     public boolean isCollection()
@@ -383,15 +410,6 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
     }
 
     /**
-     * The number of subcomponents this type has.
-     * This is always 1, i.e. the type has only itself as "subcomponents", except for CompositeType.
-     */
-    public int componentsCount()
-    {
-        return 1;
-    }
-
-    /**
      * Return a list of the "subcomponents" this type has.
      * This always return a singleton list with the type itself except for CompositeType.
      */
@@ -411,32 +429,53 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
     // This assumes that no empty values are passed
     public void writeValue(ByteBuffer value, DataOutputPlus out) throws IOException
     {
-        assert value.hasRemaining();
+        writeValue(value, ByteBufferAccessor.instance, out);
+    }
+
+    // This assumes that no empty values are passed
+    public  <V> void writeValue(V value, ValueAccessor<V> accessor, DataOutputPlus out) throws IOException
+    {
+        assert !accessor.isEmpty(value);
         if (valueLengthIfFixed() >= 0)
-            out.write(value);
+            accessor.write(value, out);
         else
-            ByteBufferUtil.writeWithVIntLength(value, out);
+            accessor.writeWithVIntLength(value, out);
     }
 
     public long writtenLength(ByteBuffer value)
     {
-        assert value.hasRemaining() : "bytes should not be empty for type " + this;
+        return writtenLength(value, ByteBufferAccessor.instance);
+    }
+
+    public <V> long writtenLength(V value, ValueAccessor<V> accessor)
+    {
+        assert !accessor.isEmpty(value) : "bytes should not be empty for type " + this;
         return valueLengthIfFixed() >= 0
-             ? value.remaining()
-             : TypeSizes.sizeofWithVIntLength(value);
+               ? accessor.size(value)
+               : accessor.sizeWithVIntLength(value);
+    }
+
+    public ByteBuffer readBuffer(DataInputPlus in) throws IOException
+    {
+        return readBuffer(in, Integer.MAX_VALUE);
     }
 
-    public ByteBuffer readValue(DataInputPlus in) throws IOException
+    public ByteBuffer readBuffer(DataInputPlus in, int maxValueSize) throws IOException
     {
-        return readValue(in, Integer.MAX_VALUE);
+        return read(ByteBufferAccessor.instance, in, maxValueSize);
     }
 
-    public ByteBuffer readValue(DataInputPlus in, int maxValueSize) throws IOException
+    public byte[] readArray(DataInputPlus in, int maxValueSize) throws IOException
+    {
+        return read(ByteArrayAccessor.instance, in, maxValueSize);
+    }
+
+    public <V> V read(ValueAccessor<V> accessor, DataInputPlus in, int maxValueSize) throws IOException
     {
         int length = valueLengthIfFixed();
 
         if (length >= 0)
-            return ByteBufferUtil.read(in, length);
+            return accessor.read(in, length);
         else
         {
             int l = (int)in.readUnsignedVInt();
@@ -448,7 +487,7 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
                                                     "which is set via max_value_size_in_mb in cassandra.yaml",
                                                     l, maxValueSize));
 
-            return ByteBufferUtil.read(in, l);
+            return accessor.read(in, l);
         }
     }
 
@@ -461,7 +500,12 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
             ByteBufferUtil.skipWithVIntLength(in);
     }
 
-    public boolean referencesUserType(ByteBuffer name)
+    public final boolean referencesUserType(ByteBuffer name)
+    {
+        return referencesUserType(name, ByteBufferAccessor.instance);
+    }
+
+    public <V> boolean referencesUserType(V name, ValueAccessor<V> accessor)
     {
         return false;
     }
diff --git a/src/java/org/apache/cassandra/db/marshal/BooleanType.java b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
index 475cae6..4ef5f95 100644
--- a/src/java/org/apache/cassandra/db/marshal/BooleanType.java
+++ b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
@@ -43,17 +43,15 @@ public class BooleanType extends AbstractType<Boolean>
         return true;
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
 
         // False is 0, True is anything else, makes False sort before True.
-        byte b1 = o1.get(o1.position());
-        byte b2 = o2.get(o2.position());
-        if (b1 == 0)
-            return b2 == 0 ? 0 : -1;
-        return b2 == 0 ? 1 : 0;
+        int v1 = accessorL.getByte(left, 0) == 0 ? 0 : 1;
+        int v2 = accessorR.getByte(right, 0) == 0 ? 0 : 1;
+        return v1 - v2;
     }
 
     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
new file mode 100644
index 0000000..92f18a2
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/ByteArrayAccessor.java
@@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.marshal;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.UUID;
+
+import org.apache.cassandra.db.Digest;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FastByteOperations;
+import org.apache.cassandra.utils.Hex;
+import org.apache.cassandra.utils.UUIDGen;
+
+public class ByteArrayAccessor implements ValueAccessor<byte[]>
+{
+    public static final ValueAccessor<byte[]> instance = new ByteArrayAccessor();
+    public static final ObjectFactory<byte[]> factory = ByteArrayObjectFactory.instance;
+    private static final byte[] EMPTY = new byte[0];
+
+    private ByteArrayAccessor() {}
+
+    public int size(byte[] value)
+    {
+        return value.length;
+    }
+
+    public byte[][] createArray(int length)
+    {
+        return new byte[length][];
+    }
+
+    public void write(byte[] value, DataOutputPlus out) throws IOException
+    {
+        out.write(value);
+    }
+
+    public void write(byte[] value, ByteBuffer out)
+    {
+        out.put(value);
+    }
+
+    public <V2> int copyTo(byte[] src, int srcOffset, V2 dst, ValueAccessor<V2> dstAccessor, int dstOffset, int size)
+    {
+        dstAccessor.copyByteArrayTo(src, srcOffset, dst, dstOffset, size);
+        return size;
+    }
+
+    public int copyByteArrayTo(byte[] src, int srcOffset, byte[] dst, int dstOffset, int size)
+    {
+        FastByteOperations.copy(src, srcOffset, dst, dstOffset, size);
+        return size;
+    }
+
+    public int copyByteBufferTo(ByteBuffer src, int srcOffset, byte[] dst, int dstOffset, int size)
+    {
+        FastByteOperations.copy(src, src.position() + srcOffset, dst, dstOffset, size);
+        return size;
+    }
+
+    public void digest(byte[] value, int offset, int size, Digest digest)
+    {
+        digest.update(value, offset, size);
+    }
+
+    public byte[] read(DataInputPlus in, int length) throws IOException
+    {
+        byte[] b = new byte[length];
+        in.readFully(b);
+        return b;
+    }
+
+    public byte[] slice(byte[] input, int offset, int length)
+    {
+        return Arrays.copyOfRange(input, offset, offset + length);
+    }
+
+    public <V2> int compare(byte[] left, V2 right, ValueAccessor<V2> accessorR)
+    {
+        return accessorR.compareByteArrayTo(left, right);
+    }
+
+    public int compareByteArrayTo(byte[] left, byte[] right)
+    {
+        return ByteArrayUtil.compareUnsigned(left, right);
+    }
+
+    public int compareByteBufferTo(ByteBuffer left, byte[] right)
+    {
+        return ByteBufferUtil.compare(left, right);
+    }
+
+    public ByteBuffer toBuffer(byte[] value)
+    {
+        if (value == null)
+            return null;
+        return ByteBuffer.wrap(value);
+    }
+
+    public byte[] toArray(byte[] value)
+    {
+        return value;
+    }
+
+    public byte[] toArray(byte[] value, int offset, int length)
+    {
+        if (value == null)
+            return null;
+        if (offset == 0 && length == value.length)
+            return value;
+        return slice(value, offset, length);
+    }
+
+    public String toString(byte[] value, Charset charset) throws CharacterCodingException
+    {
+        return new String(value, charset);
+    }
+
+    public String toHex(byte[] value)
+    {
+        return Hex.bytesToHex(value);
+    }
+
+    public byte toByte(byte[] value)
+    {
+        return value[0];
+    }
+
+    public byte getByte(byte[] value, int offset)
+    {
+        return value[offset];
+    }
+
+    public short toShort(byte[] value)
+    {
+        return getShort(value, 0);
+    }
+
+    public short getShort(byte[] value, int offset)
+    {
+        return ByteArrayUtil.getShort(value, offset);
+    }
+
+    public int toInt(byte[] value)
+    {
+        return getInt(value, 0);
+    }
+
+    public int getInt(byte[] value, int offset)
+    {
+        return ByteArrayUtil.getInt(value, offset);
+    }
+
+    public long toLong(byte[] value)
+    {
+        return getLong(value, 0);
+    }
+
+    public long getLong(byte[] value, int offset)
+    {
+        return ByteArrayUtil.getLong(value, offset);
+    }
+
+    public float toFloat(byte[] value)
+    {
+        return ByteArrayUtil.getFloat(value, 0);
+    }
+
+    public double toDouble(byte[] value)
+    {
+        return ByteArrayUtil.getDouble(value, 0);
+    }
+
+    public UUID toUUID(byte[] value)
+    {
+        return new UUID(getLong(value, 0), getLong(value, 8));
+    }
+
+    public int putShort(byte[] dst, int offset, short value)
+    {
+        ByteArrayUtil.putShort(dst, offset, value);
+        return TypeSizes.SHORT_SIZE;
+    }
+
+    public int putInt(byte[] dst, int offset, int value)
+    {
+        ByteArrayUtil.putInt(dst, offset, value);
+        return TypeSizes.INT_SIZE;
+    }
+
+    public int putLong(byte[] dst, int offset, long value)
+    {
+        ByteArrayUtil.putLong(dst, offset, value);
+        return TypeSizes.LONG_SIZE;
+    }
+
+    public byte[] empty()
+    {
+        return EMPTY;
+    }
+
+    public byte[] valueOf(byte[] bytes)
+    {
+        return bytes;
+    }
+
+    public byte[] valueOf(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.getArray(bytes);
+    }
+
+    public byte[] valueOf(String s, Charset charset)
+    {
+        return ByteArrayUtil.bytes(s, charset);
+    }
+
+    public byte[] valueOf(UUID v)
+    {
+        return UUIDGen.decompose(v);
+    }
+
+    public byte[] valueOf(boolean v)
+    {
+        return v ? new byte[] {1} : new byte[] {0};
+    }
+
+    public byte[] valueOf(byte v)
+    {
+        return ByteArrayUtil.bytes(v);
+    }
+
+    public byte[] valueOf(short v)
+    {
+        return ByteArrayUtil.bytes(v);
+    }
+
+    public byte[] valueOf(int v)
+    {
+        return ByteArrayUtil.bytes(v);
+    }
+
+    public byte[] valueOf(long v)
+    {
+        return ByteArrayUtil.bytes(v);
+    }
+
+    public byte[] valueOf(float v)
+    {
+        return ByteArrayUtil.bytes(v);
+    }
+
+    public byte[] valueOf(double v)
+    {
+        return ByteArrayUtil.bytes(v);
+    }
+
+    public <V2> byte[] convert(V2 src, ValueAccessor<V2> accessor)
+    {
+        return accessor.toArray(src);
+    }
+
+    public byte[] allocate(int size)
+    {
+        return new byte[size];
+    }
+
+    public ObjectFactory<byte[]> factory()
+    {
+        return factory;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java b/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java
new file mode 100644
index 0000000..ea9bf11
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.marshal;
+
+import org.apache.cassandra.db.ArrayClustering;
+import org.apache.cassandra.db.ArrayClusteringBound;
+import org.apache.cassandra.db.ArrayClusteringBoundary;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringBoundary;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.rows.ArrayCell;
+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 ByteArrayObjectFactory implements ValueAccessor.ObjectFactory<byte[]>
+{
+    private static final Clustering<byte[]> EMPTY_CLUSTERING = new ArrayClustering()
+    {
+        public String toString(TableMetadata metadata)
+        {
+            return "EMPTY";
+        }
+    };
+
+    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][]);
+    /** 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][]);
+
+    public Cell<byte[]> cell(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, byte[] value, CellPath path)
+    {
+        return new ArrayCell(column, timestamp, ttl, localDeletionTime, value, path);
+    }
+
+    public Clustering<byte[]> clustering(byte[]... values)
+    {
+        return new ArrayClustering(values);
+    }
+
+    public Clustering<byte[]> clustering()
+    {
+        return EMPTY_CLUSTERING;
+    }
+
+    public ClusteringBound<byte[]> bound(ClusteringPrefix.Kind kind, byte[]... values)
+    {
+        return new ArrayClusteringBound(kind, values);
+    }
+
+    public ClusteringBound<byte[]> bound(ClusteringPrefix.Kind kind)
+    {
+        return kind.isStart() ? BOTTOM_BOUND : TOP_BOUND;
+    }
+
+    public ClusteringBoundary<byte[]> boundary(ClusteringPrefix.Kind kind, byte[]... values)
+    {
+        return new ArrayClusteringBoundary(kind, values);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java b/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java
new file mode 100644
index 0000000..df27d8b
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.marshal;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.util.UUID;
+
+import org.apache.cassandra.db.Digest;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FastByteOperations;
+import org.apache.cassandra.utils.UUIDGen;
+
+/**
+ * Accessor for ByteBuffer values. ByteBufferAccessor treates {@link ByteBuffer#position()} as index 0,
+ * and {@link ByteBuffer#remaining()} as the length.
+ */
+public class ByteBufferAccessor implements ValueAccessor<ByteBuffer>
+{
+    public static final ValueAccessor<ByteBuffer> instance = new ByteBufferAccessor();
+
+    private ByteBufferAccessor() {}
+
+    public int size(ByteBuffer value)
+    {
+        return value.remaining();
+    }
+
+    public ByteBuffer[] createArray(int length)
+    {
+        return new ByteBuffer[length];
+    }
+
+    public void write(ByteBuffer value, DataOutputPlus out) throws IOException
+    {
+        out.write(value);
+    }
+
+    public void write(ByteBuffer value, ByteBuffer out)
+    {
+        out.put(value.duplicate());
+    }
+
+    public <V2> int copyTo(ByteBuffer src, int srcOffset, V2 dst, ValueAccessor<V2> dstAccessor, int dstOffset, int size)
+    {
+        dstAccessor.copyByteBufferTo(src, srcOffset, dst, dstOffset, size);
+        return size;
+    }
+
+    public int copyByteArrayTo(byte[] src, int srcOffset, ByteBuffer dst, int dstOffset, int size)
+    {
+        FastByteOperations.copy(src, srcOffset, dst, dst.position() + dstOffset, size);
+        return size;
+    }
+
+    public int copyByteBufferTo(ByteBuffer src, int srcOffset, ByteBuffer dst, int dstOffset, int size)
+    {
+        FastByteOperations.copy(src, src.position() + srcOffset, dst, dst.position() + dstOffset, size);
+        return size;
+    }
+
+    public void digest(ByteBuffer value, int offset, int size, Digest digest)
+    {
+        digest.update(value, value.position() + offset, size);
+    }
+
+    public ByteBuffer read(DataInputPlus in, int length) throws IOException
+    {
+        return ByteBufferUtil.read(in, length);
+    }
+
+    public ByteBuffer slice(ByteBuffer input, int offset, int length)
+    {
+        ByteBuffer copy = input.duplicate();
+        copy.position(copy.position() + offset);
+        copy.limit(copy.position() + length);
+        return copy;
+    }
+
+    public <V2> int compare(ByteBuffer left, V2 right, ValueAccessor<V2> accessorR)
+    {
+        return accessorR.compareByteBufferTo(left, right);
+    }
+
+    public int compareByteArrayTo(byte[] left, ByteBuffer right)
+    {
+        return ByteBufferUtil.compare(left, right);
+    }
+
+    public int compareByteBufferTo(ByteBuffer left, ByteBuffer right)
+    {
+        return ByteBufferUtil.compareUnsigned(left, right);
+    }
+
+    public ByteBuffer toBuffer(ByteBuffer value)
+    {
+        return value;
+    }
+
+    public byte[] toArray(ByteBuffer value)
+    {
+        if (value == null)
+            return null;
+        return ByteBufferUtil.getArray(value);
+    }
+
+    public byte[] toArray(ByteBuffer value, int offset, int length)
+    {
+        if (value == null)
+            return null;
+        return ByteBufferUtil.getArray(value, value.position() + offset, length);
+    }
+
+    public String toString(ByteBuffer value, Charset charset) throws CharacterCodingException
+    {
+        return ByteBufferUtil.string(value, charset);
+    }
+
+    public ByteBuffer valueOf(UUID v)
+    {
+        return UUIDGen.toByteBuffer(v);
+    }
+
+    public String toHex(ByteBuffer value)
+    {
+        return ByteBufferUtil.bytesToHex(value);
+    }
+
+    public byte toByte(ByteBuffer value)
+    {
+        return ByteBufferUtil.toByte(value);
+    }
+
+    public byte getByte(ByteBuffer value, int offset)
+    {
+        return value.get(value.position() + offset);
+    }
+
+    public short toShort(ByteBuffer value)
+    {
+        return ByteBufferUtil.toShort(value);
+    }
+
+    public short getShort(ByteBuffer value, int offset)
+    {
+        return value.getShort(value.position() + offset);
+    }
+
+    public int toInt(ByteBuffer value)
+    {
+        return ByteBufferUtil.toInt(value);
+    }
+
+    public int getInt(ByteBuffer value, int offset)
+    {
+        return value.getInt(value.position() + offset);
+    }
+
+    public long toLong(ByteBuffer value)
+    {
+        return ByteBufferUtil.toLong(value);
+    }
+
+    public long getLong(ByteBuffer value, int offset)
+    {
+        return value.getLong(value.position() + offset);
+    }
+
+    public float toFloat(ByteBuffer value)
+    {
+        return ByteBufferUtil.toFloat(value);
+    }
+
+    public double toDouble(ByteBuffer value)
+    {
+        return ByteBufferUtil.toDouble(value);
+    }
+
+    public UUID toUUID(ByteBuffer value)
+    {
+        return UUIDGen.getUUID(value);
+    }
+
+    public int putShort(ByteBuffer dst, int offset, short value)
+    {
+        dst.putShort(dst.position() + offset, value);
+        return TypeSizes.SHORT_SIZE;
+    }
+
+    public int putInt(ByteBuffer dst, int offset, int value)
+    {
+        dst.putInt(dst.position() + offset, value);
+        return TypeSizes.INT_SIZE;
+    }
+
+    public int putLong(ByteBuffer dst, int offset, long value)
+    {
+        dst.putLong(dst.position() + offset, value);
+        return TypeSizes.LONG_SIZE;
+    }
+
+    public ByteBuffer empty()
+    {
+        return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+    }
+
+    public ByteBuffer valueOf(byte[] bytes)
+    {
+        return ByteBuffer.wrap(bytes);
+    }
+
+    public ByteBuffer valueOf(ByteBuffer bytes)
+    {
+        return bytes;
+    }
+
+    public ByteBuffer valueOf(String v, Charset charset)
+    {
+        return ByteBufferUtil.bytes(v, charset);
+    }
+
+    public ByteBuffer valueOf(boolean v)
+    {
+        return v ? ByteBuffer.wrap(new byte[] {1}) : ByteBuffer.wrap(new byte[] {0});
+    }
+
+    public ByteBuffer valueOf(byte v)
+    {
+        return ByteBufferUtil.bytes(v);
+    }
+
+    public ByteBuffer valueOf(short v)
+    {
+        return ByteBufferUtil.bytes(v);
+    }
+
+    public ByteBuffer valueOf(int v)
+    {
+        return ByteBufferUtil.bytes(v);
+    }
+
+    public ByteBuffer valueOf(long v)
+    {
+        return ByteBufferUtil.bytes(v);
+    }
+
+    public ByteBuffer valueOf(float v)
+    {
+        return ByteBufferUtil.bytes(v);
+    }
+
+    public ByteBuffer valueOf(double v)
+    {
+        return ByteBufferUtil.bytes(v);
+    }
+
+    public <V2> ByteBuffer convert(V2 src, ValueAccessor<V2> accessor)
+    {
+        return accessor.toBuffer(src);
+    }
+
+    public ByteBuffer allocate(int size)
+    {
+        return ByteBuffer.allocate(size);
+    }
+
+    public ObjectFactory<ByteBuffer> factory()
+    {
+        return ByteBufferObjectFactory.instance;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java b/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java
new file mode 100644
index 0000000..00f4646
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.BufferClustering;
+import org.apache.cassandra.db.BufferClusteringBound;
+import org.apache.cassandra.db.BufferClusteringBoundary;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringBoundary;
+import org.apache.cassandra.db.ClusteringPrefix;
+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]);
+    /** 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]);
+
+    static final ValueAccessor.ObjectFactory<ByteBuffer> instance = new ByteBufferObjectFactory();
+
+    private ByteBufferObjectFactory() {}
+
+    public Cell<ByteBuffer> cell(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, ByteBuffer value, CellPath path)
+    {
+        return new BufferCell(column, timestamp, ttl, localDeletionTime, value, path);
+    }
+
+    public Clustering<ByteBuffer> clustering(ByteBuffer... values)
+    {
+        return new BufferClustering(values);
+    }
+
+    public Clustering<ByteBuffer> clustering()
+    {
+        return EMPTY_CLUSTERING;
+    }
+
+    public ClusteringBound<ByteBuffer> bound(ClusteringPrefix.Kind kind, ByteBuffer... values)
+    {
+        return new BufferClusteringBound(kind, values);
+    }
+
+    public ClusteringBound<ByteBuffer> bound(ClusteringPrefix.Kind kind)
+    {
+        return kind.isStart() ? BOTTOM_BOUND : TOP_BOUND;
+    }
+
+    public ClusteringBoundary<ByteBuffer> boundary(ClusteringPrefix.Kind kind, ByteBuffer... values)
+    {
+        return new BufferClusteringBoundary(kind, values);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteType.java b/src/java/org/apache/cassandra/db/marshal/ByteType.java
index c19fdd9..f94f4bb 100644
--- a/src/java/org/apache/cassandra/db/marshal/ByteType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ByteType.java
@@ -37,9 +37,9 @@ public class ByteType extends NumberType<Byte>
         super(ComparisonType.CUSTOM);
     } // singleton
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        return o1.get(o1.position()) - o2.get(o2.position());
+        return accessorL.getByte(left, 0) - accessorR.getByte(right, 0);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
index b198e0c..0d627a5 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -84,7 +84,7 @@ public abstract class CollectionType<T> extends AbstractType<T>
     public abstract AbstractType<?> nameComparator();
     public abstract AbstractType<?> valueComparator();
 
-    protected abstract List<ByteBuffer> serializedValues(Iterator<Cell> cells);
+    protected abstract List<ByteBuffer> serializedValues(Iterator<Cell<?>> cells);
 
     @Override
     public abstract CollectionSerializer<T> getSerializer();
@@ -94,9 +94,9 @@ public abstract class CollectionType<T> extends AbstractType<T>
         return kind.makeCollectionReceiver(collection, isKey);
     }
 
-    public String getString(ByteBuffer bytes)
+    public <V> String getString(V value, ValueAccessor<V> accessor)
     {
-        return BytesType.instance.getString(bytes);
+        return BytesType.instance.getString(value, accessor);
     }
 
     public ByteBuffer fromString(String source)
@@ -117,12 +117,12 @@ public abstract class CollectionType<T> extends AbstractType<T>
     }
 
     @Override
-    public void validateCellValue(ByteBuffer cellValue) throws MarshalException
+    public <V> void validateCellValue(V cellValue, ValueAccessor<V> accessor) throws MarshalException
     {
         if (isMultiCell())
-            valueComparator().validateCellValue(cellValue);
+            valueComparator().validateCellValue(cellValue, accessor);
         else
-            super.validateCellValue(cellValue);
+            super.validateCellValue(cellValue, accessor);
     }
 
     /**
@@ -146,12 +146,12 @@ public abstract class CollectionType<T> extends AbstractType<T>
         return values.size();
     }
 
-    public ByteBuffer serializeForNativeProtocol(Iterator<Cell> cells, ProtocolVersion version)
+    public ByteBuffer serializeForNativeProtocol(Iterator<Cell<?>> cells, ProtocolVersion version)
     {
         assert isMultiCell();
         List<ByteBuffer> values = serializedValues(cells);
         int size = collectionSize(values);
-        return CollectionSerializer.pack(values, size, version);
+        return CollectionSerializer.pack(values, ByteBufferAccessor.instance, size, version);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index e3423ff..d7041b8 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -84,9 +84,31 @@ public class CompositeType extends AbstractCompositeType
         return getInstance(Arrays.asList(types));
     }
 
-    protected boolean readIsStatic(ByteBuffer bb)
+    protected static int startingOffsetInternal(boolean isStatic)
     {
-        return readStatic(bb);
+        return isStatic ? 2 : 0;
+    }
+
+    protected int startingOffset(boolean isStatic)
+    {
+        return startingOffsetInternal(isStatic);
+    }
+
+    protected static <V> boolean readIsStaticInternal(V value, ValueAccessor<V> accessor)
+    {
+        if (accessor.size(value) < 2)
+            return false;
+
+        int header = accessor.getShort(value, 0);
+        if ((header & 0xFFFF) != STATIC_MARKER)
+            return false;
+
+        return true;
+    }
+
+    protected <V> boolean readIsStatic(V value, ValueAccessor<V> accessor)
+    {
+        return readIsStaticInternal(value, accessor);
     }
 
     private static boolean readStatic(ByteBuffer bb)
@@ -116,7 +138,7 @@ public class CompositeType extends AbstractCompositeType
         this.types = ImmutableList.copyOf(types);
     }
 
-    protected AbstractType<?> getComparator(int i, ByteBuffer bb)
+    protected <V> AbstractType<?> getComparator(int i, V value, ValueAccessor<V> accessor, int offset)
     {
         try
         {
@@ -133,12 +155,12 @@ public class CompositeType extends AbstractCompositeType
         }
     }
 
-    protected AbstractType<?> getComparator(int i, ByteBuffer bb1, ByteBuffer bb2)
+    protected <VL, VR> AbstractType<?> getComparator(int i, VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR, int offsetL, int offsetR)
     {
-        return getComparator(i, bb1);
+        return getComparator(i, left, accessorL, offsetL);
     }
 
-    protected AbstractType<?> getAndAppendComparator(int i, ByteBuffer bb, StringBuilder sb)
+    protected <V> AbstractType<?> getAndAppendComparator(int i, V value, ValueAccessor<V> accessor, StringBuilder sb, int offset)
     {
         return types.get(i);
     }
@@ -148,13 +170,18 @@ public class CompositeType extends AbstractCompositeType
         return new StaticParsedComparator(types.get(i), part);
     }
 
-    protected AbstractType<?> validateComparator(int i, ByteBuffer bb) throws MarshalException
+    protected <V> AbstractType<?> validateComparator(int i, V value, ValueAccessor<V> accessor, int offset) throws MarshalException
     {
         if (i >= types.size())
             throw new MarshalException("Too many bytes for comparator");
         return types.get(i);
     }
 
+    protected <V> int getComparatorSize(int i, V value, ValueAccessor<V> accessor, int offset)
+    {
+        return 0;
+    }
+
     public ByteBuffer decompose(Object... objects)
     {
         assert objects.length == types.size();
@@ -165,9 +192,8 @@ public class CompositeType extends AbstractCompositeType
             ByteBuffer buffer = ((AbstractType) types.get(i)).decompose(objects[i]);
             serialized[i] = buffer;
         }
-        return build(serialized);
+        return build(ByteBufferAccessor.instance, serialized);
     }
-
     // Overriding the one of AbstractCompositeType because we can do a tad better
     @Override
     public ByteBuffer[] split(ByteBuffer name)
@@ -186,15 +212,17 @@ public class CompositeType extends AbstractCompositeType
         return i == l.length ? l : Arrays.copyOfRange(l, 0, i);
     }
 
-    public static List<ByteBuffer> splitName(ByteBuffer name)
+    public static <V> List<V> splitName(V name, ValueAccessor<V> accessor)
     {
-        List<ByteBuffer> l = new ArrayList<>();
-        ByteBuffer bb = name.duplicate();
-        readStatic(bb);
-        while (bb.remaining() > 0)
+        List<V> l = new ArrayList<>();
+        boolean isStatic = readIsStaticInternal(name, accessor);
+        int offset = startingOffsetInternal(isStatic);
+        while (!accessor.isEmptyFromOffset(name, offset))
         {
-            l.add(ByteBufferUtil.readBytesWithShortLength(bb));
-            bb.get(); // skip end-of-component
+            V value = accessor.sliceWithShortLength(name, offset);
+            offset += accessor.sizeWithShortLength(value);
+            l.add(value);
+            offset++; // skip end-of-component
         }
         return l;
     }
@@ -217,15 +245,9 @@ public class CompositeType extends AbstractCompositeType
         return null;
     }
 
-    public static boolean isStaticName(ByteBuffer bb)
-    {
-        return bb.remaining() >= 2 && (ByteBufferUtil.getShortLength(bb, bb.position()) & 0xFFFF) == STATIC_MARKER;
-    }
-
-    @Override
-    public int componentsCount()
+    public static <V> boolean isStaticName(V value, ValueAccessor<V> accessor)
     {
-        return types.size();
+        return accessor.size(value) >= 2 && (accessor.getShortLength(value, 0) & 0xFFFF) == STATIC_MARKER;
     }
 
     @Override
@@ -283,9 +305,9 @@ public class CompositeType extends AbstractCompositeType
     }
 
     @Override
-    public boolean referencesUserType(ByteBuffer name)
+    public <V> boolean referencesUserType(V name, ValueAccessor<V> accessor)
     {
-        return any(types, t -> t.referencesUserType(name));
+        return any(types, t -> t.referencesUserType(name, accessor));
     }
 
     @Override
@@ -340,31 +362,31 @@ public class CompositeType extends AbstractCompositeType
         return getClass().getName() + TypeParser.stringifyTypeParameters(types);
     }
 
-    public static ByteBuffer build(ByteBuffer... buffers)
+    @SafeVarargs
+    public static <V> V build(ValueAccessor<V> accessor, V... values)
     {
-        return build(false, buffers);
+        return build(accessor, false, values);
     }
 
-    public static ByteBuffer build(boolean isStatic, ByteBuffer... buffers)
+    @SafeVarargs
+    public static <V> V build(ValueAccessor<V> accessor, boolean isStatic, V... values)
     {
         int totalLength = isStatic ? 2 : 0;
-        for (ByteBuffer bb : buffers)
-            totalLength += 2 + bb.remaining() + 1;
+        for (V v : values)
+            totalLength += 2 + accessor.size(v) + 1;
 
         ByteBuffer out = ByteBuffer.allocate(totalLength);
 
         if (isStatic)
             out.putShort((short)STATIC_MARKER);
 
-        for (ByteBuffer bb : buffers)
+        for (V v : values)
         {
-            ByteBufferUtil.writeShortLength(out, bb.remaining());
-            int toCopy = bb.remaining();
-            ByteBufferUtil.copyBytes(bb, bb.position(), out, out.position(), toCopy);
-            out.position(out.position() + toCopy);
+            ByteBufferUtil.writeShortLength(out, accessor.size(v));
+            accessor.write(v, out);
             out.put((byte) 0);
         }
         out.flip();
-        return out;
+        return accessor.valueOf(out);
     }
 }
diff --git a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
index 8777e0e..0dae092 100644
--- a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
@@ -44,10 +44,9 @@ public class CounterColumnType extends NumberType<Long>
         return true;
     }
 
-    @Override
-    public Long compose(ByteBuffer bytes)
+    public <V> Long compose(V value, ValueAccessor<V> accessor)
     {
-        return CounterContext.instance().total(bytes);
+        return CounterContext.instance().total(value, accessor);
     }
 
     @Override
@@ -57,14 +56,14 @@ public class CounterColumnType extends NumberType<Long>
     }
 
     @Override
-    public void validateCellValue(ByteBuffer cellValue) throws MarshalException
+    public <V> void validateCellValue(V cellValue, ValueAccessor<V> accessor) throws MarshalException
     {
-        CounterContext.instance().validateContext(cellValue);
+        CounterContext.instance().validateContext(cellValue, accessor);
     }
 
-    public String getString(ByteBuffer bytes)
+    public <V> String getString(V value, ValueAccessor<V> accessor)
     {
-        return ByteBufferUtil.bytesToHex(bytes);
+        return accessor.toHex(value);
     }
 
     public ByteBuffer fromString(String source)
diff --git a/src/java/org/apache/cassandra/db/marshal/DecimalType.java b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
index dd47ded..5740fdc 100644
--- a/src/java/org/apache/cassandra/db/marshal/DecimalType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
@@ -54,12 +54,9 @@ public class DecimalType extends NumberType<BigDecimal>
         return true;
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
-
-        return compose(o1).compareTo(compose(o2));
+        return compareComposed(left, accessorL, right, accessorR, this);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/DoubleType.java b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
index 6ed98dc..570d420 100644
--- a/src/java/org/apache/cassandra/db/marshal/DoubleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
@@ -45,12 +45,9 @@ public class DoubleType extends NumberType<Double>
         return true;
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
-
-        return compose(o1).compareTo(compose(o2));
+        return compareComposed(left, accessorL, right, accessorR, this);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
index 0458dc8..5df3600 100644
--- a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
@@ -27,6 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.MarshalException;
@@ -82,20 +83,39 @@ public class DynamicCompositeType extends AbstractCompositeType
         this.aliases = aliases;
     }
 
-    protected boolean readIsStatic(ByteBuffer bb)
+    protected <V> boolean readIsStatic(V value, ValueAccessor<V> accessor)
     {
         // We don't have the static nothing for DCT
         return false;
     }
 
-    private AbstractType<?> getComparator(ByteBuffer bb)
+    protected int startingOffset(boolean isStatic)
+    {
+        return 0;
+    }
+
+    protected <V> int getComparatorSize(int i, V value, ValueAccessor<V> accessor, int offset)
+    {
+        int header = accessor.getShort(value, offset);
+        if ((header & 0x8000) == 0)
+        {
+            return 2 + header;
+        }
+        else
+        {
+            return 2;
+        }
+    }
+
+    private <V> AbstractType<?> getComparator(V value, ValueAccessor<V> accessor, int offset)
     {
         try
         {
-            int header = ByteBufferUtil.readShortLength(bb);
+            int header = accessor.getShort(value, offset);
             if ((header & 0x8000) == 0)
             {
-                String name = ByteBufferUtil.string(ByteBufferUtil.readBytes(bb, header));
+
+                String name = accessor.toString(accessor.slice(value, offset + 2, header));
                 return TypeParser.parse(name);
             }
             else
@@ -109,15 +129,15 @@ public class DynamicCompositeType extends AbstractCompositeType
         }
     }
 
-    protected AbstractType<?> getComparator(int i, ByteBuffer bb)
+    protected <V> AbstractType<?> getComparator(int i, V value, ValueAccessor<V> accessor, int offset)
     {
-        return getComparator(bb);
+        return getComparator(value, accessor, offset);
     }
 
-    protected AbstractType<?> getComparator(int i, ByteBuffer bb1, ByteBuffer bb2)
+    protected <VL, VR> AbstractType<?> getComparator(int i, VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR, int offsetL, int offsetR)
     {
-        AbstractType<?> comp1 = getComparator(bb1);
-        AbstractType<?> comp2 = getComparator(bb2);
+        AbstractType<?> comp1 = getComparator(left, accessorL, offsetL);
+        AbstractType<?> comp2 = getComparator(right, accessorR, offsetR);
         AbstractType<?> rawComp = comp1;
 
         /*
@@ -154,14 +174,14 @@ public class DynamicCompositeType extends AbstractCompositeType
         return rawComp;
     }
 
-    protected AbstractType<?> getAndAppendComparator(int i, ByteBuffer bb, StringBuilder sb)
+    protected <V> AbstractType<?> getAndAppendComparator(int i, V value, ValueAccessor<V> accessor, StringBuilder sb, int offset)
     {
         try
         {
-            int header = ByteBufferUtil.readShortLength(bb);
+            int header = accessor.getShort(value, offset);
             if ((header & 0x8000) == 0)
             {
-                String name = ByteBufferUtil.string(ByteBufferUtil.readBytes(bb, header));
+                String name = accessor.toString(accessor.slice(value, offset + 2, header));
                 sb.append(name).append("@");
                 return TypeParser.parse(name);
             }
@@ -182,22 +202,23 @@ public class DynamicCompositeType extends AbstractCompositeType
         return new DynamicParsedComparator(part);
     }
 
-    protected AbstractType<?> validateComparator(int i, ByteBuffer bb) throws MarshalException
+    protected <V> AbstractType<?> validateComparator(int i, V input, ValueAccessor<V> accessor, int offset) throws MarshalException
     {
         AbstractType<?> comparator = null;
-        if (bb.remaining() < 2)
+        if (accessor.sizeFromOffset(input, offset) < 2)
             throw new MarshalException("Not enough bytes to header of the comparator part of component " + i);
-        int header = ByteBufferUtil.readShortLength(bb);
+        int header = accessor.getShort(input, offset);
+        offset += TypeSizes.SHORT_SIZE;
         if ((header & 0x8000) == 0)
         {
-            if (bb.remaining() < header)
+            if (accessor.sizeFromOffset(input, offset) < header)
                 throw new MarshalException("Not enough bytes to read comparator name of component " + i);
 
-            ByteBuffer value = ByteBufferUtil.readBytes(bb, header);
+            V value = accessor.slice(input, offset, header);
             String valueStr = null;
             try
             {
-                valueStr = ByteBufferUtil.string(value);
+                valueStr = accessor.toString(value);
                 comparator = TypeParser.parse(valueStr);
             }
             catch (CharacterCodingException ce)
@@ -257,9 +278,9 @@ public class DynamicCompositeType extends AbstractCompositeType
     }
 
     @Override
-    public boolean referencesUserType(ByteBuffer name)
+    public <V> boolean referencesUserType(V name, ValueAccessor<V> accessor)
     {
-        return any(aliases.values(), t -> t.referencesUserType(name));
+        return any(aliases.values(), t -> t.referencesUserType(name, accessor));
     }
 
     @Override
@@ -370,24 +391,23 @@ public class DynamicCompositeType extends AbstractCompositeType
             this.cmp = cmp;
         }
 
-        public int compareCustom(ByteBuffer v1, ByteBuffer v2)
+        public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
         {
             return cmp;
         }
 
         @Override
-        public Void compose(ByteBuffer bytes)
+        public <V> Void compose(V value, ValueAccessor<V> accessor)
         {
             throw new UnsupportedOperationException();
         }
 
-        @Override
         public ByteBuffer decompose(Void value)
         {
             throw new UnsupportedOperationException();
         }
 
-        public String getString(ByteBuffer bytes)
+        public <V> String getString(V value, ValueAccessor<V> accessor)
         {
             throw new UnsupportedOperationException();
         }
diff --git a/src/java/org/apache/cassandra/db/marshal/EmptyType.java b/src/java/org/apache/cassandra/db/marshal/EmptyType.java
index 808402f..357b6e8 100644
--- a/src/java/org/apache/cassandra/db/marshal/EmptyType.java
+++ b/src/java/org/apache/cassandra/db/marshal/EmptyType.java
@@ -68,12 +68,12 @@ public class EmptyType extends AbstractType<Void>
 
     private EmptyType() {super(ComparisonType.CUSTOM);} // singleton
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
         return 0;
     }
 
-    public String getString(ByteBuffer bytes)
+    public <V> String getString(V value, ValueAccessor<V> accessor)
     {
         return "";
     }
@@ -121,21 +121,20 @@ public class EmptyType extends AbstractType<Void>
     }
 
     @Override
-    public long writtenLength(ByteBuffer value)
+    public <V> long writtenLength(V value, ValueAccessor<V> accessor)
     {
         // default implemenation requires non-empty bytes but this always requires empty bytes, so special case
-        validate(value);
+        validate(value, accessor);
         return 0;
     }
 
-    @Override
-    public ByteBuffer readValue(DataInputPlus in)
+    public ByteBuffer readBuffer(DataInputPlus in)
     {
         return ByteBufferUtil.EMPTY_BYTE_BUFFER;
     }
 
     @Override
-    public ByteBuffer readValue(DataInputPlus in, int maxValueSize)
+    public ByteBuffer readBuffer(DataInputPlus in, int maxValueSize)
     {
         return ByteBufferUtil.EMPTY_BYTE_BUFFER;
     }
diff --git a/src/java/org/apache/cassandra/db/marshal/FloatType.java b/src/java/org/apache/cassandra/db/marshal/FloatType.java
index af63d80..35abee0 100644
--- a/src/java/org/apache/cassandra/db/marshal/FloatType.java
+++ b/src/java/org/apache/cassandra/db/marshal/FloatType.java
@@ -46,12 +46,9 @@ public class FloatType extends NumberType<Float>
         return true;
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
-
-        return compose(o1).compareTo(compose(o2));
+        return compareComposed(left, accessorL, right, accessorR, this);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/FrozenType.java b/src/java/org/apache/cassandra/db/marshal/FrozenType.java
index 6cabb53..64a5b6a 100644
--- a/src/java/org/apache/cassandra/db/marshal/FrozenType.java
+++ b/src/java/org/apache/cassandra/db/marshal/FrozenType.java
@@ -47,7 +47,7 @@ public class FrozenType extends AbstractType<Void>
         return innerType.freeze();
     }
 
-    public String getString(ByteBuffer bytes)
+    public <V> String getString(V value, ValueAccessor<V> accessor)
     {
         throw new UnsupportedOperationException();
     }
diff --git a/src/java/org/apache/cassandra/db/marshal/Int32Type.java b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
index a486289..98f4c83 100644
--- a/src/java/org/apache/cassandra/db/marshal/Int32Type.java
+++ b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
@@ -43,16 +43,16 @@ public class Int32Type extends NumberType<Integer>
         return true;
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
 
-        int diff = o1.get(o1.position()) - o2.get(o2.position());
+        int diff = accessorL.getByte(left, 0) - accessorR.getByte(right, 0);
         if (diff != 0)
             return diff;
 
-        return ByteBufferUtil.compareUnsigned(o1, o2);
+        return ValueAccessor.compare(left, accessorL, right, accessorR);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/IntegerType.java b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
index 044fe6e..4c913d5 100644
--- a/src/java/org/apache/cassandra/db/marshal/IntegerType.java
+++ b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
@@ -35,16 +35,16 @@ public final class IntegerType extends NumberType<BigInteger>
 {
     public static final IntegerType instance = new IntegerType();
 
-    private static int findMostSignificantByte(ByteBuffer bytes)
+    private static <V> int findMostSignificantByte(V value, ValueAccessor<V> accessor)
     {
-        int len = bytes.remaining() - 1;
+        int len = accessor.size(value) - 1;
         int i = 0;
         for (; i < len; i++)
         {
-            byte b0 = bytes.get(bytes.position() + i);
+            byte b0 = accessor.getByte(value, i);
             if (b0 != 0 && b0 != -1)
                 break;
-            byte b1 = bytes.get(bytes.position() + i + 1);
+            byte b1 = accessor.getByte(value, i + 1);
             if (b0 == 0 && b1 != 0)
             {
                 if (b1 > 0)
@@ -68,30 +68,30 @@ public final class IntegerType extends NumberType<BigInteger>
         return true;
     }
 
-    public int compareCustom(ByteBuffer lhs, ByteBuffer rhs)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        return IntegerType.compareIntegers(lhs, rhs);
+        return IntegerType.compareIntegers(left, accessorL, right, accessorR);
     }
 
-    public static int compareIntegers(ByteBuffer lhs, ByteBuffer rhs)
+    public static <VL, VR> int compareIntegers(VL lhs, ValueAccessor<VL> accessorL, VR rhs, ValueAccessor<VR> accessorR)
     {
-        int lhsLen = lhs.remaining();
-        int rhsLen = rhs.remaining();
+        int lhsLen = accessorL.size(lhs);
+        int rhsLen = accessorR.size(rhs);
 
         if (lhsLen == 0)
             return rhsLen == 0 ? 0 : -1;
         if (rhsLen == 0)
             return 1;
 
-        int lhsMsbIdx = findMostSignificantByte(lhs);
-        int rhsMsbIdx = findMostSignificantByte(rhs);
+        int lhsMsbIdx = findMostSignificantByte(lhs, accessorL);
+        int rhsMsbIdx = findMostSignificantByte(rhs, accessorR);
 
         //diffs contain number of "meaningful" bytes (i.e. ignore padding)
         int lhsLenDiff = lhsLen - lhsMsbIdx;
         int rhsLenDiff = rhsLen - rhsMsbIdx;
 
-        byte lhsMsb = lhs.get(lhs.position() + lhsMsbIdx);
-        byte rhsMsb = rhs.get(rhs.position() + rhsMsbIdx);
+        byte lhsMsb = accessorL.getByte(lhs, lhsMsbIdx);
+        byte rhsMsb = accessorR.getByte(rhs, rhsMsbIdx);
 
         /*         +    -
          *      -----------
@@ -121,8 +121,8 @@ public final class IntegerType extends NumberType<BigInteger>
         // remaining bytes are compared unsigned
         while (lhsMsbIdx < lhsLen)
         {
-            lhsMsb = lhs.get(lhs.position() + lhsMsbIdx++);
-            rhsMsb = rhs.get(rhs.position() + rhsMsbIdx++);
+            lhsMsb = accessorL.getByte(lhs, lhsMsbIdx++);
+            rhsMsb = accessorR.getByte(rhs, rhsMsbIdx++);
 
             if (lhsMsb != rhsMsb)
                 return (lhsMsb & 0xFF) - (rhsMsb & 0xFF);
diff --git a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
index de32a56..6dd4161 100644
--- a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
@@ -26,7 +26,6 @@ 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.UUIDGen;
 
 public class LexicalUUIDType extends AbstractType<UUID>
 {
@@ -42,12 +41,11 @@ public class LexicalUUIDType extends AbstractType<UUID>
         return true;
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
-
-        return UUIDGen.getUUID(o1).compareTo(UUIDGen.getUUID(o2));
+        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
+        return accessorL.toUUID(left).compareTo(accessorR.toUUID(right));
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/ListType.java b/src/java/org/apache/cassandra/db/marshal/ListType.java
index c6e0262..cc63937 100644
--- a/src/java/org/apache/cassandra/db/marshal/ListType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ListType.java
@@ -24,6 +24,7 @@ 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;
@@ -69,9 +70,9 @@ public class ListType<T> extends CollectionType<List<T>>
     }
 
     @Override
-    public boolean referencesUserType(ByteBuffer name)
+    public <V> boolean referencesUserType(V name, ValueAccessor<V> accessor)
     {
-        return elements.referencesUserType(name);
+        return elements.referencesUserType(name, accessor);
     }
 
     @Override
@@ -164,34 +165,34 @@ public class ListType<T> extends CollectionType<List<T>>
         return this.elements.isValueCompatibleWithInternal(((ListType) previous).elements);
     }
 
-    @Override
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        return compareListOrSet(elements, o1, o2);
+        return compareListOrSet(elements, left, accessorL, right, accessorR);
     }
 
-    static int compareListOrSet(AbstractType<?> elementsComparator, ByteBuffer o1, ByteBuffer o2)
+    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 (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
-
-        ByteBuffer bb1 = o1.duplicate();
-        ByteBuffer bb2 = o2.duplicate();
+        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
 
-        int size1 = CollectionSerializer.readCollectionSize(bb1, ProtocolVersion.V3);
-        int size2 = CollectionSerializer.readCollectionSize(bb2, ProtocolVersion.V3);
+        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(size1, size2); i++)
+        for (int i = 0; i < Math.min(sizeL, sizeR); i++)
         {
-            ByteBuffer v1 = CollectionSerializer.readValue(bb1, ProtocolVersion.V3);
-            ByteBuffer v2 = CollectionSerializer.readValue(bb2, ProtocolVersion.V3);
-            int cmp = elementsComparator.compare(v1, v2);
+            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 size1 == size2 ? 0 : (size1 < size2 ? -1 : 1);
+        return sizeL == sizeR ? 0 : (sizeL < sizeR ? -1 : 1);
     }
 
     @Override
@@ -209,12 +210,12 @@ public class ListType<T> extends CollectionType<List<T>>
         return sb.toString();
     }
 
-    public List<ByteBuffer> serializedValues(Iterator<Cell> cells)
+    public List<ByteBuffer> serializedValues(Iterator<Cell<?>> cells)
     {
         assert isMultiCell;
         List<ByteBuffer> bbs = new ArrayList<ByteBuffer>();
         while (cells.hasNext())
-            bbs.add(cells.next().value());
+            bbs.add(cells.next().buffer());
         return bbs;
     }
 
@@ -245,11 +246,14 @@ public class ListType<T> extends CollectionType<List<T>>
         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(", ");
-            sb.append(elementsType.toJSONString(CollectionSerializer.readValue(value, protocolVersion), protocolVersion));
+            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();
     }
diff --git a/src/java/org/apache/cassandra/db/marshal/LongType.java b/src/java/org/apache/cassandra/db/marshal/LongType.java
index 0795458..ad539f7 100644
--- a/src/java/org/apache/cassandra/db/marshal/LongType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LongType.java
@@ -40,21 +40,21 @@ public class LongType extends NumberType<Long>
         return true;
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        return compareLongs(o1, o2);
+        return compareLongs(left, accessorL, right, accessorR);
     }
 
-    public static int compareLongs(ByteBuffer o1, ByteBuffer o2)
+    public static <VL, VR> int compareLongs(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+        if (accessorL.isEmpty(left)|| accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
 
-        int diff = o1.get(o1.position()) - o2.get(o2.position());
+        int diff = accessorL.getByte(left, 0) - accessorR.getByte(right, 0);
         if (diff != 0)
             return diff;
 
-        return ByteBufferUtil.compareUnsigned(o1, o2);
+        return ValueAccessor.compare(left, accessorL, right, accessorR);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/MapType.java b/src/java/org/apache/cassandra/db/marshal/MapType.java
index 6abe388..9473e29 100644
--- a/src/java/org/apache/cassandra/db/marshal/MapType.java
+++ b/src/java/org/apache/cassandra/db/marshal/MapType.java
@@ -68,14 +68,16 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
         super(ComparisonType.CUSTOM, Kind.MAP);
         this.keys = keys;
         this.values = values;
-        this.serializer = MapSerializer.getInstance(keys.getSerializer(), values.getSerializer(), keys);
+        this.serializer = MapSerializer.getInstance(keys.getSerializer(),
+                                                    values.getSerializer(),
+                                                    keys.comparatorSet);
         this.isMultiCell = isMultiCell;
     }
 
     @Override
-    public boolean referencesUserType(ByteBuffer name)
+    public <T> boolean referencesUserType(T name, ValueAccessor<T> accessor)
     {
-        return keys.referencesUserType(name) || values.referencesUserType(name);
+        return keys.referencesUserType(name, accessor) || values.referencesUserType(name, accessor);
     }
 
     @Override
@@ -176,40 +178,44 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
         return keys.isCompatibleWith(tprev.keys) && values.isValueCompatibleWith(tprev.values);
     }
 
-    @Override
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <RL, TR> int compareCustom(RL left, ValueAccessor<RL> accessorL, TR right, ValueAccessor<TR> accessorR)
     {
-        return compareMaps(keys, values, o1, o2);
+        return compareMaps(keys, values, left, accessorL, right, accessorR);
     }
 
-    public static int compareMaps(AbstractType<?> keysComparator, AbstractType<?> valuesComparator, ByteBuffer o1, ByteBuffer o2)
+    public static <TL, TR> int compareMaps(AbstractType<?> keysComparator, AbstractType<?> valuesComparator, TL left, ValueAccessor<TL> accessorL, TR right, ValueAccessor<TR> accessorR)
     {
-         if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
 
-        ByteBuffer bb1 = o1.duplicate();
-        ByteBuffer bb2 = o2.duplicate();
 
         ProtocolVersion protocolVersion = ProtocolVersion.V3;
-        int size1 = CollectionSerializer.readCollectionSize(bb1, protocolVersion);
-        int size2 = CollectionSerializer.readCollectionSize(bb2, protocolVersion);
+        int sizeL = CollectionSerializer.readCollectionSize(left, accessorL, protocolVersion);
+        int sizeR = CollectionSerializer.readCollectionSize(right, accessorR, protocolVersion);
+
+        int offsetL = CollectionSerializer.sizeOfCollectionSize(sizeL, protocolVersion);
+        int offsetR = CollectionSerializer.sizeOfCollectionSize(sizeR, protocolVersion);
 
-        for (int i = 0; i < Math.min(size1, size2); i++)
+        for (int i = 0; i < Math.min(sizeL, sizeR); i++)
         {
-            ByteBuffer k1 = CollectionSerializer.readValue(bb1, protocolVersion);
-            ByteBuffer k2 = CollectionSerializer.readValue(bb2, protocolVersion);
-            int cmp = keysComparator.compare(k1, k2);
+            TL k1 = CollectionSerializer.readValue(left, accessorL, offsetL, protocolVersion);
+            offsetL += CollectionSerializer.sizeOfValue(k1, accessorL, protocolVersion);
+            TR k2 = CollectionSerializer.readValue(right, accessorR, offsetR, protocolVersion);
+            offsetR += CollectionSerializer.sizeOfValue(k2, accessorR, protocolVersion);
+            int cmp = keysComparator.compare(k1, accessorL, k2, accessorR);
             if (cmp != 0)
                 return cmp;
 
-            ByteBuffer v1 = CollectionSerializer.readValue(bb1, protocolVersion);
-            ByteBuffer v2 = CollectionSerializer.readValue(bb2, protocolVersion);
-            cmp = valuesComparator.compare(v1, v2);
+            TL v1 = CollectionSerializer.readValue(left, accessorL, offsetL, protocolVersion);
+            offsetL += CollectionSerializer.sizeOfValue(v1, accessorL, protocolVersion);
+            TR v2 = CollectionSerializer.readValue(right, accessorR, offsetR, protocolVersion);
+            offsetR += CollectionSerializer.sizeOfValue(v2, accessorR, protocolVersion);
+            cmp = valuesComparator.compare(v1, accessorL, v2, accessorR);
             if (cmp != 0)
                 return cmp;
         }
 
-        return size1 == size2 ? 0 : (size1 < size2 ? -1 : 1);
+        return sizeL == sizeR ? 0 : (sizeL < sizeR ? -1 : 1);
     }
 
     @Override
@@ -237,15 +243,15 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
         return sb.toString();
     }
 
-    public List<ByteBuffer> serializedValues(Iterator<Cell> cells)
+    public List<ByteBuffer> serializedValues(Iterator<Cell<?>> cells)
     {
         assert isMultiCell;
         List<ByteBuffer> bbs = new ArrayList<ByteBuffer>();
         while (cells.hasNext())
         {
-            Cell c = cells.next();
+            Cell<?> c = cells.next();
             bbs.add(c.path().get(0));
-            bbs.add(c.value());
+            bbs.add(c.buffer());
         }
         return bbs;
     }
@@ -281,20 +287,25 @@ 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 offset = CollectionSerializer.sizeOfCollectionSize(size, protocolVersion);
         for (int i = 0; i < size; i++)
         {
             if (i > 0)
                 sb.append(", ");
 
             // map keys must be JSON strings, so convert non-string keys to strings
-            String key = keys.toJSONString(CollectionSerializer.readValue(value, protocolVersion), protocolVersion);
+            ByteBuffer kv = CollectionSerializer.readValue(value, ByteBufferAccessor.instance, offset, protocolVersion);
+            offset += CollectionSerializer.sizeOfValue(kv, ByteBufferAccessor.instance, protocolVersion);
+            String key = keys.toJSONString(kv, protocolVersion);
             if (key.startsWith("\""))
                 sb.append(key);
             else
                 sb.append('"').append(Json.quoteAsJsonString(key)).append('"');
 
             sb.append(": ");
-            sb.append(values.toJSONString(CollectionSerializer.readValue(value, protocolVersion), protocolVersion));
+            ByteBuffer vv = CollectionSerializer.readValue(value, ByteBufferAccessor.instance, offset, protocolVersion);
+            offset += CollectionSerializer.sizeOfValue(vv, ByteBufferAccessor.instance, protocolVersion);
+            sb.append(values.toJSONString(vv, protocolVersion));
         }
         return sb.append("}").toString();
     }
diff --git a/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java b/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java
index 397c3be..89241b4 100644
--- a/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java
+++ b/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java
@@ -27,7 +27,6 @@ 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.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 /** for sorting columns representing row keys in the row ordering as determined by a partitioner.
@@ -55,20 +54,20 @@ public class PartitionerDefinedOrder extends AbstractType<ByteBuffer>
     }
 
     @Override
-    public ByteBuffer compose(ByteBuffer bytes)
+    public <V> ByteBuffer compose(V value, ValueAccessor<V> accessor)
     {
         throw new UnsupportedOperationException("You can't do this with a local partitioner.");
     }
 
     @Override
-    public ByteBuffer decompose(ByteBuffer bytes)
+    public ByteBuffer decompose(ByteBuffer value)
     {
         throw new UnsupportedOperationException("You can't do this with a local partitioner.");
     }
 
-    public String getString(ByteBuffer bytes)
+    public <V> String getString(V value, ValueAccessor<V> accessor)
     {
-        return ByteBufferUtil.bytesToHex(bytes);
+        return accessor.toHex(value);
     }
 
     public ByteBuffer fromString(String source)
@@ -88,10 +87,10 @@ public class PartitionerDefinedOrder extends AbstractType<ByteBuffer>
         throw new UnsupportedOperationException();
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
         // o1 and o2 can be empty so we need to use PartitionPosition, not DecoratedKey
-        return PartitionPosition.ForKey.get(o1, partitioner).compareTo(PartitionPosition.ForKey.get(o2, partitioner));
+        return PartitionPosition.ForKey.get(accessorL.toBuffer(left), partitioner).compareTo(PartitionPosition.ForKey.get(accessorR.toBuffer(right), partitioner));
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/marshal/ReversedType.java b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
index 63a900a..8a4b58d 100644
--- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
@@ -63,9 +63,9 @@ public class ReversedType<T> extends AbstractType<T>
         return baseType.isEmptyValueMeaningless();
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        return baseType.compare(o2, o1);
+        return baseType.compare(right, accessorR, left, accessorL);
     }
 
     @Override
@@ -74,9 +74,9 @@ public class ReversedType<T> extends AbstractType<T>
         return baseType.compare(v1, v2);
     }
 
-    public String getString(ByteBuffer bytes)
+    public <V> String getString(V value, ValueAccessor<V> accessor)
     {
-        return baseType.getString(bytes);
+        return baseType.getString(value, accessor);
     }
 
     public ByteBuffer fromString(String source)
@@ -123,9 +123,9 @@ public class ReversedType<T> extends AbstractType<T>
     }
 
     @Override
-    public boolean referencesUserType(ByteBuffer name)
+    public <V> boolean referencesUserType(V name, ValueAccessor<V> accessor)
     {
-        return baseType.referencesUserType(name);
+        return baseType.referencesUserType(name, accessor);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/marshal/SetType.java b/src/java/org/apache/cassandra/db/marshal/SetType.java
index 6d17b67..e5bdada 100644
--- a/src/java/org/apache/cassandra/db/marshal/SetType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SetType.java
@@ -63,14 +63,14 @@ public class SetType<T> extends CollectionType<Set<T>>
     {
         super(ComparisonType.CUSTOM, Kind.SET);
         this.elements = elements;
-        this.serializer = SetSerializer.getInstance(elements.getSerializer(), elements);
+        this.serializer = SetSerializer.getInstance(elements.getSerializer(), elements.comparatorSet);
         this.isMultiCell = isMultiCell;
     }
 
     @Override
-    public boolean referencesUserType(ByteBuffer name)
+    public <V> boolean referencesUserType(V name, ValueAccessor<V> accessor)
     {
-        return elements.referencesUserType(name);
+        return elements.referencesUserType(name, accessor);
     }
 
     @Override
@@ -152,10 +152,9 @@ public class SetType<T> extends CollectionType<Set<T>>
         return isCompatibleWithFrozen(previous);
     }
 
-    @Override
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        return ListType.compareListOrSet(elements, o1, o2);
+        return ListType.compareListOrSet(elements, left, accessorL, right, accessorR);
     }
 
     public SetSerializer<T> getSerializer()
@@ -178,7 +177,7 @@ public class SetType<T> extends CollectionType<Set<T>>
         return sb.toString();
     }
 
-    public List<ByteBuffer> serializedValues(Iterator<Cell> cells)
+    public List<ByteBuffer> serializedValues(Iterator<Cell<?>> cells)
     {
         List<ByteBuffer> bbs = new ArrayList<ByteBuffer>();
         while (cells.hasNext())
diff --git a/src/java/org/apache/cassandra/db/marshal/ShortType.java b/src/java/org/apache/cassandra/db/marshal/ShortType.java
index 84d538b..03dcf5d 100644
--- a/src/java/org/apache/cassandra/db/marshal/ShortType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ShortType.java
@@ -38,13 +38,12 @@ public class ShortType extends NumberType<Short>
         super(ComparisonType.CUSTOM);
     } // singleton
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        int diff = o1.get(o1.position()) - o2.get(o2.position());
+        int diff = accessorL.getByte(left, 0) - accessorR.getByte(right, 0);
         if (diff != 0)
             return diff;
-
-        return ByteBufferUtil.compareUnsigned(o1, o2);
+        return ValueAccessor.compare(left, accessorL, right, accessorR);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
index 39d1513..6cf1375 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
@@ -43,23 +43,20 @@ public class TimeUUIDType extends TemporalType<UUID>
         return true;
     }
 
-    public int compareCustom(ByteBuffer b1, ByteBuffer b2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
         // Compare for length
-        int s1 = b1.position(), s2 = b2.position();
-        int l1 = b1.limit(), l2 = b2.limit();
-
-        // should we assert exactly 16 bytes (or 0)? seems prudent
-        boolean p1 = l1 - s1 == 16, p2 = l2 - s2 == 16;
+        boolean p1 = accessorL.size(left) == 16, p2 = accessorR.size(right) == 16;
         if (!(p1 & p2))
         {
-            assert p1 | (l1 == s1);
-            assert p2 | (l2 == s2);
+            // should we assert exactly 16 bytes (or 0)? seems prudent
+            assert p1 || accessorL.isEmpty(left);
+            assert p2 || accessorR.isEmpty(right);
             return p1 ? 1 : p2 ? -1 : 0;
         }
 
-        long msb1 = b1.getLong(s1);
-        long msb2 = b2.getLong(s2);
+        long msb1 = accessorL.getLong(left, 0);
+        long msb2 = accessorR.getLong(right, 0);
         msb1 = reorderTimestampBytes(msb1);
         msb2 = reorderTimestampBytes(msb2);
 
@@ -72,8 +69,8 @@ public class TimeUUIDType extends TemporalType<UUID>
 
         // this has to be a signed per-byte comparison for compatibility
         // so we transform the bytes so that a simple long comparison is equivalent
-        long lsb1 = signedBytesToNativeLong(b1.getLong(s1 + 8));
-        long lsb2 = signedBytesToNativeLong(b2.getLong(s2 + 8));
+        long lsb1 = signedBytesToNativeLong(accessorL.getLong(left, 8));
+        long lsb2 = signedBytesToNativeLong(accessorR.getLong(right, 8));
         return Long.compare(lsb1, lsb2);
     }
 
diff --git a/src/java/org/apache/cassandra/db/marshal/TimestampType.java b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
index cfe05e9..0dac6b0 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimestampType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
@@ -56,9 +56,9 @@ public class TimestampType extends TemporalType<Date>
         return true;
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        return LongType.compareLongs(o1, o2);
+        return LongType.compareLongs(left, accessorL, right, accessorR);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java
index dfdb8c2..8a2f7ee 100644
--- a/src/java/org/apache/cassandra/db/marshal/TupleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java
@@ -28,6 +28,7 @@ import com.google.common.base.Objects;
 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;
@@ -91,9 +92,9 @@ public class TupleType extends AbstractType<ByteBuffer>
     }
 
     @Override
-    public boolean referencesUserType(ByteBuffer name)
+    public <V> boolean referencesUserType(V name, ValueAccessor<V> accessor)
     {
-        return any(types, t -> t.referencesUserType(name));
+        return any(types, t -> t.referencesUserType(name, accessor));
     }
 
     @Override
@@ -142,49 +143,55 @@ public class TupleType extends AbstractType<ByteBuffer>
         return true;
     }
 
-    public int compareCustom(ByteBuffer o1, ByteBuffer o2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+        if (accessorL.isEmpty(left) || accessorR.isEmpty(right))
+            return Boolean.compare(accessorR.isEmpty(right), accessorL.isEmpty(left));
 
-        ByteBuffer bb1 = o1.duplicate();
-        ByteBuffer bb2 = o2.duplicate();
+        int offsetL = 0;
+        int offsetR = 0;
 
-        for (int i = 0; bb1.remaining() > 0 && bb2.remaining() > 0; i++)
+        for (int i = 0; !accessorL.isEmptyFromOffset(left, offsetL) && !accessorR.isEmptyFromOffset(right, offsetR); i++)
         {
             AbstractType<?> comparator = types.get(i);
 
-            int size1 = bb1.getInt();
-            int size2 = bb2.getInt();
+            int sizeL = accessorL.getInt(left, offsetL);
+            offsetL += TypeSizes.INT_SIZE;
+            int sizeR = accessorR.getInt(right, offsetR);
+            offsetR += TypeSizes.INT_SIZE;
 
             // Handle nulls
-            if (size1 < 0)
+            if (sizeL < 0)
             {
-                if (size2 < 0)
+                if (sizeR < 0)
                     continue;
                 return -1;
             }
-            if (size2 < 0)
+            if (sizeR < 0)
                 return 1;
 
-            ByteBuffer value1 = ByteBufferUtil.readBytes(bb1, size1);
-            ByteBuffer value2 = ByteBufferUtil.readBytes(bb2, size2);
-            int cmp = comparator.compare(value1, value2);
+            VL valueL = accessorL.slice(left, offsetL, sizeL);
+            offsetL += sizeL;
+            VR valueR = accessorR.slice(right, offsetR, sizeR);
+            offsetR += sizeR;
+            int cmp = comparator.compare(valueL, accessorL, valueR, accessorR);
             if (cmp != 0)
                 return cmp;
         }
 
         // handle trailing nulls
-        while (bb1.remaining() > 0)
+        while (!accessorL.isEmptyFromOffset(left, offsetL))
         {
-            int size = bb1.getInt();
+            int size = accessorL.getInt(left, offsetL);
+            offsetL += TypeSizes.INT_SIZE;
             if (size > 0) // non-null
                 return 1;
         }
 
-        while (bb2.remaining() > 0)
+        while (!accessorR.isEmptyFromOffset(right, offsetR))
         {
-            int size = bb2.getInt();
+            int size = accessorR.getInt(right, offsetR);
+            offsetR += TypeSizes.INT_SIZE;
             if (size > 0) // non-null
                 return -1;
         }
@@ -217,63 +224,71 @@ public class TupleType extends AbstractType<ByteBuffer>
         if (input.hasRemaining())
         {
             throw new InvalidRequestException(String.format(
-                    "Expected %s %s for %s column, but got more",
-                    size(), size() == 1 ? "value" : "values", this.asCQL3Type()));
+            "Expected %s %s for %s column, but got more",
+            size(), size() == 1 ? "value" : "values", this.asCQL3Type()));
         }
 
         return components;
     }
 
-    public static ByteBuffer buildValue(ByteBuffer[] components)
+    public static <V> V buildValue(ValueAccessor<V> accessor, V[] components)
     {
         int totalLength = 0;
-        for (ByteBuffer component : components)
-            totalLength += 4 + (component == null ? 0 : component.remaining());
+        for (V component : components)
+            totalLength += 4 + (component == null ? 0 : accessor.size(component));
 
-        ByteBuffer result = ByteBuffer.allocate(totalLength);
-        for (ByteBuffer component : components)
+        int offset = 0;
+        V result = accessor.allocate(totalLength);
+        for (V component : components)
         {
             if (component == null)
             {
-                result.putInt(-1);
+                offset += accessor.putInt(result, offset, -1);
+
             }
             else
             {
-                result.putInt(component.remaining());
-                result.put(component.duplicate());
+                offset += accessor.putInt(result, offset, accessor.size(component));
+                offset += accessor.copyTo(component, 0, result, accessor, offset, accessor.size(component));
             }
         }
-        result.rewind();
         return result;
     }
 
+    public static ByteBuffer buildValue(ByteBuffer[] components)
+    {
+        return buildValue(ByteBufferAccessor.instance, components);
+    }
+
     @Override
-    public String getString(ByteBuffer value)
+    public <V> String getString(V input, ValueAccessor<V> accessor)
     {
-        if (value == null)
+        if (input == null)
             return "null";
 
         StringBuilder sb = new StringBuilder();
-        ByteBuffer input = value.duplicate();
+        int offset = 0;
         for (int i = 0; i < size(); i++)
         {
-            if (!input.hasRemaining())
+            if (accessor.isEmptyFromOffset(input, offset))
                 return sb.toString();
 
             if (i > 0)
                 sb.append(":");
 
             AbstractType<?> type = type(i);
-            int size = input.getInt();
+            int size = accessor.getInt(input, offset);
+            offset += TypeSizes.INT_SIZE;
             if (size < 0)
             {
                 sb.append("@");
                 continue;
             }
 
-            ByteBuffer field = ByteBufferUtil.readBytes(input, size);
+            V field = accessor.slice(input, offset, size);
+            offset += size;
             // We use ':' as delimiter, and @ to represent null, so escape them in the generated string
-            String fld = COLON_PAT.matcher(type.getString(field)).replaceAll(ESCAPED_COLON);
+            String fld = COLON_PAT.matcher(type.getString(field, accessor)).replaceAll(ESCAPED_COLON);
             fld = AT_PAT.matcher(fld).replaceAll(ESCAPED_AT);
             sb.append(fld);
         }
@@ -344,13 +359,15 @@ public class TupleType extends AbstractType<ByteBuffer>
     public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion)
     {
         ByteBuffer duplicated = buffer.duplicate();
+        int offset = 0;
         StringBuilder sb = new StringBuilder("[");
         for (int i = 0; i < types.size(); i++)
         {
             if (i > 0)
                 sb.append(", ");
 
-            ByteBuffer value = CollectionSerializer.readValue(duplicated, protocolVersion);
+            ByteBuffer value = CollectionSerializer.readValue(duplicated, ByteBufferAccessor.instance, offset, protocolVersion);
+            offset += CollectionSerializer.sizeOfValue(value, ByteBufferAccessor.instance, protocolVersion);
             if (value == null)
                 sb.append("null");
             else
diff --git a/src/java/org/apache/cassandra/db/marshal/UTF8Type.java b/src/java/org/apache/cassandra/db/marshal/UTF8Type.java
index 46e0d90..db62b57 100644
--- a/src/java/org/apache/cassandra/db/marshal/UTF8Type.java
+++ b/src/java/org/apache/cassandra/db/marshal/UTF8Type.java
@@ -43,7 +43,6 @@ public class UTF8Type extends AbstractType<String>
         return decompose(source);
     }
 
-
     @Override
     public Term fromJSONObject(Object parsed) throws MarshalException
     {
diff --git a/src/java/org/apache/cassandra/db/marshal/UUIDType.java b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
index 27e3360..55ce59d 100644
--- a/src/java/org/apache/cassandra/db/marshal/UUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
@@ -56,24 +56,22 @@ public class UUIDType extends AbstractType<UUID>
         return true;
     }
 
-    public int compareCustom(ByteBuffer b1, ByteBuffer b2)
+    public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR)
     {
-        // Compare for length
-        int s1 = b1.position(), s2 = b2.position();
-        int l1 = b1.limit(), l2 = b2.limit();
 
-        // should we assert exactly 16 bytes (or 0)? seems prudent
-        boolean p1 = l1 - s1 == 16, p2 = l2 - s2 == 16;
+        // Compare for length
+        boolean p1 = accessorL.size(left) == 16, p2 = accessorR.size(right) == 16;
         if (!(p1 & p2))
         {
-            assert p1 | (l1 == s1);
-            assert p2 | (l2 == s2);
+            // should we assert exactly 16 bytes (or 0)? seems prudent
+            assert p1 || accessorL.isEmpty(left);
+            assert p2 || accessorR.isEmpty(right);
             return p1 ? 1 : p2 ? -1 : 0;
         }
 
         // Compare versions
-        long msb1 = b1.getLong(s1);
-        long msb2 = b2.getLong(s2);
+        long msb1 = accessorL.getLong(left, 0);
+        long msb2 = accessorR.getLong(right, 0);
 
         int version1 = (int) ((msb1 >>> 12) & 0xf);
         int version2 = (int) ((msb2 >>> 12) & 0xf);
@@ -98,7 +96,7 @@ public class UUIDType extends AbstractType<UUID>
                 return c;
         }
 
-        return UnsignedLongs.compare(b1.getLong(s1 + 8), b2.getLong(s2 + 8));
+        return UnsignedLongs.compare(accessorL.getLong(left, 8), accessorR.getLong(right, 8));
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index dfc726d..f3cd7d7 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -167,7 +167,7 @@ public class UserType extends TupleType implements SchemaElement
         return ShortType.instance;
     }
 
-    public ByteBuffer serializeForNativeProtocol(Iterator<Cell> cells, ProtocolVersion protocolVersion)
+    public ByteBuffer serializeForNativeProtocol(Iterator<Cell<?>> cells, ProtocolVersion protocolVersion)
     {
         assert isMultiCell;
 
@@ -175,14 +175,14 @@ public class UserType extends TupleType implements SchemaElement
         short fieldPosition = 0;
         while (cells.hasNext())
         {
-            Cell cell = cells.next();
+            Cell<?> cell = cells.next();
 
             // handle null fields that aren't at the end
             short fieldPositionOfCell = ByteBufferUtil.toShort(cell.path().get(0));
             while (fieldPosition < fieldPositionOfCell)
                 components[fieldPosition++] = null;
 
-            components[fieldPosition++] = cell.value();
+            components[fieldPosition++] = cell.buffer();
         }
 
         // append trailing nulls for missing cells
@@ -192,18 +192,18 @@ public class UserType extends TupleType implements SchemaElement
         return TupleType.buildValue(components);
     }
 
-    public void validateCell(Cell cell) throws MarshalException
+    public <V> void validateCell(Cell<V> cell) throws MarshalException
     {
         if (isMultiCell)
         {
             ByteBuffer path = cell.path().get(0);
             nameComparator().validate(path);
             Short fieldPosition = nameComparator().getSerializer().deserialize(path);
-            fieldType(fieldPosition).validate(cell.value());
+            fieldType(fieldPosition).validate(cell.value(), cell.accessor());
         }
         else
         {
-            validate(cell.value());
+            validate(cell.value(), cell.accessor());
         }
     }
 
@@ -389,9 +389,9 @@ public class UserType extends TupleType implements SchemaElement
     }
 
     @Override
-    public boolean referencesUserType(ByteBuffer name)
+    public <V> boolean referencesUserType(V name, ValueAccessor<V> accessor)
     {
-        return this.name.equals(name) || any(fieldTypes(), t -> t.referencesUserType(name));
+        return this.name.equals(name) || any(fieldTypes(), t -> t.referencesUserType(name, accessor));
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java b/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java
new file mode 100644
index 0000000..8ab1569
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java
@@ -0,0 +1,418 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.marshal;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.UUID;
+
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringBoundOrBoundary;
+import org.apache.cassandra.db.ClusteringBoundary;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.Digest;
+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;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+
+import static org.apache.cassandra.db.ClusteringPrefix.Kind.*;
+
+/**
+ * ValueAccessor allows serializers and other code dealing with raw bytes to operate on different backing types
+ * (ie: byte arrays, byte buffers, etc) without requiring that the supported backing types share a common type
+ * ancestor and without incuring the allocation cost of a wrapper object.
+ *
+ * A note on byte buffers for implementors: the "value" of a byte buffer is always interpreted as beginning at
+ * it's {@link ByteBuffer#position()} and having a length of {@link ByteBuffer#remaining()}. ValueAccessor
+ * implementations need to maintain this internally. ValueAccessors should also never modify the state of the
+ * byte buffers view (ie: offset, limit). This would also apply to value accessors for simlilar types
+ * (ie: netty's ByteBuf}.
+ *
+ * @param <V> the backing type
+ */
+public interface ValueAccessor<V>
+{
+
+    /**
+     * Creates db objects using the given accessors value type. ObjectFactory instances are meant to be returned
+     * by the factory() method of a value accessor.
+     * @param <V> the backing type
+     */
+    public interface ObjectFactory<V>
+    {
+        Cell<V> cell(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, V value, CellPath path);
+        Clustering<V> clustering(V... values);
+        Clustering<V> clustering();
+        ClusteringBound<V> bound(ClusteringPrefix.Kind kind, V... values);
+        ClusteringBound<V> bound(ClusteringPrefix.Kind kind);
+        ClusteringBoundary<V> boundary(ClusteringPrefix.Kind kind, V... values);
+        default ClusteringBoundOrBoundary<V> boundOrBoundary(ClusteringPrefix.Kind kind, V... values)
+        {
+            return kind.isBoundary() ? boundary(kind, values) : bound(kind, values);
+        }
+
+        default ClusteringBound<V> inclusiveOpen(boolean reversed, V[] boundValues)
+        {
+            return bound(reversed ? INCL_END_BOUND : INCL_START_BOUND, boundValues);
+        }
+
+        default ClusteringBound<V> exclusiveOpen(boolean reversed, V[] boundValues)
+        {
+            return bound(reversed ? EXCL_END_BOUND : EXCL_START_BOUND, boundValues);
+        }
+
+        default ClusteringBound<V> inclusiveClose(boolean reversed, V[] boundValues)
+        {
+            return bound(reversed ? INCL_START_BOUND : INCL_END_BOUND, boundValues);
+        }
+
+        default ClusteringBound<V> exclusiveClose(boolean reversed, V[] boundValues)
+        {
+            return bound(reversed ? EXCL_START_BOUND : EXCL_END_BOUND, boundValues);
+        }
+
+        default ClusteringBoundary<V> inclusiveCloseExclusiveOpen(boolean reversed, V[] boundValues)
+        {
+            return boundary(reversed ? EXCL_END_INCL_START_BOUNDARY : INCL_END_EXCL_START_BOUNDARY, boundValues);
+        }
+
+        default ClusteringBoundary<V> exclusiveCloseInclusiveOpen(boolean reversed, V[] boundValues)
+        {
+            return boundary(reversed ? INCL_END_EXCL_START_BOUNDARY : EXCL_END_INCL_START_BOUNDARY, boundValues);
+        }
+
+    }
+    /**
+     * @return the size of the given value
+     */
+    int size(V value);
+
+    /** serializes size including a vint length prefix */
+    default int sizeWithVIntLength(V value)
+    {
+        int size = size(value);
+        return TypeSizes.sizeofUnsignedVInt(size) + size;
+    }
+
+    /** serialized size including a short length prefix */
+    default int sizeWithShortLength(V value)
+    {
+        return 2 + size(value);
+    }
+
+    /**
+     * @return true if the size of the given value is zero, false otherwise
+     */
+    default boolean isEmpty(V value)
+    {
+        return size(value) == 0;
+    }
+
+    /**
+     * @return the number of bytes remaining in the value from the given offset
+     */
+    default int sizeFromOffset(V value, int offset)
+    {
+        return size(value) - offset;
+    }
+
+    /**
+     * @return true if there are no bytes present after the given offset, false otherwise
+     */
+    default boolean isEmptyFromOffset(V value, int offset)
+    {
+        return sizeFromOffset(value, offset) == 0;
+    }
+
+    /**
+     * allocate an instance of the accessors backing type
+     * @param length size of backing typ to allocate
+     */
+    V[] createArray(int length);
+
+    /**
+     * Write the contents of the given value into the a DataOutputPlus
+     */
+    void write(V value, DataOutputPlus out) throws IOException;
+
+    default void writeWithVIntLength(V value, DataOutputPlus out) throws IOException
+    {
+        out.writeUnsignedVInt(size(value));
+        write(value, out);
+    }
+
+    /**
+     * Write the contents of the given value into the ByteBuffer
+     */
+    void write(V value, ByteBuffer out);
+
+    /**
+     * copy the {@param size} bytes from the {@param src} value, starting at the offset {@param srcOffset} into
+     * the {@param dst} value, starting at the offset {@param dstOffset}, using the accessor {@param dstAccessor}
+     * @param <V2> the destination value type
+     * @return the number of bytes copied ({@param size})
+     */
+    <V2> int copyTo(V src, int srcOffset, V2 dst, ValueAccessor<V2> dstAccessor, int dstOffset, int size);
+
+    /**
+     * copies a byte array into this accessors value.
+     */
+    int copyByteArrayTo(byte[] src, int srcOffset, V dst, int dstOffset, int size);
+
+    /**
+     * copies a byte buffer into this accessors value.
+     */
+    int copyByteBufferTo(ByteBuffer src, int srcOffset, V dst, int dstOffset, int size);
+
+    /**
+     * updates {@param digest} with {@param size} bytes from the contents of {@param value} starting
+     * at offset {@param offset}
+     */
+    void digest(V value, int offset, int size, Digest digest);
+
+    /**
+     * updates {@param digest} with te contents of {@param value}
+     */
+    default void digest(V value, Digest digest)
+    {
+        digest(value, 0, size(value), digest);
+    }
+
+    /**
+     * Reads a value of {@param length} bytes from {@param in}
+     */
+    V read(DataInputPlus in, int length) throws IOException;
+
+    /**
+     * Returns a value with the contents of {@param input} from {@param offset} to {@param length}.
+     *
+     * Depending on the accessor implementation, this method may:
+     *  * allocate a new {@param <V>} object of {@param length}, and copy data into it
+     *  * return a view of {@param input} where changes to one will be reflected in the other
+     */
+    V slice(V input, int offset, int length);
+
+    /**
+     * same as {@link ValueAccessor#slice(Object, int, int)}, except the length is taked from the first
+     * 2 bytes from the given offset (and not included in the return value)
+     */
+    default V sliceWithShortLength(V input, int offset)
+    {
+        int size = getShort(input, offset);
+        return slice(input, offset + 2, size);
+    }
+
+    /**
+     * @return a short length prefix starting at {@param offset}
+     */
+    default int getShortLength(V v, int offset)
+    {
+        return getShort(v, offset);
+    }
+
+    /**
+     * lexicographically compare {@param left} to {@param right}
+     * @param <VR> backing type of
+     */
+    <VR> int compare(V left, VR right, ValueAccessor<VR> accessorR);
+
+    /**
+     * compare a byte array on the left with a {@param <V>} on the right}
+     */
+    int compareByteArrayTo(byte[] left, V right);
+
+    /**
+     * compare a byte buffer on the left with a {@param <V>} on the right}
+     */
+    int compareByteBufferTo(ByteBuffer left, V right);
+
+    default int hashCode(V value)
+    {
+        if (value == null)
+            return 0;
+
+        int result = 1;
+        for (int i=0, isize=size(value); i<isize; i++)
+            result = 31 * result + (int) getByte(value, i);
+
+        return result;
+    }
+
+    /**
+     * returns a ByteBuffer with the contents of {@param value}
+     *
+     * Depending on the accessor implementation, this method may:
+     *  * allocate a new ByteBuffer and copy data into it
+     *  * return the value, if the backing type is a bytebuffer
+     */
+    ByteBuffer toBuffer(V value);
+
+    /**
+     * returns a byte[] with the contents of {@param value}
+     *
+     * Depending on the accessor implementation, this method may:
+     *  * allocate a new byte[] object and copy data into it
+     *  * return the value, if the backing type is byte[]
+     */
+    byte[] toArray(V value);
+
+    /**
+     * returns a byte[] with {@param length} bytes copied from the contents of {@param value}
+     * starting at offset {@param offset}.
+     *
+     * Depending on the accessor implementation, this method may:
+     *  * allocate a new byte[] object and copy data into it
+     *  * return the value, if the backing type is byte[], offset is 0 and {@param length} == size(value)
+     */
+    byte[] toArray(V value, int offset, int length);
+    String toString(V value, Charset charset) throws CharacterCodingException;
+
+    default String toString(V value) throws CharacterCodingException
+    {
+        return toString(value, StandardCharsets.UTF_8);
+    }
+
+    String toHex(V value);
+
+    /** returns a boolean from offset {@param offset} */
+    default boolean getBoolean(V value, int offset)
+    {
+        return getByte(value, offset) != 0;
+    }
+
+    /** returns a byte from offset 0 */
+    byte toByte(V value);
+    /** returns a byte from offset {@param offset} */
+    byte getByte(V value, int offset);
+    /** returns a short from offset 0 */
+    short toShort(V value);
+    /** returns a short from offset {@param offset} */
+    short getShort(V value, int offset);
+    /** returns an int from offset 0 */
+    int toInt(V value);
+    /** returns an int from offset {@param offset} */
+    int getInt(V value, int offset);
+    /** returns a long from offset 0 */
+    long toLong(V value);
+    /** returns a long from offset {@param offset} */
+    long getLong(V value, int offset);
+    /** returns a float from offset 0 */
+    float toFloat(V value);
+
+    /** returns a double from offset 0 */
+    double toDouble(V value);
+
+    /** returns a UUID from offset 0 */
+    UUID toUUID(V value);
+
+    /**
+     * writes the short value {@param value} to {@param dst} at offset {@param offset}
+     * @return the number of bytes written to {@param value}
+     */
+    int putShort(V dst, int offset, short value);
+
+    /**
+     * writes the int value {@param value} to {@param dst} at offset {@param offset}
+     * @return the number of bytes written to {@param value}
+     */
+    int putInt(V dst, int offset, int value);
+
+    /**
+     * writes the long value {@param value} to {@param dst} at offset {@param offset}
+     * @return the number of bytes written to {@param value}
+     */
+    int putLong(V dst, int offset, long value);
+
+    /** return a value with a length of 0 */
+    V empty();
+
+    /**
+     * return a value containing the {@param bytes}
+     *
+     * Caller should assume that modifying the returned value
+     * will also modify the contents of {@param bytes}
+     */
+    V valueOf(byte[] bytes);
+
+    /**
+     * return a value containing the {@param bytes}
+     *
+     * {@param src} and the returned value may share a common byte array instance, so caller should
+     * assume that modifying the returned value will also modify the contents of {@param src}
+     */
+    V valueOf(ByteBuffer bytes);
+
+    /** return a value containing the bytes for the given string and charset */
+    V valueOf(String s, Charset charset);
+
+    /** return a value with the bytes from {@param v}*/
+    V valueOf(UUID v);
+    /** return a value with the bytes from {@param v}*/
+    V valueOf(boolean v);
+    /** return a value with the bytes from {@param v}*/
+    V valueOf(byte v);
+    /** return a value with the bytes from {@param v}*/
+    V valueOf(short v);
+    /** return a value with the bytes from {@param v}*/
+    V valueOf(int v);
+    /** return a value with the bytes from {@param v}*/
+    V valueOf(long v);
+    /** return a value with the bytes from {@param v}*/
+    V valueOf(float v);
+    /** return a value with the bytes from {@param v}*/
+    V valueOf(double v);
+
+    /**
+     * Convert the data in {@param src} to {@param <V>}
+     *
+     * {@param src} and the returned value may share a common byte array instance, so caller should
+     * assume that modifying the returned value will also modify the contents of {@param src}
+     */
+    <V2> V convert(V2 src, ValueAccessor<V2> accessor);
+
+    /**
+     * Allocate and return a {@param <V>} instance of {@param size} bytes on the heap.
+     */
+    V allocate(int size);
+
+    /**
+     * returns the {@link ValueAccessor.ObjectFactory} for the backing type {@param <V>}
+     */
+    ObjectFactory<V> factory();
+
+    /**
+     * lexicographically compare {@param left} to {@param right}
+     */
+    public static <L, R> int compare(L left, ValueAccessor<L> leftAccessor, R right, ValueAccessor<R> rightAccessor)
+    {
+        return leftAccessor.compare(left, right, rightAccessor);
+    }
+
+    public static <L, R> boolean equals(L left, ValueAccessor<L> leftAccessor, R right, ValueAccessor<R> rightAccessor)
+    {
+        return compare(left, leftAccessor, right, rightAccessor) == 0;
+    }
+}
diff --git a/src/java/org/apache/cassandra/serializers/AsciiSerializer.java b/src/java/org/apache/cassandra/db/marshal/ValueComparators.java
similarity index 56%
copy from src/java/org/apache/cassandra/serializers/AsciiSerializer.java
copy to src/java/org/apache/cassandra/db/marshal/ValueComparators.java
index e265cb2..d718032 100644
--- a/src/java/org/apache/cassandra/serializers/AsciiSerializer.java
+++ b/src/java/org/apache/cassandra/db/marshal/ValueComparators.java
@@ -15,28 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.cassandra.serializers;
+
+package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
+import java.util.Comparator;
 
-public class AsciiSerializer extends AbstractTextSerializer
+public class ValueComparators
 {
-    public static final AsciiSerializer instance = new AsciiSerializer();
+    public final Comparator<byte[]> array;
+    public final Comparator<ByteBuffer> buffer;
 
-    private AsciiSerializer()
+    public ValueComparators(Comparator<byte[]> array, Comparator<ByteBuffer> buffer)
     {
-        super(StandardCharsets.US_ASCII);
+        this.array = array;
+        this.buffer = buffer;
     }
 
-    public void validate(ByteBuffer bytes) throws MarshalException
+    public Comparator getForAccessor(ValueAccessor accessor)
     {
-        // 0-127
-        for (int i = bytes.position(); i < bytes.limit(); i++)
-        {
-            byte b = bytes.get(i);
-            if (b < 0)
-                throw new MarshalException("Invalid byte for ascii: " + Byte.toString(b));
-        }
+        if (accessor == ByteArrayAccessor.instance)
+            return array;
+        if (accessor == ByteBufferAccessor.instance)
+            return buffer;
+        throw new UnsupportedOperationException("Unsupported accessor: " + accessor.getClass().getName());
     }
 }
diff --git a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
index fc50de6..3059eb4 100644
--- a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
@@ -106,7 +106,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
         return holder().stats;
     }
 
-    public Row getRow(Clustering clustering)
+    public Row getRow(Clustering<?> clustering)
     {
         Row row = searchIterator(ColumnFilter.selection(columns()), false).next(clustering);
         // Note that for statics, this will never return null, this will return an empty row. However,
@@ -124,16 +124,16 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
         return row == null ? Rows.EMPTY_STATIC_ROW : row;
     }
 
-    public SearchIterator<Clustering, Row> searchIterator(final ColumnFilter columns, final boolean reversed)
+    public SearchIterator<Clustering<?>, Row> searchIterator(final ColumnFilter columns, final boolean reversed)
     {
         // TODO: we could optimize comparison for "NativeRow" à la #6755
         final Holder current = holder();
-        return new SearchIterator<Clustering, Row>()
+        return new SearchIterator<Clustering<?>, Row>()
         {
             private final SearchIterator<Clustering, Row> rawIter = BTree.slice(current.tree, metadata().comparator, desc(reversed));
             private final DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion();
 
-            public Row next(Clustering clustering)
+            public Row next(Clustering<?> clustering)
             {
                 if (clustering == Clustering.STATIC_CLUSTERING)
                     return staticRow(current, columns, true);
@@ -189,8 +189,8 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
 
     private UnfilteredRowIterator sliceIterator(ColumnFilter selection, Slice slice, boolean reversed, Holder current, Row staticRow)
     {
-        ClusteringBound start = slice.start() == ClusteringBound.BOTTOM ? null : slice.start();
-        ClusteringBound end = slice.end() == ClusteringBound.TOP ? null : slice.end();
+        ClusteringBound<?> start = slice.start().isBottom() ? null : slice.start();
+        ClusteringBound<?> end = slice.end().isTop() ? null : slice.end();
         Iterator<Row> rowIter = BTree.slice(current.tree, metadata().comparator, start, true, end, true, desc(reversed));
         Iterator<RangeTombstone> deleteIter = current.deletionInfo.rangeIterator(slice, reversed);
         return merge(rowIter, deleteIter, selection, reversed, current, staticRow);
diff --git a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
index 486bec7..ca20e5f 100644
--- a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
@@ -212,7 +212,7 @@ public final class AtomicBTreePartition extends AbstractBTreePartition
     }
 
     @Override
-    public Row getRow(Clustering clustering)
+    public Row getRow(Clustering<?> clustering)
     {
         return allocator.ensureOnHeap().applyToRow(super.getRow(clustering));
     }
@@ -224,7 +224,7 @@ public final class AtomicBTreePartition extends AbstractBTreePartition
     }
 
     @Override
-    public SearchIterator<Clustering, Row> searchIterator(ColumnFilter columns, boolean reversed)
+    public SearchIterator<Clustering<?>, Row> searchIterator(ColumnFilter columns, boolean reversed)
     {
         return allocator.ensureOnHeap().applyToPartition(super.searchIterator(columns, reversed));
     }
@@ -352,7 +352,7 @@ public final class AtomicBTreePartition extends AbstractBTreePartition
             this.indexer = indexer;
         }
 
-        private Row.Builder builder(Clustering clustering)
+        private Row.Builder builder(Clustering<?> clustering)
         {
             boolean isStatic = clustering == Clustering.STATIC_CLUSTERING;
             // We know we only insert/update one static per PartitionUpdate, so no point in saving the builder
diff --git a/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java
index 9a2b331..2183a98 100644
--- a/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java
@@ -92,7 +92,7 @@ public class CachedBTreePartition extends ImmutableBTreePartition implements Cac
                 ++cachedLiveRows;
 
             boolean hasNonExpiringLiveCell = false;
-            for (Cell cell : row.cells())
+            for (Cell<?> cell : row.cells())
             {
                 if (!cell.isTombstone() && !cell.isExpiring())
                 {
diff --git a/src/java/org/apache/cassandra/db/partitions/Partition.java b/src/java/org/apache/cassandra/db/partitions/Partition.java
index baeb6d5..2ee9c6d 100644
--- a/src/java/org/apache/cassandra/db/partitions/Partition.java
+++ b/src/java/org/apache/cassandra/db/partitions/Partition.java
@@ -50,12 +50,12 @@ public interface Partition
     /**
      * Returns the row corresponding to the provided clustering, or null if there is not such row.
      */
-    public Row getRow(Clustering clustering);
+    public Row getRow(Clustering<?> clustering);
 
     /**
      * Returns an iterator that allows to search specific rows efficiently.
      */
-    public SearchIterator<Clustering, Row> searchIterator(ColumnFilter columns, boolean reversed);
+    public SearchIterator<Clustering<?>, Row> searchIterator(ColumnFilter columns, boolean reversed);
 
     /**
      * Returns an UnfilteredRowIterator over all the rows/RT contained by this partition.
diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java b/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java
index 510b9c8..6d37640 100644
--- a/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java
@@ -24,7 +24,7 @@ public interface PartitionStatisticsCollector
 {
     public void update(LivenessInfo info);
     public void update(DeletionTime deletionTime);
-    public void update(Cell cell);
+    public void update(Cell<?> cell);
     public void updateColumnSetPerRow(long columnSetInRow);
     public void updateHasLegacyCounterShards(boolean hasLegacyCounterShards);
 }
diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
index 076c975..0bf45b0 100644
--- a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
@@ -395,13 +395,13 @@ public class PartitionUpdate extends AbstractBTreePartition
             {
                 if (cd.column().isSimple())
                 {
-                    maxTimestamp = Math.max(maxTimestamp, ((Cell)cd).timestamp());
+                    maxTimestamp = Math.max(maxTimestamp, ((Cell<?>)cd).timestamp());
                 }
                 else
                 {
                     ComplexColumnData complexData = (ComplexColumnData)cd;
                     maxTimestamp = Math.max(maxTimestamp, complexData.complexDeletion().markedForDeleteAt());
-                    for (Cell cell : complexData)
+                    for (Cell<?> cell : complexData)
                         maxTimestamp = Math.max(maxTimestamp, cell.timestamp());
                 }
             }
@@ -413,13 +413,13 @@ public class PartitionUpdate extends AbstractBTreePartition
             {
                 if (cd.column().isSimple())
                 {
-                    maxTimestamp = Math.max(maxTimestamp, ((Cell) cd).timestamp());
+                    maxTimestamp = Math.max(maxTimestamp, ((Cell<?>) cd).timestamp());
                 }
                 else
                 {
                     ComplexColumnData complexData = (ComplexColumnData) cd;
                     maxTimestamp = Math.max(maxTimestamp, complexData.complexDeletion().markedForDeleteAt());
-                    for (Cell cell : complexData)
+                    for (Cell<?> cell : complexData)
                         maxTimestamp = Math.max(maxTimestamp, cell.timestamp());
                 }
             }
@@ -447,7 +447,7 @@ public class PartitionUpdate extends AbstractBTreePartition
 
     private static void addMarksForRow(Row row, List<CounterMark> marks)
     {
-        for (Cell cell : row.cells())
+        for (Cell<?> cell : row.cells())
         {
             if (cell.isCounterCell())
                 marks.add(new CounterMark(row, cell.column(), cell.path()));
@@ -695,7 +695,7 @@ public class PartitionUpdate extends AbstractBTreePartition
             this.path = path;
         }
 
-        public Clustering clustering()
+        public Clustering<?> clustering()
         {
             return row.clustering();
         }
@@ -713,8 +713,8 @@ public class PartitionUpdate extends AbstractBTreePartition
         public ByteBuffer value()
         {
             return path == null
-                 ? row.getCell(column).value()
-                 : row.getCell(column, path).value();
+                 ? row.getCell(column).buffer()
+                 : row.getCell(column, path).buffer();
         }
 
         public void setValue(ByteBuffer value)
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractCell.java b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
index 3f2da96..5d4fc3c 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractCell.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -37,7 +38,7 @@ import org.apache.cassandra.utils.memory.AbstractAllocator;
  * Unless you have a very good reason not to, every cell implementation
  * should probably extend this class.
  */
-public abstract class AbstractCell extends Cell
+public abstract class AbstractCell<V> extends Cell<V>
 {
     protected AbstractCell(ColumnMetadata column)
     {
@@ -64,17 +65,17 @@ public abstract class AbstractCell extends Cell
         return ttl() != NO_TTL;
     }
 
-    public Cell markCounterLocalToBeCleared()
+    public Cell<?> markCounterLocalToBeCleared()
     {
         if (!isCounterCell())
             return this;
 
-        ByteBuffer value = value();
+        ByteBuffer value = buffer();
         ByteBuffer marked = CounterContext.instance().markLocalToBeCleared(value);
         return marked == value ? this : new BufferCell(column, timestamp(), ttl(), localDeletionTime(), marked, path());
     }
 
-    public Cell purge(DeletionPurger purger, int nowInSec)
+    public Cell<?> purge(DeletionPurger purger, int nowInSec)
     {
         if (!isLive(nowInSec))
         {
@@ -97,16 +98,16 @@ public abstract class AbstractCell extends Cell
         return this;
     }
 
-    public Cell copy(AbstractAllocator allocator)
+    public Cell<?> copy(AbstractAllocator allocator)
     {
         CellPath path = path();
-        return new BufferCell(column, timestamp(), ttl(), localDeletionTime(), allocator.clone(value()), path == null ? null : path.copy(allocator));
+        return new BufferCell(column, timestamp(), ttl(), localDeletionTime(), allocator.clone(buffer()), path == null ? null : path.copy(allocator));
     }
 
     // note: while the cell returned may be different, the value is the same, so if the value is offheap it must be referenced inside a guarded context (or copied)
-    public Cell updateAllTimestamp(long newTimestamp)
+    public Cell<?> updateAllTimestamp(long newTimestamp)
     {
-        return new BufferCell(column, isTombstone() ? newTimestamp - 1 : newTimestamp, ttl(), localDeletionTime(), value(), path());
+        return new BufferCell(column, isTombstone() ? newTimestamp - 1 : newTimestamp, ttl(), localDeletionTime(), buffer(), path());
     }
 
     public int dataSize()
@@ -115,16 +116,16 @@ public abstract class AbstractCell extends Cell
         return TypeSizes.sizeof(timestamp())
                + TypeSizes.sizeof(ttl())
                + TypeSizes.sizeof(localDeletionTime())
-               + value().remaining()
+               + valueSize()
                + (path == null ? 0 : path.dataSize());
     }
 
     public void digest(Digest digest)
     {
         if (isCounterCell())
-            digest.updateWithCounterContext(value());
+            digest.updateWithCounterContext(value(), accessor());
         else
-            digest.update(value());
+            digest.update(value(), accessor());
 
         digest.updateWithLong(timestamp())
               .updateWithInt(ttl())
@@ -161,6 +162,17 @@ public abstract class AbstractCell extends Cell
         return timestamp();
     }
 
+    public static <V1, V2> boolean equals(Cell<V1> left, Cell<V2> right)
+    {
+        return left.column().equals(right.column())
+               && left.isCounterCell() == right.isCounterCell()
+               && left.timestamp() == right.timestamp()
+               && left.ttl() == right.ttl()
+               && left.localDeletionTime() == right.localDeletionTime()
+               && ValueAccessor.equals(left.value(), left.accessor(), right.value(), right.accessor())
+               && Objects.equals(left.path(), right.path());
+    }
+
     @Override
     public boolean equals(Object other)
     {
@@ -170,27 +182,20 @@ public abstract class AbstractCell extends Cell
         if(!(other instanceof Cell))
             return false;
 
-        Cell that = (Cell)other;
-        return this.column().equals(that.column())
-            && this.isCounterCell() == that.isCounterCell()
-            && this.timestamp() == that.timestamp()
-            && this.ttl() == that.ttl()
-            && this.localDeletionTime() == that.localDeletionTime()
-            && Objects.equals(this.value(), that.value())
-            && Objects.equals(this.path(), that.path());
+        return equals(this, (Cell<?>) other);
     }
 
     @Override
     public int hashCode()
     {
-        return Objects.hash(column(), isCounterCell(), timestamp(), ttl(), localDeletionTime(), value(), path());
+        return Objects.hash(column(), isCounterCell(), timestamp(), ttl(), localDeletionTime(), accessor().hashCode(value()), path());
     }
 
     @Override
     public String toString()
     {
         if (isCounterCell())
-            return String.format("[%s=%d ts=%d]", column().name, CounterContext.instance().total(value()), timestamp());
+            return String.format("[%s=%d ts=%d]", column().name, CounterContext.instance().total(value(), accessor()), timestamp());
 
         AbstractType<?> type = column().type;
         if (type instanceof CollectionType && type.isMultiCell())
@@ -199,24 +204,24 @@ public abstract class AbstractCell extends Cell
             return String.format("[%s[%s]=%s %s]",
                                  column().name,
                                  ct.nameComparator().getString(path().get(0)),
-                                 ct.valueComparator().getString(value()),
+                                 ct.valueComparator().getString(value(), accessor()),
                                  livenessInfoString());
         }
         if (isTombstone())
             return String.format("[%s=<tombstone> %s]", column().name, livenessInfoString());
         else
-            return String.format("[%s=%s %s]", column().name, safeToString(type, value()), livenessInfoString());
+            return String.format("[%s=%s %s]", column().name, safeToString(type), livenessInfoString());
     }
 
-    private static String safeToString(AbstractType<?> type, ByteBuffer data)
+    private String safeToString(AbstractType<?> type)
     {
         try
         {
-            return type.getString(data);
+            return type.getString(value(), accessor());
         }
         catch (Exception e)
         {
-            return "0x" + ByteBufferUtil.bytesToHex(data);
+            return "0x" + ByteBufferUtil.bytesToHex(buffer());
         }
     }
 
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java b/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
index a7c48c1..7dac1fa 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ClusteringBoundOrBoundary;
 
-public abstract class AbstractRangeTombstoneMarker<B extends ClusteringBoundOrBoundary> implements RangeTombstoneMarker
+public abstract class AbstractRangeTombstoneMarker<B extends ClusteringBoundOrBoundary<?>> implements RangeTombstoneMarker
 {
     protected final B bound;
 
@@ -58,12 +58,10 @@ public abstract class AbstractRangeTombstoneMarker<B extends ClusteringBoundOrBo
 
     public void validateData(TableMetadata metadata)
     {
-        ClusteringBoundOrBoundary bound = clustering();
+        ClusteringBoundOrBoundary<?> bound = clustering();
         for (int i = 0; i < bound.size(); i++)
         {
-            ByteBuffer value = bound.get(i);
-            if (value != null)
-                metadata.comparator.subtype(i).validate(value);
+            bound.validate(i, metadata.comparator);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractRow.java b/src/java/org/apache/cassandra/db/rows/AbstractRow.java
index fc90e34..74f111a 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractRow.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractRow.java
@@ -16,7 +16,6 @@
  */
 package org.apache.cassandra.db.rows;
 
-import java.nio.ByteBuffer;
 import java.util.Objects;
 import java.util.function.Function;
 import java.util.stream.Collectors;
@@ -24,6 +23,7 @@ import java.util.stream.StreamSupport;
 
 import com.google.common.collect.Iterables;
 
+import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.Digest;
 import org.apache.cassandra.db.marshal.CollectionType;
@@ -70,17 +70,17 @@ public abstract class AbstractRow implements Row
         apply(ColumnData::digest, digest);
     }
 
-    public void validateData(TableMetadata metadata)
+    private <V> void validateClustering(TableMetadata metadata, Clustering<V> clustering)
     {
-        Clustering clustering = clustering();
+        ValueAccessor<V> accessor = clustering.accessor();
         for (int i = 0; i < clustering.size(); i++)
         {
-            ByteBuffer value = clustering.get(i);
+            V value = clustering.get(i);
             if (value != null)
             {
                 try
                 {
-                    metadata.comparator.subtype(i).validate(value);
+                    metadata.comparator.subtype(i).validate(value, accessor);
                 }
                 catch (Exception e)
                 {
@@ -88,6 +88,11 @@ public abstract class AbstractRow implements Row
                 }
             }
         }
+    }
+
+    public void validateData(TableMetadata metadata)
+    {
+        validateClustering(metadata, clustering());
 
         primaryKeyLivenessInfo().validate();
         if (deletion().time().localDeletionTime() < 0)
@@ -155,7 +160,7 @@ public abstract class AbstractRow implements Row
                     ComplexColumnData complexData = (ComplexColumnData)cd;
                     if (!complexData.complexDeletion().isLive())
                         sb.append("del(").append(cd.column().name).append(")=").append(complexData.complexDeletion());
-                    for (Cell cell : complexData)
+                    for (Cell<?> cell : complexData)
                         sb.append(", ").append(cell);
                 }
             }
@@ -163,24 +168,24 @@ public abstract class AbstractRow implements Row
             {
                 if (cd.column().isSimple())
                 {
-                    Cell cell = (Cell)cd;
+                    Cell<?> cell = (Cell<?>)cd;
                     sb.append(cell.column().name).append('=');
                     if (cell.isTombstone())
                         sb.append("<tombstone>");
                     else
-                        sb.append(cell.column().type.getString(cell.value()));
+                        sb.append(Cells.valueString(cell));
                 }
                 else
                 {
                     sb.append(cd.column().name).append('=');
                     ComplexColumnData complexData = (ComplexColumnData) cd;
-                    Function<Cell, String> transform = null;
+                    Function<Cell<?>, String> transform = null;
                     if (cd.column().type.isCollection())
                     {
                         CollectionType ct = (CollectionType) cd.column().type;
                         transform = cell -> String.format("%s -> %s",
                                                   ct.nameComparator().getString(cell.path().get(0)),
-                                                  ct.valueComparator().getString(cell.value()));
+                                                  Cells.valueString(cell, ct.valueComparator()));
 
                     }
                     else if (cd.column().type.isUDT())
@@ -190,7 +195,7 @@ public abstract class AbstractRow implements Row
                             Short fId = ut.nameComparator().getSerializer().deserialize(cell.path().get(0));
                             return String.format("%s -> %s",
                                                  ut.fieldNameAsString(fId),
-                                                 ut.fieldType(fId).getString(cell.value()));
+                                                 Cells.valueString(cell, ut.fieldType(fId)));
                         };
                     }
                     else
diff --git a/src/java/org/apache/cassandra/db/rows/ArrayCell.java b/src/java/org/apache/cassandra/db/rows/ArrayCell.java
new file mode 100644
index 0000000..5911e42
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/rows/ArrayCell.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.rows;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ExpirationDateOverflowHandling;
+import org.apache.cassandra.db.marshal.ByteArrayAccessor;
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.db.marshal.ValueAccessor;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+
+public class ArrayCell extends AbstractCell<byte[]>
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new ArrayCell(ColumnMetadata.regularColumn("", "", "", ByteType.instance), 0L, 0, 0, ByteArrayUtil.EMPTY_BYTE_ARRAY, null));
+
+    private final long timestamp;
+    private final int ttl;
+    private final int localDeletionTime;
+
+    private final byte[] value;
+    private final CellPath path;
+
+    public ArrayCell(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, byte[] value, CellPath path)
+    {
+        super(column);
+        this.timestamp = timestamp;
+        this.ttl = ttl;
+        this.localDeletionTime = localDeletionTime;
+        this.value = value;
+        this.path = path;
+    }
+
+    public long timestamp()
+    {
+        return timestamp;
+    }
+
+    public int ttl()
+    {
+        return ttl;
+    }
+
+    public int localDeletionTime()
+    {
+        return localDeletionTime;
+    }
+
+    public byte[] value()
+    {
+        return value;
+    }
+
+    public ValueAccessor<byte[]> accessor()
+    {
+        return ByteArrayAccessor.instance;
+    }
+
+    public CellPath path()
+    {
+        return path;
+    }
+
+    public Cell<?> withUpdatedColumn(ColumnMetadata newColumn)
+    {
+        return new ArrayCell(newColumn, timestamp, ttl, localDeletionTime, value, path);
+    }
+
+    public Cell<?> withUpdatedValue(ByteBuffer newValue)
+    {
+        return new ArrayCell(column, timestamp, ttl, localDeletionTime, ByteBufferUtil.getArray(newValue), path);
+    }
+
+    public Cell<?> withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        return new ArrayCell(column, newTimestamp, ttl, newLocalDeletionTime, value, path);
+    }
+
+    public Cell<?> copy(AbstractAllocator allocator)
+    {
+        if (value.length == 0)
+            return this;
+
+        return new BufferCell(column, timestamp, ttl, localDeletionTime, allocator.clone(value), path == null ? null : path.copy(allocator));
+    }
+
+    public long unsharedHeapSizeExcludingData()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOfArray(value) + value.length + (path == null ? 0 : path.unsharedHeapSizeExcludingData());
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/rows/BTreeRow.java b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
index bc9e7fe..4a03af2 100644
--- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java
+++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
@@ -67,7 +67,7 @@ public class BTreeRow extends AbstractRow
 {
     private static final long EMPTY_SIZE = ObjectSizes.measure(emptyRow(Clustering.EMPTY));
 
-    private final Clustering clustering;
+    private final Clustering<?> clustering;
     private final LivenessInfo primaryKeyLivenessInfo;
     private final Deletion deletion;
 
@@ -102,13 +102,13 @@ public class BTreeRow extends AbstractRow
         this.minLocalDeletionTime = minLocalDeletionTime;
     }
 
-    private BTreeRow(Clustering clustering, Object[] btree, int minLocalDeletionTime)
+    private BTreeRow(Clustering<?> clustering, Object[] btree, int minLocalDeletionTime)
     {
         this(clustering, LivenessInfo.EMPTY, Deletion.LIVE, btree, minLocalDeletionTime);
     }
 
     // Note that it's often easier/safer to use the sortedBuilder/unsortedBuilder or one of the static creation method below. Only directly useful in a small amount of cases.
-    public static BTreeRow create(Clustering clustering,
+    public static BTreeRow create(Clustering<?> clustering,
                                   LivenessInfo primaryKeyLivenessInfo,
                                   Deletion deletion,
                                   Object[] btree)
@@ -123,7 +123,7 @@ public class BTreeRow extends AbstractRow
         return create(clustering, primaryKeyLivenessInfo, deletion, btree, minDeletionTime);
     }
 
-    public static BTreeRow create(Clustering clustering,
+    public static BTreeRow create(Clustering<?> clustering,
                                   LivenessInfo primaryKeyLivenessInfo,
                                   Deletion deletion,
                                   Object[] btree,
@@ -132,27 +132,27 @@ public class BTreeRow extends AbstractRow
         return new BTreeRow(clustering, primaryKeyLivenessInfo, deletion, btree, minDeletionTime);
     }
 
-    public static BTreeRow emptyRow(Clustering clustering)
+    public static BTreeRow emptyRow(Clustering<?> clustering)
     {
         return new BTreeRow(clustering, BTree.empty(), Integer.MAX_VALUE);
     }
 
-    public static BTreeRow singleCellRow(Clustering clustering, Cell cell)
+    public static BTreeRow singleCellRow(Clustering<?> clustering, Cell<?> cell)
     {
         if (cell.column().isSimple())
             return new BTreeRow(clustering, BTree.singleton(cell), minDeletionTime(cell));
 
-        ComplexColumnData complexData = new ComplexColumnData(cell.column(), new Cell[]{ cell }, DeletionTime.LIVE);
+        ComplexColumnData complexData = new ComplexColumnData(cell.column(), new Cell<?>[]{ cell }, DeletionTime.LIVE);
         return new BTreeRow(clustering, BTree.singleton(complexData), minDeletionTime(cell));
     }
 
-    public static BTreeRow emptyDeletedRow(Clustering clustering, Deletion deletion)
+    public static BTreeRow emptyDeletedRow(Clustering<?> clustering, Deletion deletion)
     {
         assert !deletion.isLive();
         return new BTreeRow(clustering, LivenessInfo.EMPTY, deletion, BTree.empty(), Integer.MIN_VALUE);
     }
 
-    public static BTreeRow noCellLiveRow(Clustering clustering, LivenessInfo primaryKeyLivenessInfo)
+    public static BTreeRow noCellLiveRow(Clustering<?> clustering, LivenessInfo primaryKeyLivenessInfo)
     {
         assert !primaryKeyLivenessInfo.isEmpty();
         return new BTreeRow(clustering,
@@ -162,7 +162,7 @@ public class BTreeRow extends AbstractRow
                             minDeletionTime(primaryKeyLivenessInfo));
     }
 
-    private static int minDeletionTime(Cell cell)
+    private static int minDeletionTime(Cell<?> cell)
     {
         return cell.isTombstone() ? Integer.MIN_VALUE : cell.localDeletionTime();
     }
@@ -180,7 +180,7 @@ public class BTreeRow extends AbstractRow
     private static int minDeletionTime(ComplexColumnData cd)
     {
         int min = minDeletionTime(cd.complexDeletion());
-        for (Cell cell : cd)
+        for (Cell<?> cell : cd)
         {
             min = Math.min(min, minDeletionTime(cell));
             if (min == Integer.MIN_VALUE)
@@ -191,7 +191,7 @@ public class BTreeRow extends AbstractRow
 
     private static int minDeletionTime(ColumnData cd)
     {
-        return cd.column().isSimple() ? minDeletionTime((Cell) cd) : minDeletionTime((ComplexColumnData)cd);
+        return cd.column().isSimple() ? minDeletionTime((Cell<?>) cd) : minDeletionTime((ComplexColumnData)cd);
     }
 
     public void apply(Consumer<ColumnData> function)
@@ -239,7 +239,7 @@ public class BTreeRow extends AbstractRow
         return Ints.checkedCast(min);
     }
 
-    public Clustering clustering()
+    public Clustering<?> clustering()
     {
         return clustering;
     }
@@ -271,13 +271,13 @@ public class BTreeRow extends AbstractRow
         return deletion;
     }
 
-    public Cell getCell(ColumnMetadata c)
+    public Cell<?> getCell(ColumnMetadata c)
     {
         assert !c.isComplex();
-        return (Cell) BTree.<Object>find(btree, ColumnMetadata.asymmetricColumnDataComparator, c);
+        return (Cell<?>) BTree.<Object>find(btree, ColumnMetadata.asymmetricColumnDataComparator, c);
     }
 
-    public Cell getCell(ColumnMetadata c, CellPath path)
+    public Cell<?> getCell(ColumnMetadata c, CellPath path)
     {
         assert c.isComplex();
         ComplexColumnData cd = getComplexColumnData(c);
@@ -307,7 +307,7 @@ public class BTreeRow extends AbstractRow
         return searchIterator();
     }
 
-    public Iterable<Cell> cells()
+    public Iterable<Cell<?>> cells()
     {
         return CellIterator::new;
     }
@@ -358,7 +358,7 @@ public class BTreeRow extends AbstractRow
             if (column.isComplex())
                 return ((ComplexColumnData) cd).filter(filter, mayHaveShadowed ? activeDeletion : DeletionTime.LIVE, dropped, rowLiveness);
 
-            Cell cell = (Cell) cd;
+            Cell<?> cell = (Cell<?>) cd;
             // We include the cell unless it is 1) shadowed, 2) for a dropped column or 3) skippable.
             // And a cell is skippable if it is for a column that is not queried by the user and its timestamp
             // is lower than the row timestamp (see #10657 or SerializationHelper.includes() for details).
@@ -517,22 +517,22 @@ public class BTreeRow extends AbstractRow
     {
         ColumnData current = (ColumnData) BTree.<Object>find(btree, ColumnMetadata.asymmetricColumnDataComparator, column);
         if (column.isSimple())
-            BTree.replaceInSitu(btree, ColumnData.comparator, current, ((Cell) current).withUpdatedValue(value));
+            BTree.replaceInSitu(btree, ColumnData.comparator, current, ((Cell<?>) current).withUpdatedValue(value));
         else
             ((ComplexColumnData) current).setValue(path, value);
     }
 
-    public Iterable<Cell> cellsInLegacyOrder(TableMetadata metadata, boolean reversed)
+    public Iterable<Cell<?>> cellsInLegacyOrder(TableMetadata metadata, boolean reversed)
     {
         return () -> new CellInLegacyOrderIterator(metadata, reversed);
     }
 
-    private class CellIterator extends AbstractIterator<Cell>
+    private class CellIterator extends AbstractIterator<Cell<?>>
     {
         private Iterator<ColumnData> columnData = iterator();
-        private Iterator<Cell> complexCells;
+        private Iterator<Cell<?>> complexCells;
 
-        protected Cell computeNext()
+        protected Cell<?> computeNext()
         {
             while (true)
             {
@@ -551,19 +551,19 @@ public class BTreeRow extends AbstractRow
                 if (cd.column().isComplex())
                     complexCells = ((ComplexColumnData)cd).iterator();
                 else
-                    return (Cell)cd;
+                    return (Cell<?>)cd;
             }
         }
     }
 
-    private class CellInLegacyOrderIterator extends AbstractIterator<Cell>
+    private class CellInLegacyOrderIterator extends AbstractIterator<Cell<?>>
     {
         private final Comparator<ByteBuffer> comparator;
         private final boolean reversed;
         private final int firstComplexIdx;
         private int simpleIdx;
         private int complexIdx;
-        private Iterator<Cell> complexCells;
+        private Iterator<Cell<?>> complexCells;
         private final Object[] data;
 
         private CellInLegacyOrderIterator(TableMetadata metadata, boolean reversed)
@@ -605,13 +605,13 @@ public class BTreeRow extends AbstractRow
             return idx;
         }
 
-        private Iterator<Cell> makeComplexIterator(Object complexData)
+        private Iterator<Cell<?>> makeComplexIterator(Object complexData)
         {
             ComplexColumnData ccd = (ComplexColumnData)complexData;
             return reversed ? ccd.reverseIterator() : ccd.iterator();
         }
 
-        protected Cell computeNext()
+        protected Cell<?> computeNext()
         {
             while (true)
             {
@@ -633,10 +633,10 @@ public class BTreeRow extends AbstractRow
                 else
                 {
                     if (complexIdx >= data.length)
-                        return (Cell)data[getSimpleIdxAndIncrement()];
+                        return (Cell<?>)data[getSimpleIdxAndIncrement()];
 
                     if (comparator.compare(((ColumnData) data[getSimpleIdx()]).column().name.bytes, ((ColumnData) data[getComplexIdx()]).column().name.bytes) < 0)
-                        return (Cell)data[getSimpleIdxAndIncrement()];
+                        return (Cell<?>)data[getSimpleIdxAndIncrement()];
                     else
                         complexCells = makeComplexIterator(data[getComplexIdxAndIncrement()]);
                 }
@@ -662,12 +662,12 @@ public class BTreeRow extends AbstractRow
 
             public ColumnData resolve(Object[] cells, int lb, int ub)
             {
-                Cell cell = (Cell) cells[lb];
+                Cell<?> cell = (Cell<?>) cells[lb];
                 ColumnMetadata column = cell.column;
                 if (cell.column.isSimple())
                 {
                     while (++lb < ub)
-                        cell = Cells.reconcile(cell, (Cell) cells[lb]);
+                        cell = Cells.reconcile(cell, (Cell<?>) cells[lb]);
                     return cell;
                 }
 
@@ -681,7 +681,7 @@ public class BTreeRow extends AbstractRow
                 // legacy sstables (see #10743).
                 while (lb < ub)
                 {
-                    cell = (Cell) cells[lb];
+                    cell = (Cell<?>) cells[lb];
                     if (!(cell instanceof ComplexColumnDeletion))
                         break;
 
@@ -691,10 +691,10 @@ public class BTreeRow extends AbstractRow
                 }
 
                 List<Object> buildFrom = new ArrayList<>(ub - lb);
-                Cell previous = null;
+                Cell<?> previous = null;
                 for (int i = lb; i < ub; i++)
                 {
-                    Cell c = (Cell) cells[i];
+                    Cell<?> c = (Cell<?>) cells[i];
 
                     if (deletion == DeletionTime.LIVE || c.timestamp() >= deletion.markedForDeleteAt())
                     {
@@ -716,12 +716,12 @@ public class BTreeRow extends AbstractRow
             }
         }
 
-        protected Clustering clustering;
+        protected Clustering<?> clustering;
         protected LivenessInfo primaryKeyLivenessInfo = LivenessInfo.EMPTY;
         protected Deletion deletion = Deletion.LIVE;
 
         private final boolean isSorted;
-        private BTree.Builder<Cell> cells_;
+        private BTree.Builder<Cell<?>> cells_;
         private boolean hasComplex = false;
 
         // For complex column at index i of 'columns', we store at complexDeletions[i] its complex deletion.
@@ -732,7 +732,7 @@ public class BTreeRow extends AbstractRow
             this.isSorted = isSorted;
         }
 
-        private BTree.Builder<Cell> getCells()
+        private BTree.Builder<Cell<?>> getCells()
         {
             if (cells_ == null)
             {
@@ -763,13 +763,13 @@ public class BTreeRow extends AbstractRow
             return isSorted;
         }
 
-        public void newRow(Clustering clustering)
+        public void newRow(Clustering<?> clustering)
         {
             assert this.clustering == null; // Ensures we've properly called build() if we've use this builder before
             this.clustering = clustering;
         }
 
-        public Clustering clustering()
+        public Clustering<?> clustering()
         {
             return clustering;
         }
@@ -798,7 +798,7 @@ public class BTreeRow extends AbstractRow
                 this.primaryKeyLivenessInfo = LivenessInfo.EMPTY;
         }
 
-        public void addCell(Cell cell)
+        public void addCell(Cell<?> cell)
         {
             assert cell.column().isStatic() == (clustering == Clustering.STATIC_CLUSTERING) : "Column is " + cell.column() + ", clustering = " + clustering;
 
diff --git a/src/java/org/apache/cassandra/db/rows/BufferCell.java b/src/java/org/apache/cassandra/db/rows/BufferCell.java
index 8bf8f7d..786ac3c 100644
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@ -20,13 +20,15 @@ package org.apache.cassandra.db.rows;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.db.ExpirationDateOverflowHandling;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.marshal.ByteType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
-public class BufferCell extends AbstractCell
+public class BufferCell extends AbstractCell<ByteBuffer>
 {
     private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferCell(ColumnMetadata.regularColumn("", "", "", ByteType.instance), 0L, 0, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, null));
 
@@ -100,27 +102,32 @@ public class BufferCell extends AbstractCell
         return value;
     }
 
+    public ValueAccessor<ByteBuffer> accessor()
+    {
+        return ByteBufferAccessor.instance;
+    }
+
     public CellPath path()
     {
         return path;
     }
 
-    public Cell withUpdatedColumn(ColumnMetadata newColumn)
+    public Cell<?> withUpdatedColumn(ColumnMetadata newColumn)
     {
         return new BufferCell(newColumn, timestamp, ttl, localDeletionTime, value, path);
     }
 
-    public Cell withUpdatedValue(ByteBuffer newValue)
+    public Cell<?> withUpdatedValue(ByteBuffer newValue)
     {
         return new BufferCell(column, timestamp, ttl, localDeletionTime, newValue, path);
     }
 
-    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    public Cell<?> withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
     {
         return new BufferCell(column, newTimestamp, ttl, newLocalDeletionTime, value, path);
     }
 
-    public Cell copy(AbstractAllocator allocator)
+    public Cell<?> copy(AbstractAllocator allocator)
     {
         if (!value.hasRemaining())
             return this;
diff --git a/src/java/org/apache/cassandra/db/rows/Cell.java b/src/java/org/apache/cassandra/db/rows/Cell.java
index 959676a..6dd8f61 100644
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@ -23,10 +23,10 @@ import java.util.Comparator;
 
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
@@ -38,13 +38,13 @@ import org.apache.cassandra.utils.memory.AbstractAllocator;
  *   2) expiring cells: on top of regular cells, those have a ttl and a local deletion time (when they are expired).
  *   3) tombstone cells: those won't have value, but they have a local deletion time (when the tombstone was created).
  */
-public abstract class Cell extends ColumnData
+public abstract class Cell<V> extends ColumnData
 {
     public static final int NO_TTL = 0;
     public static final int NO_DELETION_TIME = Integer.MAX_VALUE;
     public static final int MAX_DELETION_TIME = Integer.MAX_VALUE - 1;
 
-    public final static Comparator<Cell> comparator = (c1, c2) ->
+    public final static Comparator<Cell<?>> comparator = (c1, c2) ->
     {
         int cmp = c1.column().compareTo(c2.column());
         if (cmp != 0)
@@ -56,6 +56,11 @@ public abstract class Cell extends ColumnData
 
     public static final Serializer serializer = new BufferCell.Serializer();
 
+    public interface Factory<V>
+    {
+        Cell<V> create(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, V value, CellPath path);
+    }
+
     protected Cell(ColumnMetadata column)
     {
         super(column);
@@ -68,12 +73,19 @@ public abstract class Cell extends ColumnData
      */
     public abstract boolean isCounterCell();
 
-    /**
-     * The cell value.
-     *
-     * @return the cell value.
-     */
-    public abstract ByteBuffer value();
+    public abstract V value();
+
+    public abstract ValueAccessor<V> accessor();
+
+    public int valueSize()
+    {
+        return accessor().size(value());
+    }
+
+    public ByteBuffer buffer()
+    {
+        return accessor().toBuffer(value());
+    }
 
     /**
      * The cell timestamp.
@@ -132,21 +144,21 @@ public abstract class Cell extends ColumnData
      */
     public abstract CellPath path();
 
-    public abstract Cell withUpdatedColumn(ColumnMetadata newColumn);
+    public abstract Cell<?> withUpdatedColumn(ColumnMetadata newColumn);
 
-    public abstract Cell withUpdatedValue(ByteBuffer newValue);
+    public abstract Cell<?> withUpdatedValue(ByteBuffer newValue);
 
-    public abstract Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
+    public abstract Cell<?> withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
 
-    public abstract Cell copy(AbstractAllocator allocator);
+    public abstract Cell<?> copy(AbstractAllocator allocator);
 
     @Override
     // Overrides super type to provide a more precise return type.
... 10434 lines suppressed ...


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