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

[cassandra] branch cep-15-accord updated: Partial Replication of Transaction State

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

benedict pushed a commit to branch cep-15-accord
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cep-15-accord by this push:
     new c3514ce877 Partial Replication of Transaction State
c3514ce877 is described below

commit c3514ce877b5a6b49488e7e31550f163c7c81ec6
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Sun Oct 16 12:14:34 2022 +0100

    Partial Replication of Transaction State
    
    The replication of transactions and their processing state must be optimal, i.e. linearly proportional to the size of the transaction and the number of keys involved. This patch ensures a transaction is replicated in slices, with each slice being written (replication factor) times only to the shard that owns the keys, and only once (except in the case the transaction is recovered).
    
    patch by Benedict; reviewed by Ariel Weisberg for CASSANDRA-17109
---
 .../org/apache/cassandra/db/PartitionPosition.java |   3 +-
 .../org/apache/cassandra/db/SystemKeyspace.java    |   4 +-
 .../org/apache/cassandra/dht/AbstractBounds.java   |   3 +-
 .../cassandra/dht/ByteOrderedPartitioner.java      |  47 ++-
 .../cassandra/dht/ComparableObjectToken.java       |   7 +
 .../org/apache/cassandra/dht/IPartitioner.java     |   5 +
 .../dht/IPartitionerDependentSerializer.java       |   3 +-
 .../apache/cassandra/dht/Murmur3Partitioner.java   |  12 +
 .../apache/cassandra/dht/RandomPartitioner.java    |  10 +
 src/java/org/apache/cassandra/dht/Token.java       |  49 ++-
 .../apache/cassandra/io/util/DataOutputPlus.java   |   1 -
 src/java/org/apache/cassandra/net/Verb.java        |  35 +-
 .../cassandra/serializers/SetSerializer.java       |   2 +-
 .../cassandra/service/accord/AccordCommand.java    | 392 +++++++++++----------
 .../service/accord/AccordCommandStore.java         | 139 +++++---
 .../service/accord/AccordCommandStores.java        |  33 +-
 .../service/accord/AccordCommandsForKey.java       | 108 +++---
 .../cassandra/service/accord/AccordKeyspace.java   | 154 ++++----
 .../service/accord/AccordMessageSink.java          |  18 +-
 .../service/accord/AccordObjectSizes.java          |  81 ++++-
 .../service/accord/AccordPartialCommand.java       | 241 ++++---------
 .../cassandra/service/accord/AccordState.java      |  28 +-
 .../cassandra/service/accord/AccordStateCache.java |   7 +-
 .../service/accord/AccordTopologyUtils.java        |  10 +-
 .../cassandra/service/accord/AccordTxnBuilder.java |   5 +-
 .../cassandra/service/accord/ListenerProxy.java    |  51 ++-
 .../cassandra/service/accord/ReadFuture.java       |   5 +-
 .../cassandra/service/accord/TokenRange.java       |  36 +-
 .../cassandra/service/accord/api/AccordAgent.java  |   6 +
 .../cassandra/service/accord/api/AccordKey.java    | 307 ++--------------
 .../api/{AccordKey.java => AccordRoutingKey.java}  | 245 ++++---------
 .../service/accord/async/AsyncLoader.java          |   2 +-
 .../service/accord/async/AsyncOperation.java       |  21 +-
 .../service/accord/async/AsyncWriter.java          |  69 ++--
 .../service/accord/db/AbstractKeyIndexed.java      | 133 ++++---
 .../cassandra/service/accord/db/AccordData.java    |  45 +--
 .../cassandra/service/accord/db/AccordQuery.java   |  26 +-
 .../cassandra/service/accord/db/AccordRead.java    |  36 +-
 .../cassandra/service/accord/db/AccordUpdate.java  | 177 +++++++---
 .../cassandra/service/accord/db/AccordWrite.java   |  12 +-
 .../accord/serializers/AcceptSerializers.java      | 128 +++----
 .../serializers/ApplyAndCheckSerializers.java      |  99 ------
 .../accord/serializers/ApplySerializers.java       |  41 +--
 .../serializers/BeginInvalidateSerializers.java    | 126 -------
 .../serializers/BeginInvalidationSerializers.java  | 140 ++++++++
 .../accord/serializers/CheckStatusSerializers.java | 118 ++++---
 .../accord/serializers/CommandSerializers.java     | 112 ++----
 .../accord/serializers/CommitSerializers.java      |  68 ++--
 .../service/accord/serializers/DepsSerializer.java | 120 +++++++
 .../service/accord/serializers/EnumSerializer.java |  58 +++
 .../accord/serializers/GetDepsSerializers.java     |  84 +++++
 .../serializers/InformDurableSerializers.java      |  58 +++
 ...zers.java => InformHomeDurableSerializers.java} |  32 +-
 .../serializers/InformOfTxnIdSerializers.java      |  53 +++
 .../accord/serializers/InformOfTxnSerializers.java |  82 -----
 .../service/accord/serializers/KeySerializers.java | 183 ++++++++--
 .../accord/serializers/PreacceptSerializers.java   |  44 ++-
 .../accord/serializers/ReadDataSerializers.java    |  61 ++--
 .../accord/serializers/RecoverySerializers.java    | 115 +++---
 .../accord/serializers/TopologySerializers.java    |   1 -
 .../accord/serializers/TxnRequestSerializer.java   |  43 ++-
 .../accord/serializers/WaitOnCommitSerializer.java |  20 +-
 .../service/accord/store/AbstractStoredField.java  |   6 +-
 .../service/accord/store/StoredBoolean.java        |   6 +-
 .../cassandra/service/accord/store/StoredLong.java |   6 +-
 .../service/accord/store/StoredNavigableMap.java   |  37 +-
 .../cassandra/service/accord/store/StoredSet.java  |  30 +-
 .../service/accord/store/StoredValue.java          |  10 +-
 .../test/accord/AccordIntegrationTest.java         |   6 +-
 .../cassandra/simulator/asm/InterceptClasses.java  |   3 +-
 .../paxos/PairOfSequencesAccordSimulation.java     |   2 +-
 .../service/accord/AccordCommandStoreTest.java     |  63 ++--
 .../service/accord/AccordCommandTest.java          |  98 +++---
 .../cassandra/service/accord/AccordTestUtils.java  |  92 +++--
 .../service/accord/AccordTopologyTest.java         |   2 +-
 .../service/accord/api/AccordKeyTest.java          |  10 +-
 .../service/accord/async/AsyncLoaderTest.java      |  44 +--
 .../service/accord/async/AsyncOperationTest.java   |  37 +-
 .../service/accord/async/AsyncWriterTest.java      |  96 +++--
 .../service/accord/db/AccordUpdateTest.java        |   4 +-
 .../accord/serializers/CommandSerializersTest.java |  15 +-
 .../service/accord/store/StoredMapTest.java        |   8 +-
 .../service/accord/store/StoredSetTest.java        |   8 +-
 .../service/accord/store/StoredValueTest.java      |   4 +-
 84 files changed, 2609 insertions(+), 2332 deletions(-)

diff --git a/src/java/org/apache/cassandra/db/PartitionPosition.java b/src/java/org/apache/cassandra/db/PartitionPosition.java
index 5e1d6184e2..09a6cba72c 100644
--- a/src/java/org/apache/cassandra/db/PartitionPosition.java
+++ b/src/java/org/apache/cassandra/db/PartitionPosition.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.dht.*;
+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.bytecomparable.ByteComparable;
@@ -100,7 +101,7 @@ public interface PartitionPosition extends RingPosition<PartitionPosition>, Byte
                 Token.serializer.serialize(pos.getToken(), out, version);
         }
 
-        public PartitionPosition deserialize(DataInput in, IPartitioner p, int version) throws IOException
+        public PartitionPosition deserialize(DataInputPlus in, IPartitioner p, int version) throws IOException
         {
             Kind kind = Kind.fromOrdinal(in.readByte());
             if (kind == Kind.ROW_KEY)
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 4e11c93d93..ece7563d95 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.db;
 
+import java.io.ByteArrayInputStream;
 import java.io.IOError;
 import java.io.IOException;
 import java.net.InetAddress;
@@ -84,6 +85,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableId;
 import org.apache.cassandra.io.sstable.SequenceBasedSSTableId;
 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.File;
 import org.apache.cassandra.io.util.RebufferingInputStream;
@@ -1794,7 +1796,7 @@ public final class SystemKeyspace
         try
         {
             // See rangeToBytes above for why version is 0.
-            return (Range<Token>) Range.tokenSerializer.deserialize(ByteStreams.newDataInput(ByteBufferUtil.getArray(rawRange)),
+            return (Range<Token>) Range.tokenSerializer.deserialize(new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(ByteBufferUtil.getArray(rawRange))),
                                                                     partitioner,
                                                                     0);
         }
diff --git a/src/java/org/apache/cassandra/dht/AbstractBounds.java b/src/java/org/apache/cassandra/dht/AbstractBounds.java
index 7a603b0a5d..a730d6ddf4 100644
--- a/src/java/org/apache/cassandra/dht/AbstractBounds.java
+++ b/src/java/org/apache/cassandra/dht/AbstractBounds.java
@@ -27,6 +27,7 @@ import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.db.TypeSizes;
 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.net.MessagingService;
 import org.apache.cassandra.utils.Pair;
@@ -195,7 +196,7 @@ public abstract class AbstractBounds<T extends RingPosition<T>> implements Seria
             serializer.serialize(range.right, out, version);
         }
 
-        public AbstractBounds<T> deserialize(DataInput in, IPartitioner p, int version) throws IOException
+        public AbstractBounds<T> deserialize(DataInputPlus in, IPartitioner p, int version) throws IOException
         {
             boolean isToken, startInclusive, endInclusive;
             // !WARNING! See serialize method above for why we still need to have that condition.
diff --git a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
index 2c3c9aab36..788763635a 100644
--- a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
@@ -144,8 +144,51 @@ public class ByteOrderedPartitioner implements IPartitioner
         @Override
         public Token increaseSlightly()
         {
-            throw new UnsupportedOperationException(String.format("Token type %s does not support token allocation.",
-                                                                  getClass().getSimpleName()));
+            // find first byte we can increment
+            int i = token.length - 1;
+            while (i >= 0)
+            {
+                if (token[i] != -1)
+                    break;
+                --i;
+            }
+            if (i == -1)
+                return new BytesToken(Arrays.copyOf(token, token.length + 1));
+
+            // increment and fill remainder with zeros
+            byte[] newToken = token.clone();
+            ++newToken[i];
+            Arrays.fill(newToken, i + 1, newToken.length, (byte)0);
+            return new BytesToken(newToken);
+        }
+
+        @Override
+        public Token decreaseSlightly()
+        {
+            if (token.length == 0)
+                throw new IndexOutOfBoundsException("Cannot create a smaller token the MINIMUM");
+
+            // find first byte we can decrement
+            int i = token.length - 1;
+            while (i >= 0)
+            {
+                if (token[i] != 0)
+                    break;
+                --i;
+            }
+            if (i == -1)
+            {
+                byte[] newToken = Arrays.copyOf(token, token.length - 1);
+                if (newToken.length > 0)
+                    newToken[newToken.length - 1] = (byte)-1;
+                return new BytesToken(newToken);
+            }
+
+            // decrement and fill remainder with -1
+            byte[] newToken = token.clone();
+            --newToken[i];
+            Arrays.fill(newToken, i + 1, newToken.length, (byte)-1);
+            return new BytesToken(newToken);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/dht/ComparableObjectToken.java b/src/java/org/apache/cassandra/dht/ComparableObjectToken.java
index 97c0c52d0d..e358693b99 100644
--- a/src/java/org/apache/cassandra/dht/ComparableObjectToken.java
+++ b/src/java/org/apache/cassandra/dht/ComparableObjectToken.java
@@ -80,4 +80,11 @@ abstract class ComparableObjectToken<C extends Comparable<C>> extends Token
         throw new UnsupportedOperationException(String.format("Token type %s does not support token allocation.",
                                                               getClass().getSimpleName()));
     }
+
+    @Override
+    public Token decreaseSlightly()
+    {
+        throw new UnsupportedOperationException(String.format("Token type %s does not support token allocation.",
+                                                              getClass().getSimpleName()));
+    }
 }
diff --git a/src/java/org/apache/cassandra/dht/IPartitioner.java b/src/java/org/apache/cassandra/dht/IPartitioner.java
index ef8ced25b1..0d32dc9c31 100644
--- a/src/java/org/apache/cassandra/dht/IPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/IPartitioner.java
@@ -136,6 +136,11 @@ public interface IPartitioner
         return Optional.empty();
     }
 
+    default boolean isFixedLength()
+    {
+        return false;
+    }
+
     default public int getMaxTokenSize()
     {
         return Integer.MIN_VALUE;
diff --git a/src/java/org/apache/cassandra/dht/IPartitionerDependentSerializer.java b/src/java/org/apache/cassandra/dht/IPartitionerDependentSerializer.java
index 3a9a76810b..4e02c58d91 100644
--- a/src/java/org/apache/cassandra/dht/IPartitionerDependentSerializer.java
+++ b/src/java/org/apache/cassandra/dht/IPartitionerDependentSerializer.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.dht;
 import java.io.DataInput;
 import java.io.IOException;
 
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 /**
@@ -49,7 +50,7 @@ public interface IPartitionerDependentSerializer<T>
      * @return the type that was deserialized
      * @throws IOException if deserialization fails
      */
-    public T deserialize(DataInput in, IPartitioner p, int version) throws IOException;
+    public T deserialize(DataInputPlus in, IPartitioner p, int version) throws IOException;
 
     /**
      * Calculate serialized size of object without actually serializing.
diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
index 8a8ec55d70..d461d4cb79 100644
--- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
+++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
@@ -221,11 +221,17 @@ public class Murmur3Partitioner implements IPartitioner
         @Override
         public LongToken increaseSlightly()
         {
+            if (token == MAXIMUM)
+                throw new IllegalArgumentException("Cannot increase above MAXIMUM");
+
             return new LongToken(token + 1);
         }
 
         public LongToken decreaseSlightly()
         {
+            if (equals(MINIMUM))
+                throw new IllegalArgumentException("Cannot decrease below MINIMUM");
+
             return new LongToken(token - 1);
         }
 
@@ -266,6 +272,12 @@ public class Murmur3Partitioner implements IPartitioner
         return new LongToken(normalize(hash[0]));
     }
 
+    @Override
+    public boolean isFixedLength()
+    {
+        return true;
+    }
+
     public int getMaxTokenSize()
     {
         return MAXIMUM_TOKEN_SIZE;
diff --git a/src/java/org/apache/cassandra/dht/RandomPartitioner.java b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
index a27d457b09..7d1e7505a1 100644
--- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
@@ -273,9 +273,19 @@ public class RandomPartitioner implements IPartitioner
 
         public Token increaseSlightly()
         {
+            if (token.equals(MAXIMUM))
+                throw new IllegalArgumentException("Cannot increase above MAXIMUM");
             return new BigIntegerToken(token.add(BigInteger.ONE));
         }
 
+        @Override
+        public Token decreaseSlightly()
+        {
+            if (token.equals(MINIMUM.token))
+                throw new IllegalArgumentException("Cannot decrease below MINIMUM");
+            return new BigIntegerToken(token.subtract(BigInteger.ONE));
+        }
+
         @Override
         public int tokenHash()
         {
diff --git a/src/java/org/apache/cassandra/dht/Token.java b/src/java/org/apache/cassandra/dht/Token.java
index be7bbbb113..e5b76e9a30 100644
--- a/src/java/org/apache/cassandra/dht/Token.java
+++ b/src/java/org/apache/cassandra/dht/Token.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.dht;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
@@ -25,6 +24,7 @@ import java.nio.ByteBuffer;
 import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 import org.apache.cassandra.utils.bytecomparable.ByteSource;
@@ -34,6 +34,7 @@ public abstract class Token implements RingPosition<Token>, Serializable
     private static final long serialVersionUID = 1L;
 
     public static final TokenSerializer serializer = new TokenSerializer();
+    public static final CompactTokenSerializer compactSerializer = new CompactTokenSerializer();
 
     public static abstract class TokenFactory
     {
@@ -76,6 +77,14 @@ public abstract class Token implements RingPosition<Token>, Serializable
             out.put(toByteArray(token));
         }
 
+        public Token deserialize(DataInputPlus in, IPartitioner p, int version) throws IOException
+        {
+            int size = p.isFixedLength() ? p.getMaxTokenSize() : (int)in.readUnsignedVInt();
+            byte[] bytes = new byte[size];
+            in.readFully(bytes);
+            return p.getTokenFactory().fromByteArray(ByteBuffer.wrap(bytes));
+        }
+
         public Token fromByteBuffer(ByteBuffer bytes, int position, int length)
         {
             bytes = bytes.duplicate();
@@ -99,7 +108,7 @@ public abstract class Token implements RingPosition<Token>, Serializable
             p.getTokenFactory().serialize(token, out);
         }
 
-        public Token deserialize(DataInput in, IPartitioner p, int version) throws IOException
+        public Token deserialize(DataInputPlus in, IPartitioner p, int version) throws IOException
         {
             int size = deserializeSize(in);
             byte[] bytes = new byte[size];
@@ -107,7 +116,7 @@ public abstract class Token implements RingPosition<Token>, Serializable
             return p.getTokenFactory().fromByteArray(ByteBuffer.wrap(bytes));
         }
 
-        public int deserializeSize(DataInput in) throws IOException
+        public int deserializeSize(DataInputPlus in) throws IOException
         {
             return in.readInt();
         }
@@ -120,6 +129,34 @@ public abstract class Token implements RingPosition<Token>, Serializable
         }
     }
 
+    public static class CompactTokenSerializer implements IPartitionerDependentSerializer<Token>
+    {
+        public void serialize(Token token, DataOutputPlus out, int version) throws IOException
+        {
+            IPartitioner p = token.getPartitioner();
+            if (!p.isFixedLength())
+                out.writeUnsignedVInt(p.getTokenFactory().byteSize(token));
+            p.getTokenFactory().serialize(token, out);
+        }
+
+        public Token deserialize(DataInputPlus in, IPartitioner p, int version) throws IOException
+        {
+            int size = p.isFixedLength() ? p.getMaxTokenSize() : (int)in.readUnsignedVInt();
+            byte[] bytes = new byte[size];
+            in.readFully(bytes);
+            return p.getTokenFactory().fromByteArray(ByteBuffer.wrap(bytes));
+        }
+
+        public long serializedSize(Token object, int version)
+        {
+            IPartitioner p = object.getPartitioner();
+            int byteSize = p.getTokenFactory().byteSize(object);
+            if (p.isFixedLength())
+                return byteSize;
+            return TypeSizes.sizeofUnsignedVInt(byteSize) + byteSize;
+        }
+    }
+
     abstract public IPartitioner getPartitioner();
     abstract public long getHeapSize();
     abstract public Object getTokenValue();
@@ -144,6 +181,7 @@ public abstract class Token implements RingPosition<Token>, Serializable
      * Used by the token allocation algorithm (see CASSANDRA-7032).
      */
     abstract public double size(Token next);
+
     /**
      * Returns a token that is slightly greater than this. Used to avoid clashes
      * between nodes in separate datacentres trying to use the same token via
@@ -151,6 +189,11 @@ public abstract class Token implements RingPosition<Token>, Serializable
      */
     abstract public Token increaseSlightly();
 
+    /**
+     * Returns a token that is slightly less than this.
+     */
+    abstract public Token decreaseSlightly();
+
     public Token getToken()
     {
         return this;
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java
index 205dab7dc2..5d0aafed5e 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java
@@ -104,7 +104,6 @@ public interface DataOutputPlus extends DataOutput
             default:
                 throw new IllegalArgumentException();
         }
-
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java
index f8ec811e7a..74290b11b2 100644
--- a/src/java/org/apache/cassandra/net/Verb.java
+++ b/src/java/org/apache/cassandra/net/Verb.java
@@ -76,11 +76,13 @@ import org.apache.cassandra.schema.SchemaMutationsSerializer;
 import org.apache.cassandra.schema.SchemaPullVerbHandler;
 import org.apache.cassandra.schema.SchemaPushVerbHandler;
 import org.apache.cassandra.schema.SchemaVersionVerbHandler;
-import org.apache.cassandra.service.accord.serializers.ApplyAndCheckSerializers;
-import org.apache.cassandra.service.accord.serializers.BeginInvalidateSerializers;
+import org.apache.cassandra.service.accord.serializers.BeginInvalidationSerializers;
 import org.apache.cassandra.service.accord.serializers.CheckStatusSerializers;
-import org.apache.cassandra.service.accord.serializers.InformOfPersistenceSerializers;
-import org.apache.cassandra.service.accord.serializers.InformOfTxnSerializers;
+import org.apache.cassandra.service.accord.serializers.EnumSerializer;
+import org.apache.cassandra.service.accord.serializers.GetDepsSerializers;
+import org.apache.cassandra.service.accord.serializers.InformDurableSerializers;
+import org.apache.cassandra.service.accord.serializers.InformHomeDurableSerializers;
+import org.apache.cassandra.service.accord.serializers.InformOfTxnIdSerializers;
 import org.apache.cassandra.service.paxos.PaxosCommit;
 import org.apache.cassandra.service.paxos.PaxosCommitAndPrepare;
 import org.apache.cassandra.service.paxos.PaxosPrepare;
@@ -219,6 +221,8 @@ public enum Verb
     PAXOS2_CLEANUP_COMPLETE_REQ      (48, P2, repairTimeout, PAXOS_REPAIR,      () -> PaxosCleanupComplete.serializer,         () -> PaxosCleanupComplete.verbHandler,                      PAXOS2_CLEANUP_COMPLETE_RSP      ),
 
     // accord
+    ACCORD_SIMPLE_RSP               (119, P2, writeTimeout, REQUEST_RESPONSE,     () -> EnumSerializer.simpleReply,           RESPONSE_HANDLER),
+
     ACCORD_PREACCEPT_RSP            (121, P2, writeTimeout,    REQUEST_RESPONSE,  () -> PreacceptSerializers.reply,           RESPONSE_HANDLER),
     ACCORD_PREACCEPT_REQ            (120, P2, writeTimeout,    ACCORD,            () -> PreacceptSerializers.request,         () -> AccordService.instance().verbHandler(),       ACCORD_PREACCEPT_RSP),
 
@@ -233,24 +237,27 @@ public enum Verb
 
     ACCORD_APPLY_RSP                (130, P2, writeTimeout,    REQUEST_RESPONSE,  () -> ApplySerializers.reply,               RESPONSE_HANDLER),
     ACCORD_APPLY_REQ                (129, P2, writeTimeout,    ACCORD,            () -> ApplySerializers.request,             () -> AccordService.instance().verbHandler(), ACCORD_APPLY_RSP),
-    ACCORD_APPLY_AND_CHECK_RSP      (132, P2, writeTimeout,    REQUEST_RESPONSE,  () -> ApplyAndCheckSerializers.reply,       RESPONSE_HANDLER),
-    ACCORD_APPLY_AND_CHECK_REQ      (131, P2, writeTimeout,    ACCORD,            () -> ApplyAndCheckSerializers.request,     () -> AccordService.instance().verbHandler(), ACCORD_APPLY_AND_CHECK_RSP),
 
     ACCORD_RECOVER_RSP              (134, P2, writeTimeout,    REQUEST_RESPONSE,  () -> RecoverySerializers.reply,            RESPONSE_HANDLER),
     ACCORD_RECOVER_REQ              (133, P2, writeTimeout,    ACCORD,            () -> RecoverySerializers.request,          () -> AccordService.instance().verbHandler(),       ACCORD_RECOVER_RSP  ),
-    ACCORD_BEGIN_INVALIDATE_RSP     (136, P2, writeTimeout,    REQUEST_RESPONSE,  () -> BeginInvalidateSerializers.reply,     RESPONSE_HANDLER),
-    ACCORD_BEGIN_INVALIDATE_REQ     (135, P2, writeTimeout,    ACCORD,            () -> BeginInvalidateSerializers.request,   () -> AccordService.instance().verbHandler(),ACCORD_BEGIN_INVALIDATE_RSP),
+    ACCORD_BEGIN_INVALIDATE_RSP     (136, P2, writeTimeout, REQUEST_RESPONSE, () -> BeginInvalidationSerializers.reply, RESPONSE_HANDLER),
+    ACCORD_BEGIN_INVALIDATE_REQ     (135, P2, writeTimeout, ACCORD, () -> BeginInvalidationSerializers.request, () -> AccordService.instance().verbHandler(), ACCORD_BEGIN_INVALIDATE_RSP),
     ACCORD_WAIT_COMMIT_RSP          (138, P2, writeTimeout,    REQUEST_RESPONSE,  () -> WaitOnCommitSerializer.reply,         RESPONSE_HANDLER),
     ACCORD_WAIT_COMMIT_REQ          (137, P2, writeTimeout,    ACCORD,            () -> WaitOnCommitSerializer.request,       () -> AccordService.instance().verbHandler(),     ACCORD_WAIT_COMMIT_RSP),
 
-    ACCORD_INFORM_OF_TXN_RSP        (140, P2, writeTimeout, REQUEST_RESPONSE,     () -> InformOfTxnSerializers.reply,         RESPONSE_HANDLER),
-    ACCORD_INFORM_OF_TXN_REQ        (139, P2, writeTimeout,    ACCORD,            () -> InformOfTxnSerializers.request,       () -> AccordService.instance().verbHandler(),   ACCORD_INFORM_OF_TXN_RSP),
+    ACCORD_INFORM_OF_TXNID_RSP(140, P2, writeTimeout, REQUEST_RESPONSE, () -> EnumSerializer.simpleReply, RESPONSE_HANDLER),
+    ACCORD_INFORM_OF_TXNID_REQ(139, P2, writeTimeout, ACCORD, () -> InformOfTxnIdSerializers.request, () -> AccordService.instance().verbHandler(), ACCORD_INFORM_OF_TXNID_RSP),
+
+    ACCORD_INFORM_HOME_DURABLE_REQ(141, P2, writeTimeout, ACCORD, () -> InformHomeDurableSerializers.request, () -> AccordService.instance().verbHandler(), ACCORD_SIMPLE_RSP),
+
+    ACCORD_INFORM_DURABLE_REQ(143, P2, writeTimeout, ACCORD, () -> InformDurableSerializers.request, () -> AccordService.instance().verbHandler(), ACCORD_SIMPLE_RSP),
+
+    ACCORD_CHECK_STATUS_RSP         (146, P2, writeTimeout, REQUEST_RESPONSE,     () -> CheckStatusSerializers.reply,         RESPONSE_HANDLER),
+    ACCORD_CHECK_STATUS_REQ         (145, P2, writeTimeout, ACCORD,               () -> CheckStatusSerializers.request,       () -> AccordService.instance().verbHandler(), ACCORD_CHECK_STATUS_RSP),
 
-    ACCORD_INFORM_OF_PERSISTENCE_RSP(142, P2, writeTimeout, REQUEST_RESPONSE,     () -> InformOfTxnSerializers.reply,         RESPONSE_HANDLER),
-    ACCORD_INFORM_OF_PERSIETENCE_REQ(141, P2, writeTimeout, ACCORD,               () -> InformOfPersistenceSerializers.request, () -> AccordService.instance().verbHandler(), ACCORD_INFORM_OF_TXN_RSP),
+    ACCORD_GET_DEPS_RSP         (148, P2, writeTimeout, REQUEST_RESPONSE, () -> GetDepsSerializers.reply, RESPONSE_HANDLER),
+    ACCORD_GET_DEPS_REQ         (147, P2, writeTimeout, ACCORD,               () -> GetDepsSerializers.request,       () -> AccordService.instance().verbHandler(), ACCORD_GET_DEPS_RSP),
 
-    ACCORD_CHECK_STATUS_RSP         (144, P2, writeTimeout, REQUEST_RESPONSE,     () -> CheckStatusSerializers.reply,         RESPONSE_HANDLER),
-    ACCORD_CHECK_STATUS_REQ         (143, P2, writeTimeout, ACCORD,               () -> CheckStatusSerializers.request,       () -> AccordService.instance().verbHandler(), ACCORD_CHECK_STATUS_RSP),
 
     // generic failure response
     FAILURE_RSP            (99,  P0, noTimeout,       REQUEST_RESPONSE,  () -> RequestFailureReason.serializer,      RESPONSE_HANDLER                             ),
diff --git a/src/java/org/apache/cassandra/serializers/SetSerializer.java b/src/java/org/apache/cassandra/serializers/SetSerializer.java
index bbf7911aaf..9f17b23e8d 100644
--- a/src/java/org/apache/cassandra/serializers/SetSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/SetSerializer.java
@@ -117,7 +117,7 @@ public class SetSerializer<T> extends AbstractMapSerializer<Set<T>>
                 l.add(elements.deserialize(value, accessor));
             }
             if (!accessor.isEmptyFromOffset(input, offset))
-                throw new MarshalException("Unexpected extraneous bytes after set value");
+                throw new MarshalException("Unexpected extraneous bytes after set value" + l + "," + accessor.toHex(input));
             return l;
         }
         catch (BufferUnderflowException | IndexOutOfBoundsException e)
diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommand.java b/src/java/org/apache/cassandra/service/accord/AccordCommand.java
index 00feea4c4e..e7e526495f 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordCommand.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordCommand.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.service.accord;
 
-import java.nio.ByteBuffer;
 import java.util.Objects;
 import java.util.TreeMap;
 import java.util.TreeSet;
@@ -32,33 +31,41 @@ import org.slf4j.LoggerFactory;
 import accord.api.Data;
 import accord.api.Key;
 import accord.api.Result;
+import accord.api.RoutingKey;
 import accord.local.Command;
-import accord.local.Listener;
+import accord.local.CommandStore;
+import accord.local.CommandListener;
 import accord.local.Listeners;
-import accord.local.PartialCommand;
 import accord.local.PreLoadContext;
+import accord.local.SafeCommandStore;
+import accord.local.SaveStatus;
 import accord.local.Status;
+import accord.local.Status.Durability;
+import accord.local.Status.Known;
+import accord.primitives.AbstractRoute;
 import accord.primitives.Ballot;
-import accord.primitives.Deps;
 import accord.primitives.KeyRanges;
 import accord.primitives.Keys;
+import accord.primitives.PartialDeps;
+import accord.primitives.PartialTxn;
 import accord.primitives.Timestamp;
+import accord.primitives.Txn;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
-import accord.txn.Writes;
+import accord.primitives.Writes;
 import accord.utils.DeterministicIdentitySet;
 import org.apache.cassandra.service.accord.api.AccordKey;
 import org.apache.cassandra.service.accord.async.AsyncContext;
 import org.apache.cassandra.service.accord.db.AccordData;
-import org.apache.cassandra.service.accord.store.StoredBoolean;
 import org.apache.cassandra.service.accord.store.StoredNavigableMap;
 import org.apache.cassandra.service.accord.store.StoredSet;
 import org.apache.cassandra.service.accord.store.StoredValue;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.concurrent.AsyncPromise;
 import org.apache.cassandra.utils.concurrent.Future;
 
+import static accord.local.Status.Durability.Local;
+import static accord.local.Status.Durability.NotDurable;
+import static accord.local.Status.PreApplied;
 import static org.apache.cassandra.service.accord.AccordState.WriteOnly.applyMapChanges;
 import static org.apache.cassandra.service.accord.AccordState.WriteOnly.applySetChanges;
 
@@ -68,15 +75,15 @@ public class AccordCommand extends Command implements AccordState<TxnId>
 
     private static final AtomicInteger INSTANCE_COUNTER = new AtomicInteger(0);
 
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new AccordCommand(null, null));
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new AccordCommand(null));
 
     public static class WriteOnly extends AccordCommand implements AccordState.WriteOnly<TxnId, AccordCommand>
     {
         private Future<?> future = null;
 
-        public WriteOnly(AccordCommandStore commandStore, TxnId txnId)
+        public WriteOnly(TxnId txnId)
         {
-            super(commandStore, txnId);
+            super(txnId);
         }
 
         @Override
@@ -95,79 +102,61 @@ public class AccordCommand extends Command implements AccordState<TxnId>
         @Override
         public void applyChanges(AccordCommand instance)
         {
-            applyMapChanges(this, instance, cmd -> cmd.waitingOnCommit);
+            applySetChanges(this, instance, cmd -> cmd.waitingOnCommit);
             applyMapChanges(this, instance, cmd -> cmd.waitingOnApply);
             applySetChanges(this, instance, cmd -> cmd.blockingCommitOn);
             applySetChanges(this, instance, cmd -> cmd.blockingApplyOn);
         }
     }
 
-    public static class ReadOnly extends AccordCommand implements AccordState.ReadOnly<TxnId, AccordCommand>
-    {
-        public ReadOnly(AccordCommandStore commandStore, TxnId txnId)
-        {
-            super(commandStore, txnId);
-        }
-
-        @Override
-        boolean isReadOnly()
-        {
-            return true;
-        }
-    }
-
-    private final AccordCommandStore commandStore;
     private final TxnId txnId;
     private final int instanceCount = INSTANCE_COUNTER.getAndIncrement();
-    public final StoredValue<Key> homeKey;
-    public final StoredValue<Key> progressKey;
-    public final StoredValue<Txn> txn;
+    public final StoredValue<AbstractRoute> route;
+    public final StoredValue<RoutingKey> homeKey;
+    public final StoredValue<RoutingKey> progressKey;
+    public final StoredValue<PartialTxn> partialTxn;
     public final StoredValue<Ballot> promised;
     public final StoredValue<Ballot> accepted;
     public final StoredValue<Timestamp> executeAt;
-    public final StoredValue<Deps> deps;
+    public final StoredValue<PartialDeps> partialDeps;
     public final StoredValue<Writes> writes;
     public final StoredValue<Result> result;
 
-    public final StoredValue.HistoryPreserving<Status> status;
-    public final StoredBoolean isGloballyPersistent;
+    public final StoredValue.HistoryPreserving<SaveStatus> status;
+    public final StoredValue<Durability> durability;
 
-    public final StoredNavigableMap<TxnId, ByteBuffer> waitingOnCommit;
-    public final StoredNavigableMap<TxnId, ByteBuffer> waitingOnApply;
+    public final StoredSet.Navigable<TxnId> waitingOnCommit;
+    public final StoredNavigableMap<Timestamp, TxnId> waitingOnApply;
+    public final StoredSet.Navigable<TxnId> blockingCommitOn;
+    public final StoredSet.Navigable<TxnId> blockingApplyOn;
 
     public final StoredSet.DeterministicIdentity<ListenerProxy> storedListeners;
     private final Listeners transientListeners;
 
-    public final StoredSet.Navigable<TxnId> blockingCommitOn;
-    public final StoredSet.Navigable<TxnId> blockingApplyOn;
-
-    public AccordCommand(AccordCommandStore commandStore, TxnId txnId)
+    public AccordCommand(TxnId txnId)
     {
         logger.trace("Instantiating new command {} @ {}", txnId, instanceHash());
-        this.commandStore = commandStore;
         this.txnId = txnId;
-        homeKey = new StoredValue<>(kind());
-        progressKey = new StoredValue<>(kind());
-        txn = new StoredValue<>(kind());
-        promised = new StoredValue<>(kind());
-        accepted = new StoredValue<>(kind());
-        executeAt = new StoredValue<>(kind());
-        deps = new StoredValue<>(kind());
-        writes = new StoredValue<>(kind());
-        result = new StoredValue<>(kind());
-        status = new StoredValue.HistoryPreserving<>(kind());
-        isGloballyPersistent = new StoredBoolean(kind());
-        waitingOnCommit = new StoredNavigableMap<>(kind());
-        waitingOnApply = new StoredNavigableMap<>(kind());
-        storedListeners = new StoredSet.DeterministicIdentity<>(kind());
+        homeKey = new StoredValue<>(rw());
+        progressKey = new StoredValue<>(rw());
+        route = new StoredValue<>(rw());
+        partialTxn = new StoredValue<>(rw());
+        promised = new StoredValue<>(rw());
+        accepted = new StoredValue<>(rw());
+        executeAt = new StoredValue<>(rw());
+        partialDeps = new StoredValue<>(rw());
+        writes = new StoredValue<>(rw());
+        result = new StoredValue<>(rw());
+        status = new StoredValue.HistoryPreserving<>(rw());
+        durability = new StoredValue<>(rw());
+        waitingOnCommit = new StoredSet.Navigable<>(rw());
+        waitingOnApply = new StoredNavigableMap<>(rw());
+        storedListeners = new StoredSet.DeterministicIdentity<>(rw());
         transientListeners = new Listeners();
-        blockingCommitOn = new StoredSet.Navigable<>(kind());
-        blockingApplyOn = new StoredSet.Navigable<>(kind());
+        blockingCommitOn = new StoredSet.Navigable<>(rw());
+        blockingApplyOn = new StoredSet.Navigable<>(rw());
     }
 
-
-
-
     @Override
     public String toString()
     {
@@ -184,8 +173,8 @@ public class AccordCommand extends Command implements AccordState<TxnId>
 //               ", txn=" + txn +
 //               ", writes=" + writes +
 //               ", result=" + result +
-               ", txn is null?=" + (txn.get() == null) +
-               ", isGloballyPersistent=" + isGloballyPersistent +
+               ", txn is null?=" + (partialTxn.get() == null) +
+               ", durability=" + durability +
                ", waitingOnCommit=" + waitingOnCommit +
                ", waitingOnApply=" + waitingOnApply +
                ", storedListeners=" + storedListeners +
@@ -200,15 +189,16 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     {
         return homeKey.isEmpty()
                || progressKey.isEmpty()
-               || txn.isEmpty()
+               || route.isEmpty()
+               || partialTxn.isEmpty()
                || promised.isEmpty()
                || accepted.isEmpty()
                || executeAt.isEmpty()
-               || deps.isEmpty()
+               || partialDeps.isEmpty()
                || writes.isEmpty()
                || result.isEmpty()
                || status.isEmpty()
-               || isGloballyPersistent.isEmpty()
+               || durability.isEmpty()
                || waitingOnCommit.isEmpty()
                || blockingCommitOn.isEmpty()
                || waitingOnApply.isEmpty()
@@ -220,15 +210,16 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     {
         homeKey.setEmpty();
         progressKey.setEmpty();
-        txn.setEmpty();
+        route.setEmpty();
+        partialTxn.setEmpty();
         promised.setEmpty();
         accepted.setEmpty();
         executeAt.setEmpty();
-        deps.setEmpty();
+        partialDeps.setEmpty();
         writes.setEmpty();
         result.setEmpty();
         status.setEmpty();
-        isGloballyPersistent.setEmpty();
+        durability.setEmpty();
         waitingOnCommit.setEmpty();
         blockingCommitOn.setEmpty();
         waitingOnApply.setEmpty();
@@ -239,20 +230,21 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     public AccordCommand initialize()
     {
         logger.trace("Initializing command {} @ {}", txnId, instanceHash());
-        status.set(Status.NotWitnessed);
+        status.set(SaveStatus.NotWitnessed);
         homeKey.set(null);
         progressKey.set(null);
-        txn.set(null);
+        route.set(null);
+        partialTxn.set(null);
         executeAt.load(null);
         promised.set(Ballot.ZERO);
         accepted.set(Ballot.ZERO);
-        deps.set(Deps.NONE);
+        partialDeps.set(PartialDeps.NONE);
         writes.load(null);
         result.load(null);
-        isGloballyPersistent.set(false);
-        waitingOnCommit.load(new TreeMap<>());
-        blockingCommitOn.load(new TreeSet<>());
+        durability.set(Durability.NotDurable);
+        waitingOnCommit.load(new TreeSet<>());
         waitingOnApply.load(new TreeMap<>());
+        blockingCommitOn.load(new TreeSet<>());
         blockingApplyOn.load(new TreeSet<>());
         storedListeners.load(new DeterministicIdentitySet<>());
         return this;
@@ -263,15 +255,16 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     {
         return homeKey.isLoaded()
                && progressKey.isLoaded()
-               && txn.isLoaded()
+               && route.isLoaded()
+               && partialTxn.isLoaded()
                && promised.isLoaded()
                && accepted.isLoaded()
                && executeAt.isLoaded()
-               && deps.isLoaded()
+               && partialDeps.isLoaded()
                && writes.isLoaded()
                && result.isLoaded()
                && status.isLoaded()
-               && isGloballyPersistent.isLoaded()
+               && durability.isLoaded()
                && waitingOnCommit.isLoaded()
                && blockingCommitOn.isLoaded()
                && waitingOnApply.isLoaded()
@@ -283,15 +276,16 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     {
         return homeKey.isLoaded()
                || progressKey.isLoaded()
-               || txn.isLoaded()
+               || route.isLoaded()
+               || partialTxn.isLoaded()
                || promised.isLoaded()
                || accepted.isLoaded()
                || executeAt.isLoaded()
-               || deps.isLoaded()
+               || partialDeps.isLoaded()
                || writes.isLoaded()
                || result.isLoaded()
                || status.isLoaded()
-               || isGloballyPersistent.isLoaded()
+               || durability.isLoaded()
                || waitingOnCommit.isLoaded()
                || blockingCommitOn.isLoaded()
                || waitingOnApply.isLoaded()
@@ -304,15 +298,16 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     {
         return homeKey.hasModifications()
                || progressKey.hasModifications()
-               || txn.hasModifications()
+               || route.hasModifications()
+               || partialTxn.hasModifications()
                || promised.hasModifications()
                || accepted.hasModifications()
                || executeAt.hasModifications()
-               || deps.hasModifications()
+               || partialDeps.hasModifications()
                || writes.hasModifications()
                || result.hasModifications()
                || status.hasModifications()
-               || isGloballyPersistent.hasModifications()
+               || durability.hasModifications()
                || waitingOnCommit.hasModifications()
                || blockingCommitOn.hasModifications()
                || waitingOnApply.hasModifications()
@@ -326,15 +321,16 @@ public class AccordCommand extends Command implements AccordState<TxnId>
         logger.trace("Clearing modified flag on command {} @ {}", txnId, instanceHash());
         homeKey.clearModifiedFlag();
         progressKey.clearModifiedFlag();
-        txn.clearModifiedFlag();
+        route.clearModifiedFlag();
+        partialTxn.clearModifiedFlag();
         promised.clearModifiedFlag();
         accepted.clearModifiedFlag();
         executeAt.clearModifiedFlag();
-        deps.clearModifiedFlag();
+        partialDeps.clearModifiedFlag();
         writes.clearModifiedFlag();
         result.clearModifiedFlag();
         status.clearModifiedFlag();
-        isGloballyPersistent.clearModifiedFlag();
+        durability.clearModifiedFlag();
         waitingOnCommit.clearModifiedFlag();
         blockingCommitOn.clearModifiedFlag();
         waitingOnApply.clearModifiedFlag();
@@ -348,19 +344,19 @@ public class AccordCommand extends Command implements AccordState<TxnId>
         if (this == o) return true;
         if (o == null || getClass() != o.getClass()) return false;
         AccordCommand command = (AccordCommand) o;
-        return commandStore == command.commandStore
-               && homeKey.equals(command.homeKey)
+        return    homeKey.equals(command.homeKey)
                && progressKey.equals(command.progressKey)
+               && route.equals(command.route)
                && txnId.equals(command.txnId)
-               && txn.equals(command.txn)
+               && partialTxn.equals(command.partialTxn)
                && promised.equals(command.promised)
                && accepted.equals(command.accepted)
                && executeAt.equals(command.executeAt)
-               && deps.equals(command.deps)
+               && partialDeps.equals(command.partialDeps)
                && writes.equals(command.writes)
                && result.equals(command.result)
                && status.equals(command.status)
-               && isGloballyPersistent.equals(command.isGloballyPersistent)
+               && durability.equals(command.durability)
                && waitingOnCommit.equals(command.waitingOnCommit)
                && blockingCommitOn.equals(command.blockingCommitOn)
                && waitingOnApply.equals(command.waitingOnApply)
@@ -383,19 +379,19 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     @Override
     public int hashCode()
     {
-        return Objects.hash(commandStore,
-                            txnId,
+        return Objects.hash(txnId,
                             homeKey,
                             progressKey,
-                            txn,
+                            route,
+                            partialTxn,
                             promised,
                             accepted,
                             executeAt,
-                            deps,
+                            partialDeps,
                             writes,
                             result,
                             status,
-                            isGloballyPersistent,
+                            durability,
                             waitingOnCommit,
                             blockingCommitOn,
                             waitingOnApply,
@@ -404,11 +400,6 @@ public class AccordCommand extends Command implements AccordState<TxnId>
                             transientListeners);
     }
 
-    private AccordStateCache.Instance<TxnId, AccordCommand> cache()
-    {
-        return commandStore.commandCache();
-    }
-
     @Override
     public TxnId key()
     {
@@ -422,18 +413,19 @@ public class AccordCommand extends Command implements AccordState<TxnId>
         size += AccordObjectSizes.timestamp(txnId);
         size += homeKey.estimatedSizeOnHeap(AccordObjectSizes::key);
         size += progressKey.estimatedSizeOnHeap(AccordObjectSizes::key);
-        size += txn.estimatedSizeOnHeap(AccordObjectSizes::txn);
+        size += route.estimatedSizeOnHeap(AccordObjectSizes::route);
+        size += partialTxn.estimatedSizeOnHeap(AccordObjectSizes::txn);
         size += promised.estimatedSizeOnHeap(AccordObjectSizes::timestamp);
         size += accepted.estimatedSizeOnHeap(AccordObjectSizes::timestamp);
         size += executeAt.estimatedSizeOnHeap(AccordObjectSizes::timestamp);
-        size += deps.estimatedSizeOnHeap(AccordObjectSizes::dependencies);
+        size += partialDeps.estimatedSizeOnHeap(AccordObjectSizes::dependencies);
         size += writes.estimatedSizeOnHeap(AccordObjectSizes::writes);
         size += result.estimatedSizeOnHeap(r -> ((AccordData) r).estimatedSizeOnHeap());
         size += status.estimatedSizeOnHeap(s -> 0);
-        size += isGloballyPersistent.estimatedSizeOnHeap();
-        size += waitingOnCommit.estimatedSizeOnHeap(AccordObjectSizes::timestamp, ByteBufferUtil::estimatedSizeOnHeap);
+        size += durability.estimatedSizeOnHeap(s -> 0);
+        size += waitingOnCommit.estimatedSizeOnHeap(AccordObjectSizes::timestamp);
         size += blockingCommitOn.estimatedSizeOnHeap(AccordObjectSizes::timestamp);
-        size += waitingOnApply.estimatedSizeOnHeap(AccordObjectSizes::timestamp, ByteBufferUtil::estimatedSizeOnHeap);
+        size += waitingOnApply.estimatedSizeOnHeap(AccordObjectSizes::timestamp, AccordObjectSizes::timestamp);
         size += blockingApplyOn.estimatedSizeOnHeap(AccordObjectSizes::timestamp);
         size += storedListeners.estimatedSizeOnHeap(ListenerProxy::estimatedSizeOnHeap);
         return size;
@@ -453,45 +445,51 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     }
 
     @Override
-    public AccordCommandStore commandStore()
-    {
-        return commandStore;
-    }
-
-    @Override
-    public Key homeKey()
+    public RoutingKey homeKey()
     {
         return homeKey.get();
     }
 
     @Override
-    protected void setHomeKey(Key key)
+    protected void setHomeKey(RoutingKey key)
     {
         homeKey.set(key);
     }
 
     @Override
-    public Key progressKey()
+    public RoutingKey progressKey()
     {
         return progressKey.get();
     }
 
     @Override
-    protected void setProgressKey(Key key)
+    protected void setProgressKey(RoutingKey key)
     {
         progressKey.set(key);
     }
 
     @Override
-    public Txn txn()
+    public AbstractRoute route()
+    {
+        return route.get();
+    }
+
+    @Override
+    protected void setRoute(AbstractRoute newRoute)
     {
-        return txn.get();
+        route.set(newRoute);
     }
 
     @Override
-    protected void setTxn(Txn txn)
+    public PartialTxn partialTxn()
     {
-        this.txn.set(txn);
+        return partialTxn.get();
+    }
+
+    @Override
+    public void setPartialTxn(PartialTxn txn)
+    {
+        this.partialTxn.set(txn);
     }
 
     @Override
@@ -501,7 +499,7 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     }
 
     @Override
-    public void promised(Ballot ballot)
+    public void setPromised(Ballot ballot)
     {
         this.promised.set(ballot);
     }
@@ -513,7 +511,7 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     }
 
     @Override
-    public void accepted(Ballot ballot)
+    public void setAccepted(Ballot ballot)
     {
         this.accepted.set(ballot);
     }
@@ -525,22 +523,28 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     }
 
     @Override
-    public void executeAt(Timestamp timestamp)
+    public Txn.Kind kind()
+    {
+        return partialTxn.get().kind();
+    }
+
+    @Override
+    public void setExecuteAt(Timestamp timestamp)
     {
         Preconditions.checkState(!status().hasBeen(Status.Committed) || executeAt().equals(timestamp));
         this.executeAt.set(timestamp);
     }
 
     @Override
-    public Deps savedDeps()
+    public PartialDeps partialDeps()
     {
-        return deps.get();
+        return partialDeps.get();
     }
 
     @Override
-    public void savedDeps(Deps deps)
+    public void setPartialDeps(PartialDeps deps)
     {
-        this.deps.set(deps);
+        this.partialDeps.set(deps);
     }
 
     @Override
@@ -550,7 +554,7 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     }
 
     @Override
-    public void writes(Writes writes)
+    public void setWrites(Writes writes)
     {
         this.writes.set(writes);
     }
@@ -562,53 +566,69 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     }
 
     @Override
-    public void result(Result result)
+    public void setResult(Result result)
     {
         this.result.set(result);
     }
 
     @Override
-    public Status status()
+    public SaveStatus saveStatus()
     {
         return status.get();
     }
 
     @Override
-    public void status(Status status)
+    public void setSaveStatus(SaveStatus status)
     {
         this.status.set(status);
     }
 
     @Override
-    public boolean isGloballyPersistent()
+    public void setStatus(Status status)
     {
-        return isGloballyPersistent.get();
+        super.setStatus(status);
     }
 
     @Override
-    public void isGloballyPersistent(boolean v)
+    public Known known()
     {
-        isGloballyPersistent.set(v);
+        return this.status.get().known;
     }
 
     @Override
-    protected void postApply()
+    public Durability durability()
     {
-        cache().cleanupWriteFuture(txnId);
-        super.postApply();
+        Durability durability = this.durability.get();
+        if (status().hasBeen(PreApplied) && durability == NotDurable)
+            return Local; // not necessary anywhere, but helps for logical consistency
+        return durability;
     }
 
-    private boolean canApplyWithCurrentScope()
+    @Override
+    public void setDurability(Durability v)
     {
-        KeyRanges ranges = commandStore.ranges().at(executeAt().epoch);
-        Keys keys = txn().keys();
+        durability.set(v);
+    }
+
+    @Override
+    protected void postApply(SafeCommandStore safeStore)
+    {
+        AccordStateCache.Instance<TxnId, AccordCommand> cache = ((AccordCommandStore) safeStore).commandCache();
+        cache.cleanupWriteFuture(txnId);
+        super.postApply(safeStore);
+    }
+
+    private boolean canApplyWithCurrentScope(SafeCommandStore safeStore)
+    {
+        KeyRanges ranges = safeStore.ranges().at(executeAt().epoch);
+        Keys keys = partialTxn().keys();
         for (int i=0,mi=keys.size(); i<mi; i++)
         {
             Key key = keys.get(i);
-            if (commandStore.isCommandsForKeyInContext((AccordKey.PartitionKey) key))
+            if (((AccordCommandStore)safeStore).isCommandsForKeyInContext((AccordKey.PartitionKey) key))
                 continue;
 
-            if (!commandStore.hashIntersects(key))
+            if (!safeStore.commandStore().hashIntersects(key))
                 continue;
             if (!ranges.contains(key))
                 continue;
@@ -618,13 +638,13 @@ public class AccordCommand extends Command implements AccordState<TxnId>
         return true;
     }
 
-    private Future<Void> applyWithCorrectScope()
+    private Future<Void> applyWithCorrectScope(CommandStore unsafeStore)
     {
         TxnId txnId = txnId();
         AsyncPromise<Void> promise = new AsyncPromise<>();
-        commandStore().process(this, commandStore -> {
-            AccordCommand command = (AccordCommand) commandStore.command(txnId);
-            command.apply(false).addCallback((v, throwable) -> {
+        unsafeStore.execute(this, safeStore -> {
+            AccordCommand command = (AccordCommand) safeStore.command(txnId);
+            command.apply(safeStore, false).addCallback((v, throwable) -> {
                 if (throwable != null)
                     promise.tryFailure(throwable);
                 else
@@ -634,59 +654,61 @@ public class AccordCommand extends Command implements AccordState<TxnId>
         return promise;
     }
 
-    private Future<Void> apply(boolean canReschedule)
+    private Future<Void> apply(SafeCommandStore safeStore, boolean canReschedule)
     {
-        Future<Void> future = cache().getWriteFuture(txnId);
+        AccordStateCache.Instance<TxnId, AccordCommand> cache = ((AccordCommandStore) safeStore).commandCache();
+        Future<Void> future = cache.getWriteFuture(txnId);
         if (future != null)
             return future;
 
         // this can be called via a listener callback, in which case we won't
         // have the appropriate commandsForKey in scope, so start a new operation
         // with the correct scope and notify the caller when that completes
-        if (!canApplyWithCurrentScope())
+        if (!canApplyWithCurrentScope(safeStore))
         {
             Preconditions.checkArgument(canReschedule);
-            return applyWithCorrectScope();
+            return applyWithCorrectScope(safeStore.commandStore());
         }
 
-        future = super.apply();
-        cache().setWriteFuture(txnId, future);
+        future = super.apply(safeStore);
+        cache.setWriteFuture(txnId, future);
         return future;
     }
 
     @Override
-    public Future<Void> apply()
+    public Future<Void> apply(SafeCommandStore safeStore)
     {
-        return apply(true);
+        return apply(safeStore, true);
     }
 
     @Override
-    public Future<Data> read(Keys scope)
+    public Future<Data> read(SafeCommandStore safeStore)
     {
-        ReadFuture future = cache().getReadFuture(txnId);
+        AccordStateCache.Instance<TxnId, AccordCommand> cache = ((AccordCommandStore) safeStore).commandCache();
+        Future<Data> future = cache.getReadFuture(txnId);
         if (future != null)
-            return future.scope.equals(scope) ? future : super.read(scope);
-        future = new ReadFuture(scope, super.read(scope));
-        cache().setReadFuture(txnId, future);
+            return future;
+        future = super.read(safeStore);
+        cache.setReadFuture(txnId, future);
         return future;
     }
 
-    private Listener maybeWrapListener(Listener listener)
+    private CommandListener maybeWrapListener(CommandListener listener)
     {
         if (listener.isTransient())
             return listener;
 
         if (listener instanceof AccordCommand)
-            return new ListenerProxy.CommandListenerProxy(commandStore, ((AccordCommand) listener).txnId());
+            return new ListenerProxy.CommandListenerProxy(((AccordCommand) listener).txnId());
 
         if (listener instanceof AccordCommandsForKey)
-            return new ListenerProxy.CommandsForKeyListenerProxy(commandStore, ((AccordCommandsForKey) listener).key());
+            return new ListenerProxy.CommandsForKeyListenerProxy(((AccordCommandsForKey) listener).key());
 
         throw new RuntimeException("Unhandled non-transient listener: " + listener);
     }
 
     @Override
-    public Command addListener(Listener listener)
+    public Command addListener(CommandListener listener)
     {
         listener = maybeWrapListener(listener);
         if (listener instanceof ListenerProxy)
@@ -697,7 +719,7 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     }
 
     @Override
-    public void removeListener(Listener listener)
+    public void removeListener(CommandListener listener)
     {
         listener = maybeWrapListener(listener);
         if (listener instanceof ListenerProxy)
@@ -708,35 +730,36 @@ public class AccordCommand extends Command implements AccordState<TxnId>
 
     public boolean hasListenerFor(TxnId txnId)
     {
-        return storedListeners.getView().contains(new ListenerProxy.CommandListenerProxy(commandStore, txnId));
+        return storedListeners.getView().contains(new ListenerProxy.CommandListenerProxy(txnId));
     }
 
     @Override
-    public void notifyListeners()
+    public void notifyListeners(SafeCommandStore safeStore)
     {
-        storedListeners.getView().forEach(this);
+        // TODO: efficiency (introduce BiConsumer method)
+        storedListeners.getView().forEach(l -> l.onChange(safeStore, this));
         transientListeners.forEach(listener -> {
             PreLoadContext ctx = listener.listenerPreLoadContext(txnId());
-            AsyncContext context = commandStore().getContext();
+            AsyncContext context = ((AccordCommandStore)safeStore).getContext();
             if (context.containsScopedItems(ctx))
             {
                 logger.trace("{}: synchronously updating listener {}", txnId(), listener);
-                listener.onChange(this);
+                listener.onChange(safeStore, this);
             }
             else
             {
                 logger.trace("{}: asynchronously updating listener {}", txnId(), listener);
-                commandStore().process(ctx, instance -> {
-                    listener.onChange(instance.command(txnId()));
+                safeStore.execute(ctx, reSafeStore -> {
+                    listener.onChange(reSafeStore, reSafeStore.command(txnId()));
                 });
             }
         });
     }
 
     @Override
-    public void addWaitingOnCommit(Command command)
+    public void addWaitingOnCommit(TxnId txnId)
     {
-        waitingOnCommit.blindPut(command.txnId(), AccordPartialCommand.serializer.serialize(command));
+        waitingOnCommit.blindAdd(txnId);
     }
 
     @Override
@@ -746,24 +769,23 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     }
 
     @Override
-    public void removeWaitingOnCommit(PartialCommand command)
+    public void removeWaitingOnCommit(TxnId txnId)
     {
-        waitingOnCommit.blindRemove(command.txnId());
+        waitingOnCommit.blindRemove(txnId);
     }
 
     @Override
-    public PartialCommand firstWaitingOnCommit()
+    public TxnId firstWaitingOnCommit()
     {
         if (!isWaitingOnCommit())
             return null;
-        ByteBuffer bytes = waitingOnCommit.getView().firstEntry().getValue();
-        return AccordPartialCommand.serializer.deserialize(commandStore, bytes);
+        return waitingOnCommit.getView().first();
     }
 
     @Override
-    public void addWaitingOnApplyIfAbsent(PartialCommand command)
+    public void addWaitingOnApplyIfAbsent(TxnId txnId, Timestamp executeAt)
     {
-        waitingOnApply.blindPut(command.txnId(), AccordPartialCommand.serializer.serialize(command));
+        waitingOnApply.blindPut(executeAt, txnId);
     }
 
     @Override
@@ -773,17 +795,17 @@ public class AccordCommand extends Command implements AccordState<TxnId>
     }
 
     @Override
-    public void removeWaitingOnApply(PartialCommand command)
+    public void removeWaitingOn(TxnId txnId, Timestamp executeAt)
     {
-        waitingOnApply.blindRemove(command.txnId());
+        waitingOnCommit.blindRemove(txnId);
+        waitingOnApply.blindRemove(executeAt, txnId);
     }
 
     @Override
-    public PartialCommand firstWaitingOnApply()
+    public TxnId firstWaitingOnApply()
     {
         if (!isWaitingOnApply())
             return null;
-        ByteBuffer bytes = waitingOnApply.getView().firstEntry().getValue();
-        return AccordPartialCommand.serializer.deserialize(commandStore, bytes);
+        return waitingOnApply.getView().firstEntry().getValue();
     }
 }
diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java
index ce6c8b0e68..227efdf738 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java
@@ -18,13 +18,13 @@
 
 package org.apache.cassandra.service.accord;
 
+import java.util.Comparator;
 import java.util.Objects;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 import java.util.function.Function;
-import java.util.function.LongSupplier;
 
 import com.google.common.base.Preconditions;
 
@@ -33,20 +33,23 @@ import accord.api.DataStore;
 import accord.api.Key;
 import accord.api.ProgressLog;
 import accord.local.Command;
+import accord.local.CommandListener;
 import accord.local.CommandStore;
 import accord.local.CommandsForKey;
+import accord.local.NodeTimeService;
 import accord.local.PreLoadContext;
+import accord.local.SafeCommandStore;
+import accord.primitives.Keys;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
 import org.apache.cassandra.service.accord.async.AsyncContext;
 import org.apache.cassandra.service.accord.async.AsyncOperation;
 import org.apache.cassandra.utils.Clock;
-import org.apache.cassandra.utils.concurrent.AsyncPromise;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
-public class AccordCommandStore extends CommandStore
+public class AccordCommandStore extends CommandStore implements SafeCommandStore
 {
     public static long maxCacheSize()
     {
@@ -78,25 +81,36 @@ public class AccordCommandStore extends CommandStore
     private AsyncContext currentCtx = null;
     private long lastSystemTimestampMicros = Long.MIN_VALUE;
 
-    public AccordCommandStore(int generation,
+    private final NodeTimeService time;
+    private final Agent agent;
+    private final DataStore dataStore;
+    private final ProgressLog progressLog;
+    private final RangesForEpoch rangesForEpoch;
+
+    public AccordCommandStore(int id,
+                              int generation,
                               int index,
                               int numShards,
-                              Function<Timestamp, Timestamp> uniqueNow,
-                              LongSupplier currentEpoch,
+                              NodeTimeService time,
                               Agent agent,
-                              DataStore store,
+                              DataStore dataStore,
                               ProgressLog.Factory progressLogFactory,
                               RangesForEpoch rangesForEpoch,
                               ExecutorService executor)
     {
-        super(generation, index, numShards, uniqueNow, currentEpoch, agent, store, progressLogFactory, rangesForEpoch);
+        super(id, generation, index, numShards);
+        this.time = time;
+        this.agent = agent;
+        this.dataStore = dataStore;
+        this.progressLog = progressLogFactory.create(this);
+        this.rangesForEpoch = rangesForEpoch;
         this.loggingId = String.format("[%s:%s]", generation, index);
         this.executor = executor;
-        this.threadId = getThreadId(executor);
+        this.threadId = getThreadId(this.executor);
         this.stateCache = new AccordStateCache(maxCacheSize() / numShards);
         this.commandCache = stateCache.instance(TxnId.class,
                                                 AccordCommand.class,
-                                                txnId -> new AccordCommand(this, txnId));
+                                                AccordCommand::new);
         this.commandsForKeyCache = stateCache.instance(PartitionKey.class,
                                                        AccordCommandsForKey.class,
                                                        key -> new AccordCommandsForKey(this, key));
@@ -185,6 +199,18 @@ public class AccordCommandStore extends CommandStore
         return !command.isEmpty() ? command : null;
     }
 
+    @Override
+    public Command ifLoaded(TxnId txnId)
+    {
+        AccordCommand command = commandCache.getOrNull(txnId);
+        if (command != null && command.isLoaded())
+        {
+            getContext().commands.add(command);
+            return command;
+        }
+        return null;
+    }
+
     public boolean isCommandsForKeyInContext(PartitionKey key)
     {
         return currentCtx.commandsForKey.get(key) != null;
@@ -219,39 +245,31 @@ public class AccordCommandStore extends CommandStore
     }
 
     @Override
-    public Future<Void> processSetup(Consumer<? super CommandStore> function)
-    {
-        AsyncPromise<Void> promise = new AsyncPromise<>();
-        executor.execute(() -> {
-            try
-            {
-                function.accept(this);
-                promise.trySuccess(null);
-            }
-            catch (Throwable t)
-            {
-                promise.tryFailure(t);
-            }
+    public void addAndInvokeListener(TxnId txnId, CommandListener listener)
+    {
+        AccordCommand.WriteOnly command = (AccordCommand.WriteOnly) getContext().commands.getOrCreateWriteOnly(txnId, (ignore, id) -> new AccordCommand.WriteOnly(id), this);
+        command.addListener(listener);
+        execute(listener.listenerPreLoadContext(txnId), store -> {
+            listener.onChange(store, store.command(txnId));
         });
-        return promise;
     }
 
     @Override
-    public <T> Future<T> processSetup(Function<? super CommandStore, T> function)
-    {
-        AsyncPromise<T> promise = new AsyncPromise<>();
-        executor.execute(() -> {
-            try
-            {
-                T result = function.apply(this);
-                promise.trySuccess(result);
-            }
-            catch (Throwable t)
-            {
-                promise.tryFailure(t);
-            }
-        });
-        return promise;
+    public CommandStore commandStore()
+    {
+        return this;
+    }
+
+    @Override
+    public DataStore dataStore()
+    {
+        return dataStore;
+    }
+
+    @Override
+    public Timestamp uniqueNow(Timestamp atLeast)
+    {
+        return time.uniqueNow(atLeast);
     }
 
     public void processBlocking(Runnable runnable)
@@ -271,7 +289,7 @@ public class AccordCommandStore extends CommandStore
     }
 
     @Override
-    public <T> Future<T> process(PreLoadContext loadCtx, Function<? super CommandStore, T> function)
+    public <T> Future<T> submit(PreLoadContext loadCtx, Function<? super SafeCommandStore, T> function)
     {
         AsyncOperation<T> operation = AsyncOperation.create(this, loadCtx, function);
         executor.execute(operation);
@@ -279,13 +297,50 @@ public class AccordCommandStore extends CommandStore
     }
 
     @Override
-    public Future<Void> process(PreLoadContext loadCtx, Consumer<? super CommandStore> consumer)
+    public Agent agent()
     {
-        AsyncOperation<Void> operation = AsyncOperation.create(this, loadCtx, consumer);
+        return agent;
+    }
+
+    @Override
+    public ProgressLog progressLog()
+    {
+        return progressLog;
+    }
+
+    @Override
+    public RangesForEpoch ranges()
+    {
+        return rangesForEpoch;
+    }
+
+    @Override
+    public long latestEpoch()
+    {
+        return time.epoch();
+    }
+
+    @Override
+    public Timestamp maxConflict(Keys keys)
+    {
+        // TODO: efficiency
+        return keys.stream()
+                   .map(this::maybeCommandsForKey)
+                   .filter(Objects::nonNull)
+                   .map(CommandsForKey::max)
+                   .max(Comparator.naturalOrder())
+                   .orElse(Timestamp.NONE);
+    }
+
+    @Override
+    public Future<Void> execute(PreLoadContext preLoadContext, Consumer<? super SafeCommandStore> consumer)
+    {
+        AsyncOperation<Void> operation = AsyncOperation.create(this, preLoadContext, consumer);
         executor.execute(operation);
         return operation;
     }
 
+
     @Override
     public void shutdown()
     {
diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java
index 3a1cbb8064..24b77bcc5c 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java
@@ -23,36 +23,45 @@ import java.util.concurrent.ExecutorService;
 import accord.api.Agent;
 import accord.api.DataStore;
 import accord.api.ProgressLog;
+import accord.local.AsyncCommandStores;
 import accord.local.CommandStore;
-import accord.local.CommandStores;
 import accord.local.Node;
+import accord.local.NodeTimeService;
 import org.apache.cassandra.concurrent.ExecutorFactory;
 import org.apache.cassandra.utils.ExecutorUtils;
 
-public class AccordCommandStores extends CommandStores
+public class AccordCommandStores extends AsyncCommandStores
 {
     private final ExecutorService[] executors;
 
     public AccordCommandStores(int numShards, Node node, Agent agent, DataStore store,
                                ProgressLog.Factory progressLogFactory)
     {
-        super(numShards, node, agent, store, progressLogFactory);
-        this.executors = new ExecutorService[numShards];
-        for (int i=0; i<numShards; i++)
-        {
-            executors[i] = ExecutorFactory.Global.executorFactory().sequential(CommandStore.class.getSimpleName() + '[' + node + ':' + i + ']');
-        }
+        this(numShards, node, agent, store, progressLogFactory, executors(node, numShards));
     }
 
-    @Override
-    protected CommandStore createCommandStore(int generation, int index, int numShards, Node node, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, CommandStore.RangesForEpoch rangesForEpoch)
+    private AccordCommandStores(int numShards, NodeTimeService time, Agent agent, DataStore store,
+                                ProgressLog.Factory progressLogFactory, ExecutorService[] executors)
+    {
+        super(numShards, time, agent, store, progressLogFactory,
+              (id, generation, index, numShards1, time1, agent1, store1, progressLogFactory1, rangesForEpoch)
+                -> new AccordCommandStore(id, generation, index, numShards1, time1, agent1, store1, progressLogFactory1, rangesForEpoch, executors[index]));
+        this.executors = executors;
+    }
+
+    private static ExecutorService[] executors(Node node, int count)
     {
-        return new AccordCommandStore(generation, index, numShards, node::uniqueNow, node.topology()::epoch, agent, store, progressLogFactory, rangesForEpoch, executors[index]);
+        ExecutorService[] executors = new ExecutorService[count];
+        for (int i=0; i<count; i++)
+        {
+            executors[i] = ExecutorFactory.Global.executorFactory().sequential(CommandStore.class.getSimpleName() + '[' + node + ':' + i + ']');
+        }
+        return executors;
     }
 
     void setCacheSize(long bytes)
     {
-        setup(commandStore -> ((AccordCommandStore) commandStore).setCacheSize(bytes));
+        forEach(commandStore -> ((AccordCommandStore) commandStore).setCacheSize(bytes));
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandsForKey.java b/src/java/org/apache/cassandra/service/accord/AccordCommandsForKey.java
index d2d84899fc..49941dbfb3 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordCommandsForKey.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordCommandsForKey.java
@@ -27,6 +27,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.stream.Stream;
 
+import javax.annotation.Nullable;
+
 import com.google.common.base.Preconditions;
 
 import org.slf4j.Logger;
@@ -35,9 +37,9 @@ import org.slf4j.LoggerFactory;
 import accord.local.Command;
 import accord.local.CommandStore;
 import accord.local.CommandsForKey;
-import accord.local.PartialCommand;
 import accord.local.Status;
 import accord.primitives.Timestamp;
+import accord.primitives.TxnId;
 import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
 import org.apache.cassandra.service.accord.store.StoredLong;
 import org.apache.cassandra.service.accord.store.StoredNavigableMap;
@@ -46,7 +48,12 @@ import org.apache.cassandra.service.accord.store.StoredValue;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.concurrent.Future;
+import org.assertj.core.util.VisibleForTesting;
 
+import static accord.local.CommandsForKey.CommandTimeseries.TestDep.ANY_DEPS;
+import static accord.local.CommandsForKey.CommandTimeseries.TestDep.WITHOUT;
+import static accord.local.CommandsForKey.CommandTimeseries.TestKind.RorWs;
+import static accord.primitives.Txn.Kind.WRITE;
 import static org.apache.cassandra.service.accord.AccordState.WriteOnly.applyMapChanges;
 import static org.apache.cassandra.service.accord.AccordState.WriteOnly.applySetChanges;
 
@@ -111,30 +118,23 @@ public class AccordCommandsForKey extends CommandsForKey implements AccordState<
         }
     }
 
-    public class Series implements CommandTimeseries
+    public class Series<T> implements CommandTimeseries<T>
     {
         public final SeriesKind kind;
         public final StoredNavigableMap<Timestamp, ByteBuffer> map;
+        private final Function<AccordPartialCommand, T> translate;
 
-        public Series(AccordState.Kind stateKind, SeriesKind kind)
+        public Series(ReadWrite readWrite, SeriesKind kind, Function<AccordPartialCommand, T> translate)
         {
             this.kind = kind;
-            map = new StoredNavigableMap<>(stateKind);
-        }
-
-        @Override
-        public PartialCommand.WithDeps get(Timestamp timestamp)
-        {
-            ByteBuffer bytes = map.getView().get(timestamp);
-            if (bytes == null)
-                return null;
-            return AccordPartialCommand.WithDeps.serializer.deserialize(commandStore, bytes);
+            map = new StoredNavigableMap<>(readWrite);
+            this.translate = translate;
         }
 
         @Override
         public void add(Timestamp timestamp, Command command)
         {
-            map.blindPut(timestamp, AccordPartialCommand.WithDeps.serializer.serialize(command));
+            map.blindPut(timestamp, AccordPartialCommand.serializer.serialize(new AccordPartialCommand(key, command)));
         }
 
         @Override
@@ -143,9 +143,9 @@ public class AccordCommandsForKey extends CommandsForKey implements AccordState<
             map.blindRemove(timestamp);
         }
 
-        private Stream<PartialCommand.WithDeps> idsToCommands(Collection<ByteBuffer> blobs)
+        private Stream<AccordPartialCommand> idsToCommands(Collection<ByteBuffer> blobs)
         {
-            return blobs.stream().map(blob -> AccordPartialCommand.WithDeps.serializer.deserialize(commandStore, blob));
+            return blobs.stream().map(blob -> AccordPartialCommand.serializer.deserialize(AccordCommandsForKey.this, commandStore, blob));
         }
 
         @Override
@@ -155,27 +155,37 @@ public class AccordCommandsForKey extends CommandsForKey implements AccordState<
         }
 
         @Override
-        public Stream<PartialCommand.WithDeps> before(Timestamp timestamp)
+        public Stream<T> before(Timestamp timestamp, TestKind testKind, TestDep testDep, @Nullable TxnId depId, TestStatus testStatus, @Nullable Status status)
         {
-            return idsToCommands(map.getView().headMap(timestamp, false).values());
+            return idsToCommands(map.getView().headMap(timestamp, false).values())
+                   .filter(cmd -> testKind == RorWs || cmd.kind() == WRITE)
+                   .filter(cmd -> testDep == ANY_DEPS || (cmd.hasDep(depId) ^ (testDep == WITHOUT)))
+                   .filter(cmd -> TestStatus.test(cmd.status(), testStatus, status))
+                   .map(translate);
         }
 
         @Override
-        public Stream<PartialCommand.WithDeps> after(Timestamp timestamp)
+        public Stream<T> after(Timestamp timestamp, TestKind testKind, TestDep testDep, @Nullable TxnId depId, TestStatus testStatus, @Nullable Status status)
         {
-            return idsToCommands(map.getView().tailMap(timestamp, false).values());
+            return idsToCommands(map.getView().tailMap(timestamp, false).values())
+                   .filter(cmd -> testKind == RorWs || cmd.kind() == WRITE)
+                   .filter(cmd -> testDep == ANY_DEPS || (cmd.hasDep(depId) ^ (testDep == WITHOUT)))
+                   .filter(cmd -> TestStatus.test(cmd.status(), testStatus, status))
+                   .map(translate);
         }
 
-        @Override
-        public Stream<PartialCommand.WithDeps> between(Timestamp min, Timestamp max)
+        @VisibleForTesting
+        public Stream<AccordPartialCommand> all()
         {
-            return idsToCommands(map.getView().subMap(min, true, max, true).values());
+            return idsToCommands(map.getView().values());
         }
 
-        @Override
-        public Stream<PartialCommand.WithDeps> all()
+        public AccordPartialCommand get(Timestamp timestamp)
         {
-            return idsToCommands(map.getView().values());
+            ByteBuffer blob = map.getView().get(timestamp);
+            if (blob == null)
+                return null;
+            return AccordPartialCommand.serializer.deserialize(AccordCommandsForKey.this, commandStore, blob);
         }
     }
 
@@ -186,22 +196,22 @@ public class AccordCommandsForKey extends CommandsForKey implements AccordState<
     public final StoredLong lastExecutedMicros;
     public final StoredValue<Timestamp> lastWriteTimestamp;
     public final StoredSet.Navigable<Timestamp> blindWitnessed;
-    public final Series uncommitted;
-    public final Series committedById;
-    public final Series committedByExecuteAt;
+    public final Series<TxnIdWithExecuteAt> uncommitted;
+    public final Series<TxnId> committedById;
+    public final Series<TxnId> committedByExecuteAt;
 
     public AccordCommandsForKey(AccordCommandStore commandStore, PartitionKey key)
     {
         this.commandStore = commandStore;
         this.key = key;
-        maxTimestamp = new StoredValue<>(kind());
-        lastExecutedTimestamp = new StoredValue<>(kind());
-        lastExecutedMicros = new StoredLong(kind());
-        lastWriteTimestamp = new StoredValue<>(kind());
-        blindWitnessed = new StoredSet.Navigable<>(kind());
-        uncommitted = new Series(kind(), SeriesKind.UNCOMMITTED);
-        committedById = new Series(kind(), SeriesKind.COMMITTED_BY_ID);
-        committedByExecuteAt = new Series(kind(), SeriesKind.COMMITTED_BY_EXECUTE_AT);
+        maxTimestamp = new StoredValue<>(rw());
+        lastExecutedTimestamp = new StoredValue<>(rw());
+        lastExecutedMicros = new StoredLong(rw());
+        lastWriteTimestamp = new StoredValue<>(rw());
+        blindWitnessed = new StoredSet.Navigable<>(rw());
+        uncommitted = new Series<>(rw(), SeriesKind.UNCOMMITTED, x -> x);
+        committedById = new Series<>(rw(), SeriesKind.COMMITTED_BY_ID, AccordPartialCommand::txnId);
+        committedByExecuteAt = new Series<>(rw(), SeriesKind.COMMITTED_BY_EXECUTE_AT, AccordPartialCommand::txnId);
     }
 
     @Override
@@ -308,19 +318,19 @@ public class AccordCommandsForKey extends CommandsForKey implements AccordState<
     }
 
     @Override
-    public CommandTimeseries uncommitted()
+    public Series<TxnIdWithExecuteAt> uncommitted()
     {
         return uncommitted;
     }
 
     @Override
-    public CommandTimeseries committedById()
+    public Series<TxnId> committedById()
     {
         return committedById;
     }
 
     @Override
-    public CommandTimeseries committedByExecuteAt()
+    public Series<TxnId> committedByExecuteAt()
     {
         return committedByExecuteAt;
     }
@@ -359,18 +369,18 @@ public class AccordCommandsForKey extends CommandsForKey implements AccordState<
 
     public void updateSummaries(AccordCommand command)
     {
-        if (command.status.get().hasBeen(Status.Committed))
+        if (command.status().hasBeen(Status.Committed))
         {
-            if (command.status.previous() == null || !command.status.previous().hasBeen(Status.Committed))
+            if (command.status.previous() == null || !command.status.previous().status.hasBeen(Status.Committed))
                 uncommitted.map.blindRemove(command.txnId());
 
-            ByteBuffer bb = AccordPartialCommand.WithDeps.serializer.serialize(command);
-            committedById.map.blindPut(command.txnId(), bb);
-            committedByExecuteAt.map.blindPut(command.executeAt(), bb);
+            ByteBuffer partialCommand = AccordPartialCommand.serializer.serialize(new AccordPartialCommand(key, command));
+            committedById.map.blindPut(command.txnId(), partialCommand);
+            committedByExecuteAt.map.blindPut(command.executeAt(), partialCommand);
         }
         else
-        {
-            uncommitted.map.blindPut(command.txnId(), AccordPartialCommand.WithDeps.serializer.serialize(command));
+        {   // TODO: somebody is inserting large buffers into this map (presumably from loading from disk)
+            uncommitted.map.blindPut(command.txnId(), AccordPartialCommand.serializer.serialize(new AccordPartialCommand(key, command)));
         }
     }
 
@@ -450,10 +460,6 @@ public class AccordCommandsForKey extends CommandsForKey implements AccordState<
                ", lastExecutedTimestamp=" + lastExecutedTimestamp +
                ", lastExecutedMicros=" + lastExecutedMicros +
                ", lastWriteTimestamp=" + lastWriteTimestamp +
-//               ", blindWitnessed=" + blindWitnessed +
-//               ", uncommitted=" + uncommitted.map +
-//               ", committedById=" + committedById.map +
-//               ", committedByExecuteAt=" + committedByExecuteAt.map +
                '}';
     }
 }
diff --git a/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java b/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java
index d37f39a32c..e17e7c01e8 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java
@@ -39,13 +39,15 @@ import org.slf4j.LoggerFactory;
 
 import accord.local.CommandStore;
 import accord.local.Node;
+import accord.local.SaveStatus;
 import accord.local.Status;
+import accord.primitives.AbstractRoute;
 import accord.primitives.Ballot;
-import accord.primitives.Deps;
+import accord.primitives.PartialDeps;
+import accord.primitives.PartialTxn;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
-import accord.txn.Writes;
+import accord.primitives.Writes;
 import accord.utils.DeterministicIdentitySet;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -96,8 +98,11 @@ import org.apache.cassandra.serializers.UUIDSerializer;
 import org.apache.cassandra.service.accord.AccordCommandsForKey.SeriesKind;
 import org.apache.cassandra.service.accord.api.AccordKey;
 import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey;
 import org.apache.cassandra.service.accord.db.AccordData;
 import org.apache.cassandra.service.accord.serializers.CommandSerializers;
+import org.apache.cassandra.service.accord.serializers.DepsSerializer;
+import org.apache.cassandra.service.accord.serializers.KeySerializers;
 import org.apache.cassandra.service.accord.store.StoredNavigableMap;
 import org.apache.cassandra.service.accord.store.StoredSet;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -134,7 +139,8 @@ public class AccordKeyspace
               + "status int,"
               + "home_key blob,"
               + "progress_key blob,"
-              + "is_globally_persistent boolean,"
+              + "route blob,"
+              + "durability int,"
               + "txn blob,"
               + format("execute_at %s,", TIMESTAMP_TUPLE)
               + format("promised_ballot %s,", TIMESTAMP_TUPLE)
@@ -142,7 +148,7 @@ public class AccordKeyspace
               + "dependencies blob,"
               + "writes blob,"
               + "result blob,"
-              + format("waiting_on_commit map<%s, blob>,", TIMESTAMP_TUPLE)
+              + format("waiting_on_commit set<%s>,", TIMESTAMP_TUPLE)
               + format("waiting_on_apply map<%s, blob>,", TIMESTAMP_TUPLE)
               + "listeners set<blob>, "
               + format("blocking_commit_on set<%s>, ", TIMESTAMP_TUPLE)
@@ -150,13 +156,15 @@ public class AccordKeyspace
               + "PRIMARY KEY((store_generation, store_index, txn_id))"
               + ')');
 
+    // TODO: naming is not very clearly distinct from the base serializers
     private static class CommandsSerializers
     {
-        static final LocalVersionedSerializer<AccordKey> ACCORD_KEY_SERIALIZER = localSerializer(AccordKey.serializer);
-        static final LocalVersionedSerializer<Txn> TXN_SERIALIZER = localSerializer(CommandSerializers.txn);
-        static final LocalVersionedSerializer<Deps> DEPS_SERIALIZER = localSerializer(CommandSerializers.deps);
-        static final LocalVersionedSerializer<Writes> WRITES_SERIALIZER = localSerializer(CommandSerializers.writes);
-        static final LocalVersionedSerializer<AccordData> RESULT_DATA_SERIALIZER = localSerializer(AccordData.serializer);
+        static final LocalVersionedSerializer<AbstractRoute> abstractRoute = localSerializer(KeySerializers.abstractRoute);
+        static final LocalVersionedSerializer<AccordRoutingKey> routingKey = localSerializer(AccordRoutingKey.serializer);
+        static final LocalVersionedSerializer<PartialTxn> partialTxn = localSerializer(CommandSerializers.partialTxn);
+        static final LocalVersionedSerializer<PartialDeps> partialDeps = localSerializer(DepsSerializer.partialDeps);
+        static final LocalVersionedSerializer<Writes> writes = localSerializer(CommandSerializers.writes);
+        static final LocalVersionedSerializer<AccordData> result = localSerializer(AccordData.serializer);
 
         private static <T> LocalVersionedSerializer<T> localSerializer(IVersionedSerializer<T> serializer)
         {
@@ -178,7 +186,8 @@ public class AccordKeyspace
         static final ColumnMetadata status = getColumn(Commands, "status");
         static final ColumnMetadata home_key = getColumn(Commands, "home_key");
         static final ColumnMetadata progress_key = getColumn(Commands, "progress_key");
-        static final ColumnMetadata is_globally_persistent = getColumn(Commands, "is_globally_persistent");
+        static final ColumnMetadata route = getColumn(Commands, "route");
+        static final ColumnMetadata durability = getColumn(Commands, "durability");
         static final ColumnMetadata txn = getColumn(Commands, "txn");
         static final ColumnMetadata execute_at = getColumn(Commands, "execute_at");
         static final ColumnMetadata promised_ballot = getColumn(Commands, "promised_ballot");
@@ -312,22 +321,17 @@ public class AccordKeyspace
         return bytes != null && ! ByteBufferAccessor.instance.isEmpty(bytes) ? deserialize(bytes, serializer) : null;
     }
 
-    private static NavigableMap<TxnId, ByteBuffer> deserializeWaitingOn(Map<ByteBuffer, ByteBuffer> serialized)
+    private static NavigableMap<Timestamp, TxnId> deserializeWaitingOnApply(Map<ByteBuffer, ByteBuffer> serialized)
     {
         if (serialized == null || serialized.isEmpty())
             return new TreeMap<>();
 
-        NavigableMap<TxnId, ByteBuffer> result = new TreeMap<>();
+        NavigableMap<Timestamp, TxnId> result = new TreeMap<>();
         for (Map.Entry<ByteBuffer, ByteBuffer> entry : serialized.entrySet())
-            result.put(deserializeTimestampOrNull(entry.getKey(), TxnId::new), entry.getValue());
+            result.put(deserializeTimestampOrNull(entry.getKey(), Timestamp::new), deserializeTimestampOrNull(entry.getValue(), TxnId::new));
         return result;
     }
 
-    private static NavigableMap<TxnId, ByteBuffer> deserializeWaitingOn(UntypedResultSet.Row row, String name)
-    {
-        return deserializeWaitingOn(row.getMap(name, BytesType.instance, BytesType.instance));
-    }
-
     private static <T extends Timestamp, S extends Set<T>> S deserializeTimestampSet(Set<ByteBuffer> serialized, Supplier<S> setFactory, TimestampFactory<T> timestampFactory)
     {
         S result = setFactory.get();
@@ -340,26 +344,26 @@ public class AccordKeyspace
         return result;
     }
 
-    private static NavigableSet<TxnId> deserializeBlocking(UntypedResultSet.Row row, String name)
+    private static NavigableSet<TxnId> deserializeTxnIdNavigableSet(UntypedResultSet.Row row, String name)
     {
         return deserializeTimestampSet(row.getSet(name, BytesType.instance), TreeSet::new, TxnId::new);
     }
 
-    private static DeterministicIdentitySet<ListenerProxy> deserializeListeners(CommandStore commandStore, Set<ByteBuffer> serialized) throws IOException
+    private static DeterministicIdentitySet<ListenerProxy> deserializeListeners(Set<ByteBuffer> serialized) throws IOException
     {
         if (serialized == null || serialized.isEmpty())
             return new DeterministicIdentitySet<>();
         DeterministicIdentitySet<ListenerProxy> result = new DeterministicIdentitySet<>();
         for (ByteBuffer bytes : serialized)
         {
-            result.add(ListenerProxy.deserialize(commandStore, bytes, ByteBufferAccessor.instance, 0));
+            result.add(ListenerProxy.deserialize(bytes, ByteBufferAccessor.instance, 0));
         }
         return result;
     }
 
-    private static DeterministicIdentitySet<ListenerProxy> deserializeListeners(CommandStore commandStore, UntypedResultSet.Row row, String name) throws IOException
+    private static DeterministicIdentitySet<ListenerProxy> deserializeListeners(UntypedResultSet.Row row, String name) throws IOException
     {
-        return deserializeListeners(commandStore, row.getSet(name, BytesType.instance));
+        return deserializeListeners(row.getSet(name, BytesType.instance));
     }
 
     private static <K extends Comparable<?>, V> void addStoredMapChanges(Row.Builder builder,
@@ -411,7 +415,7 @@ public class AccordKeyspace
             map.forEachDeletion(k -> builder.addCell(tombstone(column, timestamp, nowInSec, CellPath.create(serialize.apply(k)))));
     }
 
-    public static Mutation getCommandMutation(AccordCommand command, long timestampMicros)
+    public static Mutation getCommandMutation(AccordCommandStore commandStore, AccordCommand command, long timestampMicros)
     {
         try
         {
@@ -429,16 +433,19 @@ public class AccordKeyspace
                 builder.addCell(live(CommandsColumns.status, timestampMicros, accessor.valueOf(command.status.get().ordinal())));
 
             if (command.homeKey.hasModifications())
-                builder.addCell(live(CommandsColumns.home_key, timestampMicros, serializeOrNull((AccordKey) command.homeKey.get(), CommandsSerializers.ACCORD_KEY_SERIALIZER)));
+                builder.addCell(live(CommandsColumns.home_key, timestampMicros, serializeOrNull((AccordRoutingKey) command.homeKey.get(), CommandsSerializers.routingKey)));
 
             if (command.progressKey.hasModifications())
-                builder.addCell(live(CommandsColumns.progress_key, timestampMicros, serializeOrNull((AccordKey) command.progressKey.get(), CommandsSerializers.ACCORD_KEY_SERIALIZER)));
+                builder.addCell(live(CommandsColumns.progress_key, timestampMicros, serializeOrNull((AccordRoutingKey) command.progressKey.get(), CommandsSerializers.routingKey)));
 
-            if (command.isGloballyPersistent.hasModifications())
-                builder.addCell(live(CommandsColumns.is_globally_persistent, timestampMicros, accessor.valueOf(command.isGloballyPersistent.get())));
+            if (command.route.hasModifications())
+                builder.addCell(live(CommandsColumns.route, timestampMicros, serializeOrNull(command.route.get(), CommandsSerializers.abstractRoute)));
 
-            if (command.txn.hasModifications())
-                builder.addCell(live(CommandsColumns.txn, timestampMicros, serializeOrNull(command.txn.get(), CommandsSerializers.TXN_SERIALIZER)));
+            if (command.durability.hasModifications())
+                builder.addCell(live(CommandsColumns.durability, timestampMicros, accessor.valueOf(command.durability.get().ordinal())));
+
+            if (command.partialTxn.hasModifications())
+                builder.addCell(live(CommandsColumns.txn, timestampMicros, serializeOrNull(command.partialTxn.get(), CommandsSerializers.partialTxn)));
 
             if (command.executeAt.hasModifications())
                 builder.addCell(live(CommandsColumns.execute_at, timestampMicros, serializeTimestamp(command.executeAt.get())));
@@ -449,20 +456,20 @@ public class AccordKeyspace
             if (command.accepted.hasModifications())
                 builder.addCell(live(CommandsColumns.accepted_ballot, timestampMicros, serializeTimestamp(command.accepted.get())));
 
-            if (command.deps.hasModifications())
-                builder.addCell(live(CommandsColumns.dependencies, timestampMicros, serialize(command.deps.get(), CommandsSerializers.DEPS_SERIALIZER)));
+            if (command.partialDeps.hasModifications())
+                builder.addCell(live(CommandsColumns.dependencies, timestampMicros, serialize(command.partialDeps.get(), CommandsSerializers.partialDeps)));
 
             if (command.writes.hasModifications())
-                builder.addCell(live(CommandsColumns.writes, timestampMicros, serialize(command.writes.get(), CommandsSerializers.WRITES_SERIALIZER)));
+                builder.addCell(live(CommandsColumns.writes, timestampMicros, serialize(command.writes.get(), CommandsSerializers.writes)));
 
             if (command.result.hasModifications())
-                builder.addCell(live(CommandsColumns.result, timestampMicros, serialize((AccordData) command.result.get(), CommandsSerializers.RESULT_DATA_SERIALIZER)));
+                builder.addCell(live(CommandsColumns.result, timestampMicros, serialize((AccordData) command.result.get(), CommandsSerializers.result)));
 
             if (command.waitingOnCommit.hasModifications())
             {
-                addStoredMapChanges(builder, CommandsColumns.waiting_on_commit,
+                addStoredSetChanges(builder, CommandsColumns.waiting_on_commit,
                                     timestampMicros, nowInSeconds, command.waitingOnCommit,
-                                    AccordKeyspace::serializeTimestamp, bytes -> bytes);
+                                    AccordKeyspace::serializeTimestamp);
             }
 
             if (command.blockingCommitOn.hasModifications())
@@ -476,7 +483,7 @@ public class AccordKeyspace
             {
                 addStoredMapChanges(builder, CommandsColumns.waiting_on_apply,
                                     timestampMicros, nowInSeconds, command.waitingOnApply,
-                                    AccordKeyspace::serializeTimestamp, bytes -> bytes);
+                                    AccordKeyspace::serializeTimestamp, AccordKeyspace::serializeTimestamp);
             }
 
             if (command.blockingApplyOn.hasModifications())
@@ -492,8 +499,8 @@ public class AccordKeyspace
                                     timestampMicros, nowInSeconds, command.storedListeners,
                                     ListenerProxy::identifier);
             }
-            ByteBuffer key = CommandsColumns.keyComparator.make(command.commandStore().generation(),
-                                                                command.commandStore().index(),
+            ByteBuffer key = CommandsColumns.keyComparator.make(commandStore.generation(),
+                                                                commandStore.index(),
                                                                 serializeTimestamp(command.txnId())).serializeAsPartitionKey();
             PartitionUpdate update = PartitionUpdate.singleRowUpdate(Commands, key, builder.build());
             return new Mutation(update);
@@ -535,8 +542,8 @@ public class AccordKeyspace
 
     public static AccordCommand loadCommand(AccordCommandStore commandStore, TxnId txnId)
     {
-        AccordCommand command = new AccordCommand(commandStore, txnId);
-        loadCommand(command);
+        AccordCommand command = new AccordCommand(txnId);
+        loadCommand(commandStore, command);
         return command;
     }
 
@@ -561,11 +568,10 @@ public class AccordKeyspace
                                    txnId.epoch, txnId.real, txnId.logical, txnId.node.id);
     }
 
-    public static void loadCommand(AccordCommand command)
+    public static void loadCommand(AccordCommandStore commandStore, AccordCommand command)
     {
         Preconditions.checkArgument(!command.isLoaded());
         TxnId txnId = command.txnId();
-        AccordCommandStore commandStore = command.commandStore();
         commandStore.checkNotInStoreThread();
 
         UntypedResultSet result = loadCommandRow(commandStore, command.txnId());
@@ -580,22 +586,24 @@ public class AccordKeyspace
         {
             UntypedResultSet.Row row = result.one();
             Preconditions.checkState(deserializeTimestampOrNull(row, "txn_id", TxnId::new).equals(txnId));
-            command.status.load(Status.values()[row.getInt("status")]);
-            command.homeKey.load(deserializeOrNull(row.getBlob("home_key"), CommandsSerializers.ACCORD_KEY_SERIALIZER));
-            command.progressKey.load(deserializeOrNull(row.getBlob("progress_key"), CommandsSerializers.ACCORD_KEY_SERIALIZER));
-            command.isGloballyPersistent.load(row.getBoolean("is_globally_persistent"));
-            command.txn.load(deserializeOrNull(row.getBlob("txn"), CommandsSerializers.TXN_SERIALIZER));
+            command.status.load(SaveStatus.values()[row.getInt("status")]);
+            command.homeKey.load(deserializeOrNull(row.getBlob("home_key"), CommandsSerializers.routingKey));
+            command.progressKey.load(deserializeOrNull(row.getBlob("progress_key"), CommandsSerializers.routingKey));
+            command.route.load(deserializeOrNull(row.getBlob("route"), CommandsSerializers.abstractRoute));
+            // TODO: something less brittle than ordinal, more efficient than values()
+            command.durability.load(Status.Durability.values()[row.getInt("durability", 0)]);
+            command.partialTxn.load(deserializeOrNull(row.getBlob("txn"), CommandsSerializers.partialTxn));
             command.executeAt.load(deserializeTimestampOrNull(row, "execute_at", Timestamp::new));
             command.promised.load(deserializeTimestampOrNull(row, "promised_ballot", Ballot::new));
             command.accepted.load(deserializeTimestampOrNull(row, "accepted_ballot", Ballot::new));
-            command.deps.load(deserializeWithVersionOr(row, "dependencies", CommandsSerializers.DEPS_SERIALIZER, () -> Deps.NONE));
-            command.writes.load(deserializeWithVersionOr(row, "writes", CommandsSerializers.WRITES_SERIALIZER, () -> null));
-            command.result.load(deserializeWithVersionOr(row, "result", CommandsSerializers.RESULT_DATA_SERIALIZER, () -> null));
-            command.waitingOnCommit.load(deserializeWaitingOn(row, "waiting_on_commit"));
-            command.blockingCommitOn.load(deserializeBlocking(row, "blocking_commit_on"));
-            command.waitingOnApply.load(deserializeWaitingOn(row, "waiting_on_apply"));
-            command.blockingApplyOn.load(deserializeBlocking(row, "blocking_apply_on"));
-            command.storedListeners.load(deserializeListeners(commandStore, row, "listeners"));
+            command.partialDeps.load(deserializeWithVersionOr(row, "dependencies", CommandsSerializers.partialDeps, () -> PartialDeps.NONE));
+            command.writes.load(deserializeWithVersionOr(row, "writes", CommandsSerializers.writes, () -> null));
+            command.result.load(deserializeWithVersionOr(row, "result", CommandsSerializers.result, () -> null));
+            command.waitingOnCommit.load(deserializeTxnIdNavigableSet(row, "waiting_on_commit"));
+            command.blockingCommitOn.load(deserializeTxnIdNavigableSet(row, "blocking_commit_on"));
+            command.waitingOnApply.load(deserializeWaitingOnApply(row.getMap("waiting_on_apply", BytesType.instance, BytesType.instance)));
+            command.blockingApplyOn.load(deserializeTxnIdNavigableSet(row, "blocking_apply_on"));
+            command.storedListeners.load(deserializeListeners(row, "listeners"));
         }
         catch (IOException e)
         {
@@ -609,7 +617,7 @@ public class AccordKeyspace
         }
     }
 
-    private static void addSeriesMutations(AccordCommandsForKey.Series series,
+    private static void addSeriesMutations(AccordCommandsForKey.Series<?> series,
                                            PartitionUpdate.Builder partitionBuilder,
                                            Row.Builder rowBuilder,
                                            long timestampMicros,
@@ -647,7 +655,7 @@ public class AccordKeyspace
         return makeKey(cfk.commandStore(), cfk.key());
     }
 
-    public static Mutation getCommandsForKeyMutation(AccordCommandsForKey cfk, long timestampMicros)
+    public static Mutation getCommandsForKeyMutation(AccordCommandStore commandStore, AccordCommandsForKey cfk, long timestampMicros)
     {
         Preconditions.checkArgument(cfk.hasModifications());
 
@@ -667,29 +675,31 @@ public class AccordKeyspace
         boolean updateStaticRow = cfk.maxTimestamp.hasModifications()
                                   || cfk.lastExecutedTimestamp.hasModifications()
                                   || cfk.lastExecutedMicros.hasModifications()
+                                  || cfk.lastWriteTimestamp.hasModifications()
                                   || cfk.blindWitnessed.hasModifications();
         if (updateStaticRow)
+        {
             rowBuilder.newRow(Clustering.STATIC_CLUSTERING);
 
-        if (cfk.maxTimestamp.hasModifications())
-            rowBuilder.addCell(live(CommandsForKeyColumns.max_timestamp, timestampMicros, serializeTimestamp(cfk.maxTimestamp.get())));
+            if (cfk.maxTimestamp.hasModifications())
+                rowBuilder.addCell(live(CommandsForKeyColumns.max_timestamp, timestampMicros, serializeTimestamp(cfk.maxTimestamp.get())));
 
-        if (cfk.lastExecutedTimestamp.hasModifications())
-            rowBuilder.addCell(live(CommandsForKeyColumns.last_executed_timestamp, timestampMicros, serializeTimestamp(cfk.lastExecutedTimestamp.get())));
+            if (cfk.lastExecutedTimestamp.hasModifications())
+                rowBuilder.addCell(live(CommandsForKeyColumns.last_executed_timestamp, timestampMicros, serializeTimestamp(cfk.lastExecutedTimestamp.get())));
 
-        if (cfk.lastExecutedMicros.hasModifications())
-            rowBuilder.addCell(live(CommandsForKeyColumns.last_executed_micros, timestampMicros, ByteBufferUtil.bytes(cfk.lastExecutedMicros.get())));
+            if (cfk.lastExecutedMicros.hasModifications())
+                rowBuilder.addCell(live(CommandsForKeyColumns.last_executed_micros, timestampMicros, ByteBufferUtil.bytes(cfk.lastExecutedMicros.get())));
 
-        if (cfk.lastWriteTimestamp.hasModifications())
-            rowBuilder.addCell(live(CommandsForKeyColumns.last_write_timestamp, timestampMicros, serializeTimestamp(cfk.lastWriteTimestamp.get())));
+            if (cfk.lastWriteTimestamp.hasModifications())
+                rowBuilder.addCell(live(CommandsForKeyColumns.last_write_timestamp, timestampMicros, serializeTimestamp(cfk.lastWriteTimestamp.get())));
 
-        if (cfk.blindWitnessed.hasModifications())
-            addStoredSetChanges(rowBuilder, CommandsForKeyColumns.blind_witnessed,
-                                timestampMicros, nowInSeconds, cfk.blindWitnessed,
-                                AccordKeyspace::serializeTimestamp);
+            if (cfk.blindWitnessed.hasModifications())
+                addStoredSetChanges(rowBuilder, CommandsForKeyColumns.blind_witnessed,
+                                    timestampMicros, nowInSeconds, cfk.blindWitnessed,
+                                    AccordKeyspace::serializeTimestamp);
 
-        if (updateStaticRow)
             partitionBuilder.add(rowBuilder.build());
+        }
 
         addSeriesMutations(cfk.uncommitted, partitionBuilder, rowBuilder, timestampMicros, nowInSeconds);
         addSeriesMutations(cfk.committedById, partitionBuilder, rowBuilder, timestampMicros, nowInSeconds);
diff --git a/src/java/org/apache/cassandra/service/accord/AccordMessageSink.java b/src/java/org/apache/cassandra/service/accord/AccordMessageSink.java
index 32c136b1b7..dc329e8807 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordMessageSink.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordMessageSink.java
@@ -62,8 +62,6 @@ public class AccordMessageSink implements MessageSink
             mapping.put(MessageType.COMMIT_INVALIDATE,      Verb.ACCORD_COMMIT_INVALIDATE_REQ);
             mapping.put(MessageType.APPLY_REQ,              Verb.ACCORD_APPLY_REQ);
             mapping.put(MessageType.APPLY_RSP,              Verb.ACCORD_APPLY_RSP);
-            mapping.put(MessageType.APPLY_AND_CHECK_REQ,    Verb.ACCORD_APPLY_AND_CHECK_REQ);
-            mapping.put(MessageType.APPLY_AND_CHECK_RSP,    Verb.ACCORD_APPLY_AND_CHECK_RSP);
             mapping.put(MessageType.READ_REQ,               Verb.ACCORD_READ_REQ);
             mapping.put(MessageType.READ_RSP,               Verb.ACCORD_READ_RSP);
             mapping.put(MessageType.BEGIN_RECOVER_REQ,      Verb.ACCORD_RECOVER_REQ);
@@ -72,12 +70,20 @@ public class AccordMessageSink implements MessageSink
             mapping.put(MessageType.BEGIN_INVALIDATE_RSP,   Verb.ACCORD_BEGIN_INVALIDATE_RSP);
             mapping.put(MessageType.WAIT_ON_COMMIT_REQ,     Verb.ACCORD_WAIT_COMMIT_REQ);
             mapping.put(MessageType.WAIT_ON_COMMIT_RSP,     Verb.ACCORD_WAIT_COMMIT_RSP);
-            mapping.put(MessageType.INFORM_REQ,             Verb.ACCORD_INFORM_OF_TXN_REQ);
-            mapping.put(MessageType.INFORM_RSP,             Verb.ACCORD_INFORM_OF_TXN_RSP);
-            mapping.put(MessageType.INFORM_PERSISTED_REQ,   Verb.ACCORD_INFORM_OF_PERSIETENCE_REQ);
-            mapping.put(MessageType.INFORM_PERSISTED_RSP,   Verb.ACCORD_INFORM_OF_PERSISTENCE_RSP);
+            mapping.put(MessageType.INFORM_TXNID_REQ,       Verb.ACCORD_INFORM_OF_TXNID_REQ);
+            mapping.put(MessageType.INFORM_HOME_DURABLE_REQ,Verb.ACCORD_INFORM_HOME_DURABLE_REQ);
+            mapping.put(MessageType.INFORM_DURABLE_REQ,     Verb.ACCORD_INFORM_DURABLE_REQ);
             mapping.put(MessageType.CHECK_STATUS_REQ,       Verb.ACCORD_CHECK_STATUS_REQ);
             mapping.put(MessageType.CHECK_STATUS_RSP,       Verb.ACCORD_CHECK_STATUS_RSP);
+            mapping.put(MessageType.GET_DEPS_REQ,           Verb.ACCORD_GET_DEPS_REQ);
+            mapping.put(MessageType.GET_DEPS_RSP,           Verb.ACCORD_GET_DEPS_RSP);
+            mapping.put(MessageType.SIMPLE_RSP,             Verb.ACCORD_SIMPLE_RSP);
+
+            for (MessageType type : MessageType.values())
+            {
+                if (!mapping.containsKey(type))
+                    throw new AssertionError("Missing mapping for Accord MessageType " + type);
+            }
         }
     }
 
diff --git a/src/java/org/apache/cassandra/service/accord/AccordObjectSizes.java b/src/java/org/apache/cassandra/service/accord/AccordObjectSizes.java
index e4ea7049b7..85da64ae8c 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordObjectSizes.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordObjectSizes.java
@@ -21,14 +21,24 @@ package org.apache.cassandra.service.accord;
 import java.util.Map;
 
 import accord.api.Key;
+import accord.api.RoutingKey;
 import accord.local.Node;
+import accord.primitives.AbstractRoute;
 import accord.primitives.Deps;
+import accord.primitives.KeyRange;
+import accord.primitives.KeyRanges;
 import accord.primitives.Keys;
+import accord.primitives.PartialRoute;
+import accord.primitives.PartialTxn;
+import accord.primitives.Route;
+import accord.primitives.RoutingKeys;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
-import accord.txn.Writes;
+import accord.primitives.Writes;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.accord.api.AccordKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey.TokenKey;
 import org.apache.cassandra.service.accord.db.AccordQuery;
 import org.apache.cassandra.service.accord.db.AccordRead;
 import org.apache.cassandra.service.accord.db.AccordUpdate;
@@ -42,6 +52,28 @@ public class AccordObjectSizes
         return ((AccordKey.PartitionKey) key).estimatedSizeOnHeap();
     }
 
+    public static long key(RoutingKey key)
+    {
+        return ((AccordRoutingKey) key).estimatedSizeOnHeap();
+    }
+
+    private static final long EMPTY_KEY_RANGE_SIZE = ObjectSizes.measure(TokenRange.fullRange(TableId.generate()));
+    public static long range(KeyRange range)
+    {
+        return EMPTY_KEY_RANGE_SIZE + key(range.start()) + key(range.end());
+    }
+
+    private static final long EMPTY_KEY_RANGES_SIZE = ObjectSizes.measure(KeyRanges.of());
+    public static long ranges(KeyRanges ranges)
+    {
+        long size = EMPTY_KEY_RANGES_SIZE;
+        size += ObjectSizes.sizeOfReferenceArray(ranges.size());
+        // TODO: many ranges are fixed size, can compute by multiplication
+        for (int i = 0, mi = ranges.size() ; i < mi ; i++)
+            size += range(ranges.get(i));
+        return size;
+    }
+
     private static final long EMPTY_KEYS_SIZE = ObjectSizes.measure(Keys.of());
     public static long keys(Keys keys)
     {
@@ -52,15 +84,54 @@ public class AccordObjectSizes
         return size;
     }
 
-    private static final long EMPTY_TXN = ObjectSizes.measure(new Txn.InMemory(null, null, null));
-    public static long txn(Txn txn)
+    private static long routingKeysOnly(RoutingKeys keys)
+    {
+        // TODO: many routing keys are fixed size, can compute by multiplication
+        long size = ObjectSizes.sizeOfReferenceArray(keys.size());
+        for (int i=0, mi=keys.size(); i<mi; i++)
+            size += key(keys.get(i));
+        return size;
+    }
+
+    private static final long EMPTY_ROUTING_KEYS_SIZE = ObjectSizes.measure(RoutingKeys.of());
+    public static long routingKeys(RoutingKeys keys)
+    {
+        return routingKeysOnly(keys) + EMPTY_ROUTING_KEYS_SIZE;
+    }
+
+    private static final long EMPTY_ROUTE_SIZE = ObjectSizes.measure(new Route(new TokenKey(null, null), new RoutingKey[0]));
+    public static long route(Route route)
+    {
+        return EMPTY_ROUTE_SIZE
+               + routingKeysOnly(route)
+               + key(route.homeKey); // TODO: we will probably dedup homeKey, serializer dependent, but perhaps this is an acceptable error
+    }
+
+    private static final long EMPTY_PARTIAL_ROUTE_KEYS_SIZE = ObjectSizes.measure(new PartialRoute(KeyRanges.EMPTY, new TokenKey(null, null), new RoutingKey[0]));
+    public static long route(PartialRoute route)
+    {
+        return EMPTY_PARTIAL_ROUTE_KEYS_SIZE
+               + routingKeysOnly(route)
+               + ranges(route.covering)
+               + key(route.homeKey);
+    }
+
+    public static long route(AbstractRoute route)
+    {
+        if (route instanceof Route) return route((Route) route);
+        else return route((PartialRoute) route);
+    }
+
+    private static final long EMPTY_TXN = ObjectSizes.measure(new PartialTxn.InMemory(null, null, null, null, null, null));
+    public static long txn(PartialTxn txn)
     {
         long size = EMPTY_TXN;
         size += keys(txn.keys());
         size += ((AccordRead) txn.read()).estimatedSizeOnHeap();
         if (txn.update() != null)
             size += ((AccordUpdate) txn.update()).estimatedSizeOnHeap();
-        size += ((AccordQuery) txn.query()).estimatedSizeOnHeap();
+        if (txn.query() != null)
+            size += ((AccordQuery) txn.query()).estimatedSizeOnHeap();
         return size;
     }
 
diff --git a/src/java/org/apache/cassandra/service/accord/AccordPartialCommand.java b/src/java/org/apache/cassandra/service/accord/AccordPartialCommand.java
index c2c3010e43..e0384489c2 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordPartialCommand.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordPartialCommand.java
@@ -21,222 +21,108 @@ package org.apache.cassandra.service.accord;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
-import java.util.function.Consumer;
 
-import accord.local.Listener;
-import accord.local.PartialCommand;
+import accord.api.Key;
+import accord.local.Command;
+import accord.local.CommandsForKey;
 import accord.local.Status;
-import accord.primitives.Deps;
 import accord.primitives.Timestamp;
+import accord.primitives.Txn;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
 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.service.accord.async.AsyncContext;
 import org.apache.cassandra.service.accord.serializers.CommandSerializers;
+import org.apache.cassandra.utils.CollectionSerializer;
 
 import static org.apache.cassandra.service.accord.serializers.NullableSerializer.deserializeNullable;
 import static org.apache.cassandra.service.accord.serializers.NullableSerializer.serializeNullable;
 import static org.apache.cassandra.service.accord.serializers.NullableSerializer.serializedSizeNullable;
 
-public class AccordPartialCommand implements PartialCommand
+public class AccordPartialCommand extends CommandsForKey.TxnIdWithExecuteAt
 {
-    public static final PartialCommandSerializer<PartialCommand> serializer = new PartialCommandSerializer<PartialCommand>()
-    {
-        @Override
-        public PartialCommand getCachedFull(TxnId txnId, AsyncContext context)
-        {
-            return context.commands.get(txnId);
-        }
-
-        @Override
-        public void addToContext(PartialCommand command, AsyncContext context)
-        {
-            context.commands.addPartialCommand((AccordPartialCommand) command);
-        }
+    public static final PartialCommandSerializer serializer = new PartialCommandSerializer();
 
-        @Override
-        public PartialCommand deserializeBody(TxnId txnId, Txn txn, Timestamp executeAt, Status status, DataInputPlus in, AccordSerializerVersion version) throws IOException
-        {
-            return new AccordPartialCommand(txnId, txn, executeAt, status);
-        }
-    };
-
-    private final TxnId txnId;
-    private final Txn txn;
-    private final Timestamp executeAt;
+    // TODO (soon): this should only be a list of TxnId (the deps for the key we are persisted against); but should also be stored separately and not brought into memory
+    private final List<TxnId> deps;
+    // TODO (soon): we only require this for Accepted; perhaps more tightly couple query API for efficiency
     private final Status status;
-    private List<Listener> removedListeners = null;
+    private final Txn.Kind kind;
 
-    public AccordPartialCommand(TxnId txnId, Txn txn, Timestamp executeAt, Status status)
+    AccordPartialCommand(TxnId txnId, Timestamp executeAt, List<TxnId> deps, Status status, Txn.Kind kind)
     {
-        this.txnId = txnId;
-        this.txn = txn;
-        this.executeAt = executeAt;
+        super(txnId, executeAt);
+        this.deps = deps;
         this.status = status;
+        this.kind = kind;
     }
 
-    @Override
-    public TxnId txnId()
+    public AccordPartialCommand(Key key, Command command)
     {
-        return txnId;
+        this(command.txnId(), command.executeAt(), command.partialDeps().txnIds(key), command.status(), command.kind());
     }
 
-    @Override
-    public Txn txn()
+    public TxnId txnId()
     {
-        return txn;
+        return txnId;
     }
 
-    @Override
     public Timestamp executeAt()
     {
         return executeAt;
     }
 
-    @Override
-    public Status status()
-    {
-        return status;
-    }
-
-    @Override
-    public void removeListener(Listener listener)
+    public List<TxnId> deps()
     {
-        if (removedListeners == null)
-            removedListeners = new ArrayList<>();
-        removedListeners.add(listener);
+        return deps;
     }
 
-    public boolean hasRemovedListeners()
+    public boolean hasDep(TxnId txnId)
     {
-        return removedListeners != null && !removedListeners.isEmpty();
+        return Collections.binarySearch(deps, txnId) >= 0;
     }
 
-    public void forEachRemovedListener(Consumer<Listener> consumer)
+    public Status status()
     {
-        if (removedListeners != null)
-            removedListeners.forEach(consumer);
+        return status;
     }
 
-    @Override
-    public boolean equals(Object o)
+    public Txn.Kind kind()
     {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        AccordPartialCommand that = (AccordPartialCommand) o;
-        return Objects.equals(txnId, that.txnId) && Objects.equals(txn, that.txn) && Objects.equals(executeAt, that.executeAt) && status == that.status && Objects.equals(removedListeners, that.removedListeners);
+        return kind;
     }
 
     @Override
-    public int hashCode()
+    public boolean equals(Object obj)
     {
-        return Objects.hash(txnId, txn, executeAt, status, removedListeners);
+        if (obj.getClass() != AccordPartialCommand.class)
+            return false;
+        AccordPartialCommand that = (AccordPartialCommand) obj;
+        return txnId.equals(that.txnId)
+               && Objects.equals(executeAt, that.executeAt)
+               && Objects.equals(deps, that.deps)
+               && status == that.status
+               && kind == that.kind;
     }
 
-    @Override
-    public String toString()
+    public static class PartialCommandSerializer
     {
-        return "AccordPartialCommand{" +
-               "txnId=" + txnId +
-               ", txn=" + txn +
-               ", executeAt=" + executeAt +
-               ", status=" + status +
-               ", removedListeners=" + removedListeners +
-               '}';
-    }
-
-    public static class WithDeps extends AccordPartialCommand implements PartialCommand.WithDeps
-    {
-        public static final PartialCommandSerializer<PartialCommand.WithDeps> serializer = new PartialCommandSerializer<PartialCommand.WithDeps>()
+        public void serialize(AccordPartialCommand command, DataOutputPlus out, AccordSerializerVersion version) throws IOException
         {
-            @Override
-            public PartialCommand.WithDeps getCachedFull(TxnId txnId, AsyncContext context)
-            {
-                return context.commands.get(txnId);
-            }
-
-            @Override
-            public void addToContext(PartialCommand.WithDeps command, AsyncContext context)
-            {
-                context.commands.addPartialCommand((AccordPartialCommand) command);
-            }
-
-            @Override
-            public PartialCommand.WithDeps deserializeBody(TxnId txnId, Txn txn, Timestamp executeAt, Status status, DataInputPlus in, AccordSerializerVersion version) throws IOException
-            {
-                Deps deps = deserializeNullable(in, version.msgVersion, CommandSerializers.deps);
-                return new AccordPartialCommand.WithDeps(txnId, txn, executeAt, status, deps);
-            }
-
-            @Override
-            public void serialize(PartialCommand.WithDeps command, DataOutputPlus out, AccordSerializerVersion version) throws IOException
-            {
-                super.serialize(command, out, version);
-                serializeNullable(command.savedDeps(), out, version.msgVersion, CommandSerializers.deps);
-            }
-
-            @Override
-            public int serializedSize(PartialCommand.WithDeps command, AccordSerializerVersion version)
-            {
-                int size = super.serializedSize(command, version) ;
-                size += serializedSizeNullable(command.savedDeps(), version.msgVersion, CommandSerializers.deps);
-                return size;
-            }
-
-            @Override
-            public boolean needsUpdate(AccordCommand command)
-            {
-                return super.needsUpdate(command) || command.deps.hasModifications();
-            }
-        };
-
-        private final Deps deps;
-
-        public WithDeps(TxnId txnId, Txn txn, Timestamp executeAt, Status status, Deps deps)
-        {
-            super(txnId, txn, executeAt, status);
-            this.deps = deps;
-        }
-
-        @Override
-        public Deps savedDeps()
-        {
-            return deps;
-        }
-
-        @Override
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            AccordPartialCommand.WithDeps withDeps = (AccordPartialCommand.WithDeps) o;
-            return super.equals(o) && Objects.equals(deps, withDeps.deps);
-        }
-
-        @Override
-        public int hashCode()
-        {
-            return Objects.hash(super.hashCode(), deps);
-        }
-    }
-
-    public static abstract class PartialCommandSerializer<T extends PartialCommand>
-    {
-        public void serialize(T command, DataOutputPlus out, AccordSerializerVersion version) throws IOException
-        {
-            AccordSerializerVersion.serializer.serialize(version, out);
+            out.write(version.version);
             CommandSerializers.txnId.serialize(command.txnId(), out, version.msgVersion);
-            CommandSerializers.status.serialize(command.status(), out, version.msgVersion);
-            serializeNullable(command.txn(), out, version.msgVersion, CommandSerializers.txn);
             serializeNullable(command.executeAt(), out, version.msgVersion, CommandSerializers.timestamp);
+            CommandSerializers.status.serialize(command.status(), out, version.msgVersion);
+            CommandSerializers.kind.serialize(command.kind(), out, version.msgVersion);
+            CollectionSerializer.serializeCollection(CommandSerializers.txnId, command.deps, out, version.msgVersion);
         }
 
-        public ByteBuffer serialize(T command)
+        public ByteBuffer serialize(AccordPartialCommand command)
         {
             AccordSerializerVersion version = AccordSerializerVersion.CURRENT;
             int size = serializedSize(command, version);
@@ -257,28 +143,29 @@ public class AccordPartialCommand implements PartialCommand
         }
 
         // check for cached command first, otherwise deserialize
-        public T deserialize(AccordCommandStore commandStore, DataInputPlus in) throws IOException
+        private AccordPartialCommand deserialize(AccordCommandsForKey commandsForKey, AccordCommandStore commandStore, DataInputPlus in) throws IOException
         {
             AccordSerializerVersion version = deserializeVersion(in);
             TxnId txnId = CommandSerializers.txnId.deserialize(in, version.msgVersion);
             AsyncContext context = commandStore.getContext();
-            T command = getCachedFull(txnId, context);
+            AccordPartialCommand command = getCachedFull(commandsForKey, txnId, context);
             if (command != null)
                 return command;
 
-            Status status = CommandSerializers.status.deserialize(in, version.msgVersion);
-            Txn txn = deserializeNullable(in, version.msgVersion, CommandSerializers.txn);
             Timestamp executeAt = deserializeNullable(in, version.msgVersion, CommandSerializers.timestamp);
-            T partial = deserializeBody(txnId, txn, executeAt, status, in, version);
+            Status status = CommandSerializers.status.deserialize(in, version.msgVersion);
+            Txn.Kind kind = CommandSerializers.kind.deserialize(in, version.msgVersion);
+            List<TxnId> deps = CollectionSerializer.deserializeCollection(CommandSerializers.txnId, ArrayList::new, in, version.msgVersion);
+            AccordPartialCommand partial = new AccordPartialCommand(txnId, executeAt, deps, status, kind);
             addToContext(partial, context);
             return partial;
         }
 
-        public T deserialize(AccordCommandStore commandStore, ByteBuffer bytes)
+        public AccordPartialCommand deserialize(AccordCommandsForKey commandsForKey, AccordCommandStore commandStore, ByteBuffer bytes)
         {
             try (DataInputBuffer in = new DataInputBuffer(bytes, true))
             {
-                return deserialize(commandStore, in);
+                return deserialize(commandsForKey, commandStore, in);
             }
             catch (IOException e)
             {
@@ -286,26 +173,36 @@ public class AccordPartialCommand implements PartialCommand
             }
         }
 
-        public int serializedSize(T command, AccordSerializerVersion version)
+        public int serializedSize(AccordPartialCommand command, AccordSerializerVersion version)
         {
             int size = Math.toIntExact(AccordSerializerVersion.serializer.serializedSize(version));
             size += CommandSerializers.txnId.serializedSize();
-            size += CommandSerializers.status.serializedSize(command.status(), version.msgVersion);
-            size += serializedSizeNullable(command.txn(), version.msgVersion, CommandSerializers.txn);
             size += serializedSizeNullable(command.executeAt(), version.msgVersion, CommandSerializers.timestamp);
+            size += CommandSerializers.status.serializedSize(command.status(), version.msgVersion);
+            size += CommandSerializers.kind.serializedSize(command.kind(), version.msgVersion);
+            size += CollectionSerializer.serializedSizeCollection(CommandSerializers.txnId, command.deps, version.msgVersion);
             return size;
         }
 
-        public abstract T getCachedFull(TxnId txnId, AsyncContext context);
-        public abstract void addToContext(T command, AsyncContext context);
-        public abstract T deserializeBody(TxnId txnId, Txn txn, Timestamp executeAt, Status status, DataInputPlus in, AccordSerializerVersion version) throws IOException;
+        private AccordPartialCommand getCachedFull(AccordCommandsForKey commandsForKey, TxnId txnId, AsyncContext context)
+        {
+            AccordCommand command = context.commands.get(txnId);
+            if (command == null)
+                return null;
+            return new AccordPartialCommand(commandsForKey.key(), command);
+        }
+
+        private void addToContext(AccordPartialCommand command, AsyncContext context)
+        {
+            context.commands.addPartialCommand(command);
+        }
 
         /**
          * Determines if current modifications require updating command data duplicated elsewhere
          */
         public boolean needsUpdate(AccordCommand command)
         {
-            return command.txn.hasModifications() || command.executeAt.hasModifications() || command.status.hasModifications();
+            return command.executeAt.hasModifications() || command.status.hasModifications() || command.partialDeps.hasModifications();
         }
     }
 }
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/service/accord/AccordState.java b/src/java/org/apache/cassandra/service/accord/AccordState.java
index ba755fb1eb..2f5a9dc68c 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordState.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordState.java
@@ -18,6 +18,7 @@
 
 package org.apache.cassandra.service.accord;
 
+import java.util.function.BiConsumer;
 import java.util.function.Function;
 
 import org.apache.cassandra.service.accord.store.StoredNavigableMap;
@@ -26,7 +27,7 @@ import org.apache.cassandra.utils.concurrent.Future;
 
 public interface AccordState<K>
 {
-    public enum Kind { FULL, WRITE_ONLY, READ_ONLY }
+    enum ReadWrite { FULL, WRITE_ONLY, READ_ONLY }
 
     K key();
 
@@ -40,32 +41,32 @@ public interface AccordState<K>
 
     long estimatedSizeOnHeap();
 
-    default Kind kind()
+    default ReadWrite rw()
     {
-        return Kind.FULL;
+        return ReadWrite.FULL;
     }
 
     default boolean isFullInstance()
     {
-        return kind() == Kind.FULL;
+        return rw() == ReadWrite.FULL;
     }
 
     default boolean isWriteOnlyInstance()
     {
-        return kind() == Kind.WRITE_ONLY;
+        return rw() == ReadWrite.WRITE_ONLY;
     }
 
     default boolean isReadOnlyInstance()
     {
-        return kind() == Kind.READ_ONLY;
+        return rw() == ReadWrite.READ_ONLY;
     }
 
     interface WriteOnly<K, V extends AccordState<K>> extends AccordState<K>
     {
         @Override
-        default Kind kind()
+        default ReadWrite rw()
         {
-            return Kind.WRITE_ONLY;
+            return ReadWrite.WRITE_ONLY;
         }
 
         void future(Future<?> future);
@@ -86,7 +87,7 @@ public interface AccordState<K>
 
             StoredNavigableMap<K, V> toMap = getMap.apply(to);
             fromMap.forEachAddition(toMap::blindPut);
-            fromMap.forEachDeletion(toMap::blindRemove);
+            fromMap.forEachDeletion((BiConsumer<K, V>) toMap::blindRemove);
         }
 
         static <T, V extends Comparable<?>> void applySetChanges(T from, T to, Function<T, StoredSet<V, ?>> getSet)
@@ -101,13 +102,4 @@ public interface AccordState<K>
             fromSet.forEachDeletion(toSet::blindRemove);
         }
     }
-
-    interface ReadOnly<K, V extends AccordState<K>> extends AccordState<K>
-    {
-        @Override
-        default Kind kind()
-        {
-            return Kind.READ_ONLY;
-        }
-    }
 }
diff --git a/src/java/org/apache/cassandra/service/accord/AccordStateCache.java b/src/java/org/apache/cassandra/service/accord/AccordStateCache.java
index 6b3f2b8f02..37e7ba17cf 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordStateCache.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordStateCache.java
@@ -34,6 +34,7 @@ import com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import accord.api.Data;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.FutureCombiner;
@@ -156,7 +157,7 @@ public class AccordStateCache
     private final NamedMap<Object, Future<?>> loadFutures = new NamedMap<>("loadFutures");
     private final NamedMap<Object, Future<?>> saveFutures = new NamedMap<>("saveFutures");
 
-    private final NamedMap<Object, ReadFuture> readFutures = new NamedMap<>("readFutures");
+    private final NamedMap<Object, Future<Data>> readFutures = new NamedMap<>("readFutures");
     private final NamedMap<Object, Future<?>> writeFutures = new NamedMap<>("writeFutures");
 
     Node<?, ?> head;
@@ -537,12 +538,12 @@ public class AccordStateCache
             return saveFutures.get(key) != null;
         }
 
-        public ReadFuture getReadFuture(K key)
+        public Future<Data> getReadFuture(K key)
         {
             return getFuture(readFutures, key);
         }
 
-        public void setReadFuture(K key, ReadFuture future)
+        public void setReadFuture(K key, Future<Data> future)
         {
             setFuture(readFutures, key, future);
         }
diff --git a/src/java/org/apache/cassandra/service/accord/AccordTopologyUtils.java b/src/java/org/apache/cassandra/service/accord/AccordTopologyUtils.java
index 332a51379a..22bd631e31 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordTopologyUtils.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordTopologyUtils.java
@@ -39,8 +39,8 @@ import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.accord.api.AccordKey.SentinelKey;
-import org.apache.cassandra.service.accord.api.AccordKey.TokenKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey.SentinelKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey.TokenKey;
 
 public class AccordTopologyUtils
 {
@@ -54,17 +54,17 @@ public class AccordTopologyUtils
 
     private static TokenRange minRange(TableId tableId, Token token)
     {
-        return new TokenRange(SentinelKey.min(tableId), TokenKey.max(tableId, token));
+        return new TokenRange(SentinelKey.min(tableId), new TokenKey(tableId, token));
     }
 
     private static TokenRange maxRange(TableId tableId, Token token)
     {
-        return new TokenRange(TokenKey.max(tableId, token), SentinelKey.max(tableId));
+        return new TokenRange(new TokenKey(tableId, token), SentinelKey.max(tableId));
     }
 
     private static TokenRange range(TableId tableId, Token left, Token right)
     {
-        return new TokenRange(TokenKey.max(tableId, left), TokenKey.max(tableId, right));
+        return new TokenRange(new TokenKey(tableId, left), new TokenKey(tableId, right));
     }
 
     public static List<Shard> createShards(TableMetadata tableMetadata, TokenMetadata tokenMetadata)
diff --git a/src/java/org/apache/cassandra/service/accord/AccordTxnBuilder.java b/src/java/org/apache/cassandra/service/accord/AccordTxnBuilder.java
index 1eb8b3915f..f32940345e 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordTxnBuilder.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordTxnBuilder.java
@@ -33,7 +33,7 @@ import com.google.common.base.Preconditions;
 
 import accord.api.Key;
 import accord.primitives.Keys;
-import accord.txn.Txn;
+import accord.primitives.Txn;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.QueryHandler;
 import org.apache.cassandra.cql3.QueryProcessor;
@@ -52,6 +52,7 @@ import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.accord.api.AccordKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey;
 import org.apache.cassandra.service.accord.db.AccordQuery;
 import org.apache.cassandra.service.accord.db.AccordRead;
 import org.apache.cassandra.service.accord.db.AccordUpdate;
@@ -184,7 +185,7 @@ public class AccordTxnBuilder
     public Txn build()
     {
         Key[] keyArray = keys.toArray(new Key[0]);
-        Arrays.sort(keyArray, AccordKey::compareKeys);
+        Arrays.sort(keyArray, AccordRoutingKey::compareKeys);
         predicates.sort(Comparator.comparing(UpdatePredicate::partitionKey));
         if (updates.isEmpty())
         {
diff --git a/src/java/org/apache/cassandra/service/accord/ListenerProxy.java b/src/java/org/apache/cassandra/service/accord/ListenerProxy.java
index 2c9c5e07a9..e0397e2651 100644
--- a/src/java/org/apache/cassandra/service/accord/ListenerProxy.java
+++ b/src/java/org/apache/cassandra/service/accord/ListenerProxy.java
@@ -28,9 +28,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import accord.local.Command;
-import accord.local.CommandStore;
-import accord.local.Listener;
+import accord.local.CommandListener;
 import accord.local.PreLoadContext;
+import accord.local.SafeCommandStore;
 import accord.primitives.TxnId;
 import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.db.marshal.ValueAccessor;
@@ -39,7 +39,7 @@ import org.apache.cassandra.service.accord.async.AsyncContext;
 import org.apache.cassandra.service.accord.serializers.CommandSerializers;
 import org.apache.cassandra.utils.ObjectSizes;
 
-public abstract class ListenerProxy implements Listener, Comparable<ListenerProxy>
+public abstract class ListenerProxy implements CommandListener, Comparable<ListenerProxy>
 {
     private static final Logger logger = LoggerFactory.getLogger(ListenerProxy.class);
 
@@ -48,11 +48,8 @@ public abstract class ListenerProxy implements Listener, Comparable<ListenerProx
     public abstract Kind kind();
     public abstract ByteBuffer identifier();
 
-    final CommandStore commandStore;
-
-    private ListenerProxy(CommandStore commandStore)
+    private ListenerProxy()
     {
-        this.commandStore = commandStore;
     }
 
     @Override
@@ -65,12 +62,11 @@ public abstract class ListenerProxy implements Listener, Comparable<ListenerProx
 
     static class CommandListenerProxy extends ListenerProxy
     {
-        private static final long EMPTY_SIZE = ObjectSizes.measure(new CommandListenerProxy(null, null));
+        private static final long EMPTY_SIZE = ObjectSizes.measure(new CommandListenerProxy(null));
         private final TxnId txnId;
 
-        public CommandListenerProxy(CommandStore commandStore, TxnId txnId)
+        public CommandListenerProxy(TxnId txnId)
         {
-            super(commandStore);
             this.txnId = txnId;
         }
 
@@ -129,10 +125,10 @@ public abstract class ListenerProxy implements Listener, Comparable<ListenerProx
         }
 
         @Override
-        public void onChange(Command c)
+        public void onChange(SafeCommandStore safeStore, Command c)
         {
             AccordCommand command = (AccordCommand) c;
-            AccordCommandStore commandStore = command.commandStore();
+            AccordCommandStore commandStore = (AccordCommandStore) safeStore;
             AsyncContext context = commandStore.getContext();
             PreLoadContext loadCtx = PreLoadContext.contextFor(ImmutableList.of(command.txnId(), txnId), Collections.emptyList());
             if (context.containsScopedItems(loadCtx))
@@ -140,15 +136,15 @@ public abstract class ListenerProxy implements Listener, Comparable<ListenerProx
                 // TODO (soon): determine if this can break anything by not waiting for the current operation to denormalize it's data
                 //  the summary loader may default to commands in context, in case it wouldn't
                 logger.trace("{}: synchronously updating listening command {}", c.txnId(), txnId);
-                commandStore.command(txnId).onChange(c);
+                commandStore.command(txnId).onChange(safeStore, c);
             }
             else
             {
                 TxnId callingTxnId = command.txnId();
                 logger.trace("{}: asynchronously updating listening command {}", c.txnId(), txnId);
-                commandStore.process(loadCtx, instance -> {
-                    Command caller = instance.command(callingTxnId);
-                    commandStore.command(txnId).onChange(caller);
+                commandStore.execute(loadCtx, reSafeStore -> {
+                    Command caller = reSafeStore.command(callingTxnId);
+                    commandStore.command(txnId).onChange(reSafeStore, caller);
                 });
             }
         }
@@ -166,12 +162,11 @@ public abstract class ListenerProxy implements Listener, Comparable<ListenerProx
      */
     static class CommandsForKeyListenerProxy extends ListenerProxy
     {
-        private static final long EMPTY_SIZE = ObjectSizes.measure(new CommandsForKeyListenerProxy(null, null));
+        private static final long EMPTY_SIZE = ObjectSizes.measure(new CommandsForKeyListenerProxy(null));
         private final AccordKey.PartitionKey key;
 
-        public CommandsForKeyListenerProxy(CommandStore commandStore, AccordKey.PartitionKey key)
+        public CommandsForKeyListenerProxy(AccordKey.PartitionKey key)
         {
-            super(commandStore);
             this.key = key;
         }
 
@@ -230,24 +225,24 @@ public abstract class ListenerProxy implements Listener, Comparable<ListenerProx
         }
 
         @Override
-        public void onChange(Command c)
+        public void onChange(SafeCommandStore safeStore, Command c)
         {
             AccordCommand command = (AccordCommand) c;
-            AccordCommandStore commandStore = command.commandStore();
+            AccordCommandStore commandStore = (AccordCommandStore) safeStore;
             AsyncContext context = commandStore.getContext();
             PreLoadContext loadCtx = PreLoadContext.contextFor(ImmutableList.of(command.txnId()), ImmutableList.of(key));
             if (context.containsScopedItems(loadCtx))
             {
                 logger.trace("{}: synchronously updating listening cfk {}", c.txnId(), key);
-                commandStore.commandsForKey(key).onChange(c);
+                commandStore.commandsForKey(key).onChange(safeStore, c);
             }
             else
             {
                 TxnId callingTxnId = command.txnId();
                 logger.trace("{}: asynchronously updating listening cfk {}", c.txnId(), key);
-                commandStore.process(loadCtx, instance -> {
-                    Command caller = instance.command(callingTxnId);
-                    commandStore.commandsForKey(key).onChange(caller);
+                commandStore.execute(loadCtx, reSafeStore -> {
+                    Command caller = reSafeStore.command(callingTxnId);
+                    commandStore.commandsForKey(key).onChange(reSafeStore, caller);
                 });
             }
         }
@@ -259,7 +254,7 @@ public abstract class ListenerProxy implements Listener, Comparable<ListenerProx
         }
     }
 
-    public static <V> ListenerProxy deserialize(CommandStore commandStore, V src, ValueAccessor<V> accessor, int offset) throws IOException
+    public static <V> ListenerProxy deserialize(V src, ValueAccessor<V> accessor, int offset) throws IOException
     {
         int ordinal = accessor.getByte(src, offset);
         Kind kind = Kind.values()[ordinal];
@@ -268,10 +263,10 @@ public abstract class ListenerProxy implements Listener, Comparable<ListenerProx
         {
             case COMMAND:
                 TxnId txnId = CommandSerializers.txnId.deserialize(src, accessor, offset);
-                return new CommandListenerProxy(commandStore, txnId);
+                return new CommandListenerProxy(txnId);
             case COMMANDS_FOR_KEY:
                 AccordKey.PartitionKey key = AccordKey.PartitionKey.serializer.deserialize(src, accessor, offset);
-                return new CommandsForKeyListenerProxy(commandStore, key);
+                return new CommandsForKeyListenerProxy(key);
             default:
                 throw new IOException("Unknown kind ordinal " + ordinal);
         }
diff --git a/src/java/org/apache/cassandra/service/accord/ReadFuture.java b/src/java/org/apache/cassandra/service/accord/ReadFuture.java
index 46eb4f86ad..11c1800383 100644
--- a/src/java/org/apache/cassandra/service/accord/ReadFuture.java
+++ b/src/java/org/apache/cassandra/service/accord/ReadFuture.java
@@ -29,19 +29,16 @@ import java.util.function.Function;
 import com.google.common.util.concurrent.FutureCallback;
 
 import accord.api.Data;
-import accord.primitives.Keys;
 import io.netty.util.concurrent.GenericFutureListener;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 
 public class ReadFuture implements Future<Data>
 {
-    final Keys scope;
     private final Future<Data> wrappped;
 
-    public ReadFuture(Keys scope, Future<Data> wrappped)
+    public ReadFuture(Future<Data> wrappped)
     {
-        this.scope = scope;
         this.wrappped = wrappped;
     }
 
diff --git a/src/java/org/apache/cassandra/service/accord/TokenRange.java b/src/java/org/apache/cassandra/service/accord/TokenRange.java
index 0f8d7d5293..020197d995 100644
--- a/src/java/org/apache/cassandra/service/accord/TokenRange.java
+++ b/src/java/org/apache/cassandra/service/accord/TokenRange.java
@@ -20,33 +20,21 @@ package org.apache.cassandra.service.accord;
 
 import java.io.IOException;
 
-import com.google.common.base.Preconditions;
-
-import accord.api.Key;
+import accord.api.RoutingKey;
 import accord.primitives.KeyRange;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.accord.api.AccordKey;
-import org.apache.cassandra.service.accord.api.AccordKey.SentinelKey;
-import org.apache.cassandra.service.accord.api.AccordKey.TokenKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey.SentinelKey;
 
 public class TokenRange extends KeyRange.EndInclusive
 {
-    public TokenRange(AccordKey start, AccordKey end)
+    public TokenRange(AccordRoutingKey start, AccordRoutingKey end)
     {
         super(start, end);
-        Preconditions.checkArgument(start.kind().supportsRanges());
-        Preconditions.checkArgument(end.kind().supportsRanges());
-    }
-
-    private static AccordKey toAccordTokenOrSentinel(Key key)
-    {
-        AccordKey aKey = (AccordKey) key;
-        if (aKey instanceof TokenKey || aKey instanceof SentinelKey)
-            return aKey;
-        return new TokenKey(aKey.tableId(), aKey.partitionKey().getToken().maxKeyBound());
     }
 
     public static TokenRange fullRange(TableId tableId)
@@ -55,9 +43,9 @@ public class TokenRange extends KeyRange.EndInclusive
     }
 
     @Override
-    public TokenRange subRange(Key start, Key end)
+    public TokenRange subRange(RoutingKey start, RoutingKey end)
     {
-        return new TokenRange(toAccordTokenOrSentinel(start), toAccordTokenOrSentinel(end));
+        return new TokenRange((AccordRoutingKey) start, (AccordRoutingKey) end);
     }
 
     public static final IVersionedSerializer<TokenRange> serializer = new IVersionedSerializer<TokenRange>()
@@ -65,22 +53,22 @@ public class TokenRange extends KeyRange.EndInclusive
         @Override
         public void serialize(TokenRange range, DataOutputPlus out, int version) throws IOException
         {
-            AccordKey.serializer.serialize((AccordKey) range.start(), out, version);
-            AccordKey.serializer.serialize((AccordKey) range.end(), out, version);
+            AccordRoutingKey.serializer.serialize((AccordRoutingKey) range.start(), out, version);
+            AccordRoutingKey.serializer.serialize((AccordRoutingKey) range.end(), out, version);
         }
 
         @Override
         public TokenRange deserialize(DataInputPlus in, int version) throws IOException
         {
-            return new TokenRange(AccordKey.serializer.deserialize(in, version),
-                                  AccordKey.serializer.deserialize(in, version));
+            return new TokenRange(AccordRoutingKey.serializer.deserialize(in, version),
+                                  AccordRoutingKey.serializer.deserialize(in, version));
         }
 
         @Override
         public long serializedSize(TokenRange range, int version)
         {
-            return AccordKey.serializer.serializedSize((AccordKey) range.start(), version)
-                 + AccordKey.serializer.serializedSize((AccordKey) range.end(), version);
+            return AccordRoutingKey.serializer.serializedSize((AccordRoutingKey) range.start(), version)
+                 + AccordRoutingKey.serializer.serializedSize((AccordRoutingKey) range.end(), version);
         }
     };
 }
diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java b/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java
index a5e94d550f..8ed2c5ea24 100644
--- a/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java
+++ b/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java
@@ -43,4 +43,10 @@ public class AccordAgent implements Agent
     {
         // TODO: this
     }
+
+    @Override
+    public void onHandledException(Throwable throwable)
+    {
+        // TODO: this
+    }
 }
diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordKey.java b/src/java/org/apache/cassandra/service/accord/api/AccordKey.java
index 845d43cec0..05a47fb443 100644
--- a/src/java/org/apache/cassandra/service/accord/api/AccordKey.java
+++ b/src/java/org/apache/cassandra/service/accord/api/AccordKey.java
@@ -20,11 +20,11 @@ package org.apache.cassandra.service.accord.api;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Objects;
 
 import com.google.common.base.Preconditions;
 
 import accord.api.Key;
+import accord.api.RoutingKey;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.PartitionPosition;
@@ -45,26 +45,8 @@ import org.apache.cassandra.utils.ObjectSizes;
 
 public interface AccordKey extends Key
 {
-    enum Kind
-    {
-        TOKEN, PARTITION, SENTINEL;
-
-        public boolean supportsRanges()
-        {
-            switch (this)
-            {
-                case SENTINEL:
-                case TOKEN:
-                    return true;
-                default:
-                    return false;
-            }
-        }
-    };
-
     TableId tableId();
     PartitionPosition partitionKey();
-    Kind kind();
 
     static AccordKey of(Key key)
     {
@@ -81,185 +63,45 @@ public interface AccordKey extends Key
         return new PartitionKey(command.metadata().id, command.partitionKey());
     }
 
-    static int compare(AccordKey left, AccordKey right)
-    {
-        int cmp = left.tableId().compareTo(right.tableId());
-        if (cmp != 0)
-            return cmp;
-
-        if (left instanceof SentinelKey || right instanceof SentinelKey)
-        {
-            int leftInt = left instanceof SentinelKey ? ((SentinelKey) left).asInt() : 0;
-            int rightInt = right instanceof SentinelKey ? ((SentinelKey) right).asInt() : 0;
-            return Integer.compare(leftInt, rightInt);
-        }
-
-        return left.partitionKey().compareTo(right.partitionKey());
-    }
-
-    static int compareKeys(Key left, Key right)
+    abstract class AbstractKey<T extends PartitionPosition> extends AccordRoutingKey.AbstractRoutingKey implements AccordKey
     {
-        return compare((AccordKey) left, (AccordKey) right);
-    }
-
-    default int compareTo(AccordKey that)
-    {
-        return compare(this, that);
-    }
-
-    @Override
-    default int routingHash()
-    {
-        return partitionKey().getToken().tokenHash();
-    }
-
-    class SentinelKey implements AccordKey
-    {
-        private final TableId tableId;
-        private final boolean isMin;
-
-        private SentinelKey(TableId tableId, boolean isMin)
-        {
-            this.tableId = tableId;
-            this.isMin = isMin;
-        }
-
-        @Override
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            SentinelKey that = (SentinelKey) o;
-            return isMin == that.isMin && tableId.equals(that.tableId);
-        }
-
-        @Override
-        public int hashCode()
-        {
-            return Objects.hash(tableId, isMin);
-        }
-
-        @Override
-        public int compareTo(Key that)
-        {
-            return compare(this, (AccordKey) that);
-        }
-
-        @Override
-        public Kind kind()
-        {
-            return Kind.SENTINEL;
-        }
-
-        public static SentinelKey min(TableId tableId)
-        {
-            return new SentinelKey(tableId, true);
-        }
-
-        public static SentinelKey max(TableId tableId)
-        {
-            return new SentinelKey(tableId, false);
-        }
-
-        @Override
-        public TableId tableId()
-        {
-            return tableId;
-        }
-
-        @Override
-        public PartitionPosition partitionKey()
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        int asInt()
-        {
-            return isMin ? -1 : 1;
-        }
-
-        @Override
-        public String toString()
-        {
-            return "SentinelKey{" +
-                   "tableId=" + tableId +
-                   ", key=" + (isMin ? "min": "max") +
-                   '}';
-        }
-
-        public static final IVersionedSerializer<SentinelKey> serializer = new IVersionedSerializer<SentinelKey>()
-        {
-            @Override
-            public void serialize(SentinelKey key, DataOutputPlus out, int version) throws IOException
-            {
-                out.writeBoolean(key.isMin);
-                key.tableId().serialize(out);
-            }
-
-            @Override
-            public SentinelKey deserialize(DataInputPlus in, int version) throws IOException
-            {
-                boolean isMin = in.readBoolean();
-                TableId tableId = TableId.deserialize(in);
-                return new SentinelKey(tableId, isMin);
-            }
-
-            @Override
-            public long serializedSize(SentinelKey key, int version)
-            {
-                return TypeSizes.BOOL_SIZE + TableId.serializedSize();
-            }
-        };
-    }
-
-    abstract class AbstractKey<T extends PartitionPosition> implements AccordKey
-    {
-        private final TableId tableId;
         private final T key;
 
         public AbstractKey(TableId tableId, T key)
         {
-            this.tableId = tableId;
+            super(tableId);
             this.key = key;
         }
 
         @Override
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            AbstractKey<?> that = (AbstractKey<?>) o;
-            return tableId.equals(that.tableId) && key.equals(that.key);
-        }
-
-        @Override
-        public int hashCode()
+        public Token token()
         {
-            return Objects.hash(tableId, key);
+            return partitionKey().getToken();
         }
 
         @Override
-        public int compareTo(Key that)
+        public Kind kind()
         {
-            return compare(this, (AccordKey) that);
+            return Kind.TOKEN;
         }
 
         @Override
-        public TableId tableId()
+        public T partitionKey()
         {
-            return tableId;
+            return key;
         }
 
         @Override
-        public T partitionKey()
+        public RoutingKey toRoutingKey()
         {
-            return key;
+            return this;
         }
     }
 
     class PartitionKey extends AbstractKey<DecoratedKey>
     {
         private static final long EMPTY_SIZE;
+
         static
         {
             DecoratedKey key = DatabaseDescriptor.getPartitioner().decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
@@ -271,12 +113,6 @@ public interface AccordKey extends Key
             super(tableId, key);
         }
 
-        @Override
-        public Kind kind()
-        {
-            return Kind.PARTITION;
-        }
-
         @Override
         public String toString()
         {
@@ -286,6 +122,12 @@ public interface AccordKey extends Key
                    '}';
         }
 
+        @Override
+        public RoutingKey toRoutingKey()
+        {
+            return new TokenKey(tableId(), token());
+        }
+
         public long estimatedSizeOnHeap()
         {
             return EMPTY_SIZE + ByteBufferAccessor.instance.size(partitionKey().getKey());
@@ -321,7 +163,7 @@ public interface AccordKey extends Key
             public PartitionKey deserialize(DataInputPlus in, int version) throws IOException
             {
                 TableId tableId = TableId.deserialize(in);
-                TableMetadata metadata = Schema.instance.getTableMetadata(tableId);
+                TableMetadata metadata = Schema.instance.getExistingTableMetadata(tableId);
                 DecoratedKey key = metadata.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(in));
                 return new PartitionKey(tableId, key);
             }
@@ -352,129 +194,24 @@ public interface AccordKey extends Key
         }
     }
 
-    class TokenKey extends AbstractKey<Token.KeyBound>
-    {
-        public TokenKey(TableId tableId, Token.KeyBound key)
-        {
-            super(tableId, key);
-        }
-
-        public static TokenKey min(TableId tableId, Token token)
-        {
-            return new TokenKey(tableId, token.minKeyBound());
-        }
-
-        public static TokenKey max(TableId tableId, Token token)
-        {
-            return new TokenKey(tableId, token.maxKeyBound());
-        }
-
-        @Override
-        public Kind kind()
-        {
-            return Kind.TOKEN;
-        }
-
-        @Override
-        public String toString()
-        {
-            return "TokenKey{" +
-                   "tableId=" + tableId() +
-                   ", key=" + partitionKey() +
-                   '}';
-        }
-
-        public static final IVersionedSerializer<TokenKey> serializer = new IVersionedSerializer<TokenKey>()
-        {
-            @Override
-            public void serialize(TokenKey key, DataOutputPlus out, int version) throws IOException
-            {
-                key.tableId().serialize(out);
-                Token.KeyBound bound = key.partitionKey();
-                out.writeBoolean(bound.isMinimumBound);
-                Token.serializer.serialize(bound.getToken(), out, version);
-            }
-
-            @Override
-            public TokenKey deserialize(DataInputPlus in, int version) throws IOException
-            {
-                TableId tableId = TableId.deserialize(in);
-                TableMetadata metadata = Schema.instance.getTableMetadata(tableId);
-                boolean isMinimumBound = in.readBoolean();
-                Token token = Token.serializer.deserialize(in, metadata.partitioner, version);
-                return new TokenKey(tableId, isMinimumBound ? token.minKeyBound() : token.maxKeyBound());
-            }
-
-            @Override
-            public long serializedSize(TokenKey key, int version)
-            {
-                Token.KeyBound bound = key.partitionKey();
-                return key.tableId().serializedSize()
-                     + TypeSizes.sizeof(bound.isMinimumBound)
-                     + Token.serializer.serializedSize(bound.getToken(), version);
-            }
-        };
-    }
-
     IVersionedSerializer<AccordKey> serializer = new IVersionedSerializer<AccordKey>()
     {
         @Override
         public void serialize(AccordKey key, DataOutputPlus out, int version) throws IOException
         {
-            out.write(key.kind().ordinal());
-            switch (key.kind())
-            {
-                case TOKEN:
-                    TokenKey.serializer.serialize((TokenKey) key, out, version);
-                    break;
-                case PARTITION:
-                    PartitionKey.serializer.serialize((PartitionKey) key, out, version);
-                    break;
-                case SENTINEL:
-                    SentinelKey.serializer.serialize((SentinelKey) key, out, version);
-                    break;
-                default:
-                    throw new IllegalArgumentException();
-            }
-
+            PartitionKey.serializer.serialize((PartitionKey) key, out, version);
         }
 
         @Override
         public AccordKey deserialize(DataInputPlus in, int version) throws IOException
         {
-            Kind kind = Kind.values()[in.readByte()];
-            switch (kind)
-            {
-                case TOKEN:
-                    return TokenKey.serializer.deserialize(in, version);
-                case PARTITION:
-                    return PartitionKey.serializer.deserialize(in, version);
-                case SENTINEL:
-                    return SentinelKey.serializer.deserialize(in, version);
-                default:
-                    throw new IllegalArgumentException();
-            }
+            return PartitionKey.serializer.deserialize(in, version);
         }
 
         @Override
         public long serializedSize(AccordKey key, int version)
         {
-            long size = TypeSizes.BYTE_SIZE; // kind ordinal
-            switch (key.kind())
-            {
-                case TOKEN:
-                    size += TokenKey.serializer.serializedSize((TokenKey) key, version);
-                    break;
-                case PARTITION:
-                    size += PartitionKey.serializer.serializedSize((PartitionKey) key, version);
-                    break;
-                case SENTINEL:
-                    size += SentinelKey.serializer.serializedSize((SentinelKey) key, version);
-                    break;
-                default:
-                    throw new IllegalArgumentException();
-            }
-            return size;
+            return PartitionKey.serializer.serializedSize((PartitionKey) key, version);
         }
     };
 }
diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordKey.java b/src/java/org/apache/cassandra/service/accord/api/AccordRoutingKey.java
similarity index 52%
copy from src/java/org/apache/cassandra/service/accord/api/AccordKey.java
copy to src/java/org/apache/cassandra/service/accord/api/AccordRoutingKey.java
index 845d43cec0..543e497c07 100644
--- a/src/java/org/apache/cassandra/service/accord/api/AccordKey.java
+++ b/src/java/org/apache/cassandra/service/accord/api/AccordRoutingKey.java
@@ -19,20 +19,12 @@
 package org.apache.cassandra.service.accord.api;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.Objects;
 
-import com.google.common.base.Preconditions;
-
 import accord.api.Key;
+import accord.api.RoutingKey;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.PartitionPosition;
-import org.apache.cassandra.db.SinglePartitionReadCommand;
 import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.db.marshal.ByteBufferAccessor;
-import org.apache.cassandra.db.marshal.ValueAccessor;
-import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -43,45 +35,24 @@ import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 
-public interface AccordKey extends Key
+public interface AccordRoutingKey extends RoutingKey
 {
     enum Kind
     {
-        TOKEN, PARTITION, SENTINEL;
-
-        public boolean supportsRanges()
-        {
-            switch (this)
-            {
-                case SENTINEL:
-                case TOKEN:
-                    return true;
-                default:
-                    return false;
-            }
-        }
-    };
+        TOKEN, SENTINEL;
+    }
 
     TableId tableId();
-    PartitionPosition partitionKey();
+    Token token();
     Kind kind();
+    long estimatedSizeOnHeap();
 
-    static AccordKey of(Key key)
+    static AccordRoutingKey of(Key key)
     {
-        return (AccordKey) key;
+        return (AccordRoutingKey) key;
     }
 
-    static PartitionKey of(Partition partition)
-    {
-        return new PartitionKey(partition.metadata().id, partition.partitionKey());
-    }
-
-    static PartitionKey of(SinglePartitionReadCommand command)
-    {
-        return new PartitionKey(command.metadata().id, command.partitionKey());
-    }
-
-    static int compare(AccordKey left, AccordKey right)
+    static int compare(AccordRoutingKey left, AccordRoutingKey right)
     {
         int cmp = left.tableId().compareTo(right.tableId());
         if (cmp != 0)
@@ -94,15 +65,15 @@ public interface AccordKey extends Key
             return Integer.compare(leftInt, rightInt);
         }
 
-        return left.partitionKey().compareTo(right.partitionKey());
+        return left.token().compareTo(right.token());
     }
 
     static int compareKeys(Key left, Key right)
     {
-        return compare((AccordKey) left, (AccordKey) right);
+        return compare((AccordRoutingKey) left, (AccordRoutingKey) right);
     }
 
-    default int compareTo(AccordKey that)
+    default int compareTo(AccordRoutingKey that)
     {
         return compare(this, that);
     }
@@ -110,11 +81,13 @@ public interface AccordKey extends Key
     @Override
     default int routingHash()
     {
-        return partitionKey().getToken().tokenHash();
+        return token().tokenHash();
     }
 
-    class SentinelKey implements AccordKey
+    class SentinelKey implements AccordRoutingKey
     {
+        private static final long EMPTY_SIZE = ObjectSizes.measure(new SentinelKey(null, true));
+
         private final TableId tableId;
         private final boolean isMin;
 
@@ -140,9 +113,9 @@ public interface AccordKey extends Key
         }
 
         @Override
-        public int compareTo(Key that)
+        public int compareTo(RoutingKey that)
         {
-            return compare(this, (AccordKey) that);
+            return compare(this, (AccordRoutingKey) that);
         }
 
         @Override
@@ -151,6 +124,12 @@ public interface AccordKey extends Key
             return Kind.SENTINEL;
         }
 
+        @Override
+        public long estimatedSizeOnHeap()
+        {
+            return EMPTY_SIZE;
+        }
+
         public static SentinelKey min(TableId tableId)
         {
             return new SentinelKey(tableId, true);
@@ -168,7 +147,7 @@ public interface AccordKey extends Key
         }
 
         @Override
-        public PartitionPosition partitionKey()
+        public Token token()
         {
             throw new UnsupportedOperationException();
         }
@@ -212,15 +191,13 @@ public interface AccordKey extends Key
         };
     }
 
-    abstract class AbstractKey<T extends PartitionPosition> implements AccordKey
+    abstract class AbstractRoutingKey implements AccordRoutingKey
     {
         private final TableId tableId;
-        private final T key;
 
-        public AbstractKey(TableId tableId, T key)
+        public AbstractRoutingKey(TableId tableId)
         {
             this.tableId = tableId;
-            this.key = key;
         }
 
         @Override
@@ -228,20 +205,20 @@ public interface AccordKey extends Key
         {
             if (this == o) return true;
             if (o == null || getClass() != o.getClass()) return false;
-            AbstractKey<?> that = (AbstractKey<?>) o;
-            return tableId.equals(that.tableId) && key.equals(that.key);
+            AbstractRoutingKey that = (AbstractRoutingKey) o;
+            return tableId.equals(that.tableId) && token().equals(that.token());
         }
 
         @Override
         public int hashCode()
         {
-            return Objects.hash(tableId, key);
+            return Objects.hash(tableId, token());
         }
 
         @Override
-        public int compareTo(Key that)
+        public int compareTo(RoutingKey that)
         {
-            return compare(this, (AccordKey) that);
+            return compare(this, (AccordRoutingKey) that);
         }
 
         @Override
@@ -249,150 +226,61 @@ public interface AccordKey extends Key
         {
             return tableId;
         }
-
-        @Override
-        public T partitionKey()
-        {
-            return key;
-        }
     }
 
-    class PartitionKey extends AbstractKey<DecoratedKey>
+    class TokenKey extends AbstractRoutingKey
     {
         private static final long EMPTY_SIZE;
+
         static
         {
-            DecoratedKey key = DatabaseDescriptor.getPartitioner().decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
-            EMPTY_SIZE = ObjectSizes.measureDeep(new PartitionKey(null, key));
+            Token key = DatabaseDescriptor.getPartitioner().decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER).getToken();
+            EMPTY_SIZE = ObjectSizes.measureDeep(new TokenKey(null, key));
+        }
+
+        final Token token;
+        public TokenKey(TableId tableId, Token token)
+        {
+            super(tableId);
+            this.token = token;
         }
 
-        public PartitionKey(TableId tableId, DecoratedKey key)
+        @Override
+        public Token token()
         {
-            super(tableId, key);
+            return token;
         }
 
         @Override
         public Kind kind()
         {
-            return Kind.PARTITION;
+            return Kind.TOKEN;
         }
 
         @Override
         public String toString()
         {
-            return "PartitionKey{" +
+            return "TokenKey{" +
                    "tableId=" + tableId() +
-                   ", key=" + partitionKey() +
+                   ", key=" + token() +
                    '}';
         }
 
         public long estimatedSizeOnHeap()
         {
-            return EMPTY_SIZE + ByteBufferAccessor.instance.size(partitionKey().getKey());
+            return EMPTY_SIZE + token().getHeapSize();
         }
 
         public static final Serializer serializer = new Serializer();
-        public static class Serializer implements IVersionedSerializer<PartitionKey>
+        public static class Serializer implements IVersionedSerializer<TokenKey>
         {
-            // TODO: add vint to value accessor and use vints
             private Serializer() {}
 
-            @Override
-            public void serialize(PartitionKey key, DataOutputPlus out, int version) throws IOException
-            {
-                key.tableId().serialize(out);
-                ByteBufferUtil.writeWithShortLength(key.partitionKey().getKey(), out);
-            }
-
-            public <V> int serialize(PartitionKey key, V dst, ValueAccessor<V> accessor, int offset)
-            {
-                int position = offset;
-                position += key.tableId().serialize(dst, accessor, position);
-                ByteBuffer bytes = key.partitionKey().getKey();
-                int numBytes = ByteBufferAccessor.instance.size(bytes);
-                Preconditions.checkState(numBytes <= Short.MAX_VALUE);
-                position += accessor.putShort(dst, position, (short) numBytes);
-                position += accessor.copyByteBufferTo(bytes, 0, dst, position, numBytes);
-                return position - offset;
-
-            }
-
-            @Override
-            public PartitionKey deserialize(DataInputPlus in, int version) throws IOException
-            {
-                TableId tableId = TableId.deserialize(in);
-                TableMetadata metadata = Schema.instance.getTableMetadata(tableId);
-                DecoratedKey key = metadata.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(in));
-                return new PartitionKey(tableId, key);
-            }
-
-            public <V> PartitionKey deserialize(V src, ValueAccessor<V> accessor, int offset) throws IOException
-            {
-                TableId tableId = TableId.deserialize(src, accessor, offset);
-                offset += TableId.serializedSize();
-                TableMetadata metadata = Schema.instance.getTableMetadata(tableId);
-                int numBytes = accessor.getShort(src, offset);
-                offset += TypeSizes.SHORT_SIZE;
-                ByteBuffer bytes = ByteBuffer.allocate(numBytes);
-                accessor.copyTo(src, offset, bytes, ByteBufferAccessor.instance, 0, numBytes);
-                DecoratedKey key = metadata.partitioner.decorateKey(bytes);
-                return new PartitionKey(tableId, key);
-            }
-
-            @Override
-            public long serializedSize(PartitionKey key, int version)
-            {
-                return serializedSize(key);
-            }
-
-            public long serializedSize(PartitionKey key)
-            {
-                return key.tableId().serializedSize() + ByteBufferUtil.serializedSizeWithShortLength(key.partitionKey().getKey());
-            }
-        }
-    }
-
-    class TokenKey extends AbstractKey<Token.KeyBound>
-    {
-        public TokenKey(TableId tableId, Token.KeyBound key)
-        {
-            super(tableId, key);
-        }
-
-        public static TokenKey min(TableId tableId, Token token)
-        {
-            return new TokenKey(tableId, token.minKeyBound());
-        }
-
-        public static TokenKey max(TableId tableId, Token token)
-        {
-            return new TokenKey(tableId, token.maxKeyBound());
-        }
-
-        @Override
-        public Kind kind()
-        {
-            return Kind.TOKEN;
-        }
-
-        @Override
-        public String toString()
-        {
-            return "TokenKey{" +
-                   "tableId=" + tableId() +
-                   ", key=" + partitionKey() +
-                   '}';
-        }
-
-        public static final IVersionedSerializer<TokenKey> serializer = new IVersionedSerializer<TokenKey>()
-        {
             @Override
             public void serialize(TokenKey key, DataOutputPlus out, int version) throws IOException
             {
                 key.tableId().serialize(out);
-                Token.KeyBound bound = key.partitionKey();
-                out.writeBoolean(bound.isMinimumBound);
-                Token.serializer.serialize(bound.getToken(), out, version);
+                Token.compactSerializer.serialize(key.token, out, version);
             }
 
             @Override
@@ -400,26 +288,22 @@ public interface AccordKey extends Key
             {
                 TableId tableId = TableId.deserialize(in);
                 TableMetadata metadata = Schema.instance.getTableMetadata(tableId);
-                boolean isMinimumBound = in.readBoolean();
-                Token token = Token.serializer.deserialize(in, metadata.partitioner, version);
-                return new TokenKey(tableId, isMinimumBound ? token.minKeyBound() : token.maxKeyBound());
+                Token token = Token.compactSerializer.deserialize(in, metadata.partitioner, version);
+                return new TokenKey(tableId, token);
             }
 
             @Override
             public long serializedSize(TokenKey key, int version)
             {
-                Token.KeyBound bound = key.partitionKey();
-                return key.tableId().serializedSize()
-                     + TypeSizes.sizeof(bound.isMinimumBound)
-                     + Token.serializer.serializedSize(bound.getToken(), version);
+                return TableId.serializedSize() + Token.compactSerializer.serializedSize(key.token(), version);
             }
-        };
+        }
     }
 
-    IVersionedSerializer<AccordKey> serializer = new IVersionedSerializer<AccordKey>()
+    IVersionedSerializer<AccordRoutingKey> serializer = new IVersionedSerializer<AccordRoutingKey>()
     {
         @Override
-        public void serialize(AccordKey key, DataOutputPlus out, int version) throws IOException
+        public void serialize(AccordRoutingKey key, DataOutputPlus out, int version) throws IOException
         {
             out.write(key.kind().ordinal());
             switch (key.kind())
@@ -427,28 +311,22 @@ public interface AccordKey extends Key
                 case TOKEN:
                     TokenKey.serializer.serialize((TokenKey) key, out, version);
                     break;
-                case PARTITION:
-                    PartitionKey.serializer.serialize((PartitionKey) key, out, version);
-                    break;
                 case SENTINEL:
                     SentinelKey.serializer.serialize((SentinelKey) key, out, version);
                     break;
                 default:
                     throw new IllegalArgumentException();
             }
-
         }
 
         @Override
-        public AccordKey deserialize(DataInputPlus in, int version) throws IOException
+        public AccordRoutingKey deserialize(DataInputPlus in, int version) throws IOException
         {
             Kind kind = Kind.values()[in.readByte()];
             switch (kind)
             {
                 case TOKEN:
                     return TokenKey.serializer.deserialize(in, version);
-                case PARTITION:
-                    return PartitionKey.serializer.deserialize(in, version);
                 case SENTINEL:
                     return SentinelKey.serializer.deserialize(in, version);
                 default:
@@ -457,7 +335,7 @@ public interface AccordKey extends Key
         }
 
         @Override
-        public long serializedSize(AccordKey key, int version)
+        public long serializedSize(AccordRoutingKey key, int version)
         {
             long size = TypeSizes.BYTE_SIZE; // kind ordinal
             switch (key.kind())
@@ -465,9 +343,6 @@ public interface AccordKey extends Key
                 case TOKEN:
                     size += TokenKey.serializer.serializedSize((TokenKey) key, version);
                     break;
-                case PARTITION:
-                    size += PartitionKey.serializer.serializedSize((PartitionKey) key, version);
-                    break;
                 case SENTINEL:
                     size += SentinelKey.serializer.serializedSize((SentinelKey) key, version);
                     break;
diff --git a/src/java/org/apache/cassandra/service/accord/async/AsyncLoader.java b/src/java/org/apache/cassandra/service/accord/async/AsyncLoader.java
index cc824b3890..74c25739e1 100644
--- a/src/java/org/apache/cassandra/service/accord/async/AsyncLoader.java
+++ b/src/java/org/apache/cassandra/service/accord/async/AsyncLoader.java
@@ -134,7 +134,7 @@ public class AsyncLoader
             try
             {
                 logger.trace("Starting load of {} for {}", command.txnId(), callback);
-                AccordKeyspace.loadCommand(command);
+                AccordKeyspace.loadCommand(commandStore, command);
                 logger.trace("Completed load of {} for {}", command.txnId(), callback);
             }
             catch (Throwable t)
diff --git a/src/java/org/apache/cassandra/service/accord/async/AsyncOperation.java b/src/java/org/apache/cassandra/service/accord/async/AsyncOperation.java
index 5fa51425a5..7f5bc1a1c5 100644
--- a/src/java/org/apache/cassandra/service/accord/async/AsyncOperation.java
+++ b/src/java/org/apache/cassandra/service/accord/async/AsyncOperation.java
@@ -30,12 +30,13 @@ import org.slf4j.MDC;
 import accord.api.Key;
 import accord.local.CommandStore;
 import accord.local.PreLoadContext;
+import accord.local.SafeCommandStore;
 import accord.primitives.TxnId;
 import org.apache.cassandra.service.accord.AccordCommandStore;
 import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
 import org.apache.cassandra.utils.concurrent.AsyncPromise;
 
-public abstract class AsyncOperation<R> extends AsyncPromise<R> implements Runnable, Function<CommandStore, R>, BiConsumer<Object, Throwable>
+public abstract class AsyncOperation<R> extends AsyncPromise<R> implements Runnable, Function<SafeCommandStore, R>, BiConsumer<Object, Throwable>
 {
     private static final Logger logger = LoggerFactory.getLogger(AsyncOperation.class);
 
@@ -214,45 +215,45 @@ public abstract class AsyncOperation<R> extends AsyncPromise<R> implements Runna
 
     static class ForFunction<R> extends AsyncOperation<R>
     {
-        private final Function<? super CommandStore, R> function;
+        private final Function<? super SafeCommandStore, R> function;
 
-        public ForFunction(AccordCommandStore commandStore, Iterable<TxnId> txnIds, Iterable<PartitionKey> keys, Function<? super CommandStore, R> function)
+        public ForFunction(AccordCommandStore commandStore, Iterable<TxnId> txnIds, Iterable<PartitionKey> keys, Function<? super SafeCommandStore, R> function)
         {
             super(commandStore, txnIds, keys);
             this.function = function;
         }
 
         @Override
-        public R apply(CommandStore commandStore)
+        public R apply(SafeCommandStore commandStore)
         {
             return function.apply(commandStore);
         }
     }
 
-    public static <T> AsyncOperation<T> create(CommandStore commandStore, PreLoadContext loadCtx, Function<? super CommandStore, T> function)
+    public static <T> AsyncOperation<T> create(CommandStore commandStore, PreLoadContext loadCtx, Function<? super SafeCommandStore, T> function)
     {
         return new ForFunction<>((AccordCommandStore) commandStore, loadCtx.txnIds(), AsyncOperation.toPartitionKeys(loadCtx.keys()), function);
     }
 
-    static class ForConsumer  extends AsyncOperation<Void>
+    static class ForConsumer extends AsyncOperation<Void>
     {
-        private final Consumer<? super CommandStore> consumer;
+        private final Consumer<? super SafeCommandStore> consumer;
 
-        public ForConsumer(AccordCommandStore commandStore, Iterable<TxnId> txnIds, Iterable<PartitionKey> keys, Consumer<? super CommandStore> consumer)
+        public ForConsumer(AccordCommandStore commandStore, Iterable<TxnId> txnIds, Iterable<PartitionKey> keys, Consumer<? super SafeCommandStore> consumer)
         {
             super(commandStore, txnIds, keys);
             this.consumer = consumer;
         }
 
         @Override
-        public Void apply(CommandStore commandStore)
+        public Void apply(SafeCommandStore commandStore)
         {
             consumer.accept(commandStore);
             return null;
         }
     }
 
-    public static AsyncOperation<Void> create(CommandStore commandStore, PreLoadContext loadCtx, Consumer<? super CommandStore> consumer)
+    public static AsyncOperation<Void> create(CommandStore commandStore, PreLoadContext loadCtx, Consumer<? super SafeCommandStore> consumer)
     {
         return new ForConsumer((AccordCommandStore) commandStore, loadCtx.txnIds(), AsyncOperation.toPartitionKeys(loadCtx.keys()), consumer);
     }
diff --git a/src/java/org/apache/cassandra/service/accord/async/AsyncWriter.java b/src/java/org/apache/cassandra/service/accord/async/AsyncWriter.java
index 04a63a1670..8b20347d77 100644
--- a/src/java/org/apache/cassandra/service/accord/async/AsyncWriter.java
+++ b/src/java/org/apache/cassandra/service/accord/async/AsyncWriter.java
@@ -19,11 +19,11 @@
 package org.apache.cassandra.service.accord.async;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.function.BiConsumer;
 import java.util.function.BiFunction;
+import java.util.function.Consumer;
 import java.util.function.Function;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -33,7 +33,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import accord.api.Key;
-import accord.local.Status;
+import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.db.Mutation;
@@ -45,7 +45,6 @@ import org.apache.cassandra.service.accord.AccordPartialCommand;
 import org.apache.cassandra.service.accord.AccordStateCache;
 import org.apache.cassandra.service.accord.AccordState;
 import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
-import org.apache.cassandra.service.accord.store.StoredNavigableMap;
 import org.apache.cassandra.service.accord.store.StoredSet;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.FutureCombiner;
@@ -77,7 +76,7 @@ public class AsyncWriter
 
     private interface StateMutationFunction<K, V extends AccordState<K>>
     {
-        Mutation apply(V state, long timestamp);
+        Mutation apply(AccordCommandStore commandStore, V state, long timestamp);
     }
 
     private static <K, V extends AccordState<K>> List<Future<?>> dispatchWrites(AsyncContext.Group<K, V> ctxGroup,
@@ -100,7 +99,7 @@ public class AsyncWriter
             if (futures == null)
                 futures = new ArrayList<>();
             K key = item.key();
-            Mutation mutation = mutationFunction.apply(item, timestamp);
+            Mutation mutation = mutationFunction.apply(commandStore, item, timestamp);
             if (logger.isTraceEnabled())
                 logger.trace("Dispatching mutation for {} for {}, {} -> {}", key, callback, item, mutation);
             Future<?> future = Stage.MUTATION.submit(() -> {
@@ -125,9 +124,8 @@ public class AsyncWriter
         for (AccordState.WriteOnly<K, V> item : ctxGroup.writeOnly.values())
         {
             Preconditions.checkState(item.hasModifications());
-            if (futures == null)
-                futures = new ArrayList<>();
-            Mutation mutation = mutationFunction.apply((V) item, timestamp);
+            if (futures == null) futures = new ArrayList<>();
+            Mutation mutation = mutationFunction.apply(commandStore, (V) item, timestamp);
             Future<?> future = Stage.MUTATION.submit((Runnable) mutation::apply);
             future.addListener(() -> cache.purgeWriteOnly(item.key()), commandStore.executor());
             item.future(future);
@@ -163,16 +161,16 @@ public class AsyncWriter
 
     private void denormalizeBlockedOn(AccordCommand command,
                                       AsyncContext context,
-                                      Function<AccordCommand, StoredNavigableMap<TxnId, ByteBuffer>> waitingField,
+                                      Function<AccordCommand, StoredSet.Changes<TxnId>> waitingField,
                                       Function<AccordCommand, StoredSet.Navigable<TxnId>> blockingField)
     {
-        StoredNavigableMap<TxnId, ?> waitingOn = waitingField.apply(command);
+        StoredSet.Changes<TxnId> waitingOn = waitingField.apply(command);
         waitingOn.forEachDeletion(deletedId -> {
             AccordCommand blockedOn = commandForDenormalization(deletedId, context);
             blockingField.apply(blockedOn).blindRemove(command.txnId());
         });
 
-        waitingOn.forEachAddition((addedId, unused) -> {
+        waitingOn.forEachAddition(addedId -> {
             AccordCommand blockedOn = commandForDenormalization(addedId, context);
             blockingField.apply(blockedOn).blindAdd(command.txnId());
         });
@@ -180,13 +178,12 @@ public class AsyncWriter
 
     private void denormalizeWaitingOnSummaries(AccordCommand command,
                                                AsyncContext context,
-                                               ByteBuffer summary,
-                                               Function<AccordCommand, StoredNavigableMap<TxnId, ByteBuffer>> waitingField,
+                                               Function<AccordCommand, BiConsumer<TxnId, Timestamp>> waitingField,
                                                Function<AccordCommand, StoredSet.Navigable<TxnId>> blockingField)
     {
         blockingField.apply(command).getView().forEach(blockingId -> {
             AccordCommand blocking = commandForDenormalization(blockingId, context);
-            waitingField.apply(blocking).blindPut(command.txnId(), summary.duplicate());
+            waitingField.apply(blocking).accept(command.txnId(), command.executeAt());
         });
     }
 
@@ -202,7 +199,7 @@ public class AsyncWriter
             return item;
 
         item = cache.getOrNull(key);
-        if (item != null)
+        if (item != null && !cache.hasLoadFuture(key))
         {
             ctxGroup.items.put(key, item);
             return item;
@@ -213,7 +210,7 @@ public class AsyncWriter
 
     private AccordCommand commandForDenormalization(TxnId txnId, AsyncContext context)
     {
-        return (AccordCommand) getForDenormalization(txnId, commandStore, context.commands, commandCache, AccordCommand.WriteOnly::new);
+        return (AccordCommand) getForDenormalization(txnId, commandStore, context.commands, commandCache, (ignore, id) -> new AccordCommand.WriteOnly(id));
     }
 
     private AccordCommandsForKey cfkForDenormalization(PartitionKey key, AsyncContext context)
@@ -228,21 +225,38 @@ public class AsyncWriter
 
         // notify commands we're waiting on that they need to update the summaries in our maps
         if (command.waitingOnCommit.hasModifications())
+        {
             denormalizeBlockedOn(command, context, cmd -> cmd.waitingOnCommit, cmd -> cmd.blockingCommitOn);
+        }
         if (command.waitingOnApply.hasModifications())
-            denormalizeBlockedOn(command, context, cmd -> cmd.waitingOnApply, cmd -> cmd.blockingApplyOn);
+        {
+            denormalizeBlockedOn(command, context, cmd -> new StoredSet.Changes<TxnId>()
+            {
+                @Override
+                public void forEachAddition(Consumer<TxnId> consumer)
+                {
+                    cmd.waitingOnApply.forEachAddition((ignore, txnId) -> consumer.accept(txnId));
+                }
+
+                @Override
+                public void forEachDeletion(Consumer<TxnId> consumer)
+                {
+                    cmd.waitingOnApply.forEachDeletion((ignore, txnId) -> consumer.accept(txnId));
+
+                }
+            }, cmd -> cmd.blockingApplyOn);
+        }
 
         if (command.shouldUpdateDenormalizedWaitingOn())
         {
-            ByteBuffer summary = AccordPartialCommand.serializer.serialize(command);
-            denormalizeWaitingOnSummaries(command, context, summary, cmd -> cmd.waitingOnCommit, cmd -> cmd.blockingCommitOn);
-            denormalizeWaitingOnSummaries(command, context, summary, cmd -> cmd.waitingOnApply, cmd -> cmd.blockingApplyOn);
+            denormalizeWaitingOnSummaries(command, context, cmd -> (txnId, ignore) -> cmd.waitingOnCommit.blindAdd(txnId), cmd -> cmd.blockingCommitOn);
+            denormalizeWaitingOnSummaries(command, context, cmd -> (txnId, executeAt) -> cmd.waitingOnApply.blindPut(executeAt, txnId), cmd -> cmd.blockingApplyOn);
         }
 
         // There won't be a txn to denormalize against until the command has been preaccepted
-        if (command.status().hasBeen(Status.PreAccepted) && AccordPartialCommand.WithDeps.serializer.needsUpdate(command) && !(command.txn() == null && command.status().isInvalidated()))
+        if (command.known().hasTxn && AccordPartialCommand.serializer.needsUpdate(command))
         {
-            for (Key key : command.txn().keys())
+            for (Key key : command.partialTxn().keys())
             {
                 PartitionKey partitionKey = (PartitionKey) key;
                 AccordCommandsForKey cfk = cfkForDenormalization(partitionKey, context);
@@ -257,21 +271,10 @@ public class AsyncWriter
         }
     }
 
-    private void denormalizePartial(AccordPartialCommand partial, AsyncContext context, Object callback)
-    {
-        if (!partial.hasRemovedListeners())
-            return;
-
-        logger.trace("Denormalizing partial command change on {} for {}", partial, callback);
-        AccordCommand forUpdate = commandForDenormalization(partial.txnId(), context);
-        partial.forEachRemovedListener(forUpdate::removeListener);
-    }
-
     private void denormalize(AsyncContext context, Object callback)
     {
         // need to clone "values" as denormalize will mutate it
         new ArrayList<>(context.commands.items.values()).forEach(command -> denormalize(command, context, callback));
-        context.commands.partials.forEach(command -> denormalizePartial(command, context, callback));
     }
 
     @VisibleForTesting
diff --git a/src/java/org/apache/cassandra/service/accord/db/AbstractKeyIndexed.java b/src/java/org/apache/cassandra/service/accord/db/AbstractKeyIndexed.java
index 84bb9cc216..9e1fb75a9e 100644
--- a/src/java/org/apache/cassandra/service/accord/db/AbstractKeyIndexed.java
+++ b/src/java/org/apache/cassandra/service/accord/db/AbstractKeyIndexed.java
@@ -20,16 +20,23 @@ package org.apache.cassandra.service.accord.db;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Objects;
 import java.util.TreeMap;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 
 import com.google.common.base.Preconditions;
 
+import accord.api.Key;
+import accord.primitives.KeyRange;
+import accord.primitives.KeyRanges;
+import accord.primitives.Keys;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.marshal.ByteBufferAccessor;
 import org.apache.cassandra.io.IVersionedSerializer;
@@ -38,6 +45,7 @@ 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.net.MessagingService;
+import org.apache.cassandra.service.accord.AccordObjectSizes;
 import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -47,7 +55,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
  */
 public abstract class AbstractKeyIndexed<T>
 {
-    final NavigableMap<PartitionKey, ByteBuffer> serialized;
+    final Keys keys;
+    final ByteBuffer[] serialized;
 
     abstract void serialize(T t, DataOutputPlus out, int version) throws IOException;
     abstract T deserialize(DataInputPlus in, int version) throws IOException;
@@ -70,6 +79,22 @@ public abstract class AbstractKeyIndexed<T>
         }
     }
 
+    protected static <T> ByteBuffer serialize(T item, IVersionedSerializer<T> serializer)
+    {
+        int version = MessagingService.current_version;
+        long size = serializer.serializedSize(item, version) + TypeSizes.INT_SIZE;
+        try (DataOutputBuffer out = new DataOutputBuffer((int) size))
+        {
+            out.writeInt(version);
+            serializer.serialize(item, out, version);
+            return out.buffer(false);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
     protected T deserialize(ByteBuffer bytes)
     {
         try (DataInputBuffer in = new DataInputBuffer(bytes, true))
@@ -89,65 +114,93 @@ public abstract class AbstractKeyIndexed<T>
         if (this == o) return true;
         if (o == null || getClass() != o.getClass()) return false;
         AbstractKeyIndexed<?> that = (AbstractKeyIndexed<?>) o;
-        return serialized.equals(that.serialized);
+        return Arrays.equals(serialized, that.serialized);
     }
 
     @Override
     public int hashCode()
     {
-        return Objects.hash(serialized);
+        return Arrays.hashCode(serialized);
     }
 
     @Override
     public String toString()
     {
-        return getClass().getSimpleName() + serialized.entrySet().stream()
-                         .map(e -> e.getKey() + "=" + deserialize(e.getValue()))
-                         .collect(Collectors.joining(", ", "{", "}"));
+        return getClass().getSimpleName() + IntStream.range(0, keys.size())
+                                                     .mapToObj(i -> keys.get(i) + "=" + deserialize(serialized[i]))
+                                                     .collect(Collectors.joining(", ", "{", "}"));
+    }
+
+    protected AbstractKeyIndexed(Keys keys, ByteBuffer[] serialized)
+    {
+        this.keys = keys;
+        this.serialized = serialized;
     }
 
     public AbstractKeyIndexed(List<T> items, Function<T, PartitionKey> keyFunction)
     {
-        this.serialized = new TreeMap<>();
+        Key[] keys = new Key[items.size()];
+        for (int i=0, mi=items.size(); i<mi; i++)
+            keys[i] = keyFunction.apply(items.get(i));
+        this.keys = Keys.of(keys);
+        this.serialized = new ByteBuffer[items.size()];
         for (int i=0, mi=items.size(); i<mi; i++)
+            serialized[this.keys.indexOf(keyFunction.apply(items.get(i)))] = serialize(items.get(i));
+    }
+
+    protected <V> V slice(KeyRanges ranges, BiFunction<Keys, ByteBuffer[], V> constructor)
+    {
+        // TODO: Routables patch permits us to do this more efficiently
+        Keys keys = this.keys.slice(ranges);
+        ByteBuffer[] serialized = new ByteBuffer[keys.size()];
+        int j = 0;
+        for (int i = 0 ; i < keys.size() ; ++i)
         {
-            T item = items.get(i);
-            PartitionKey key = keyFunction.apply(item);
-            // TODO: support multiple reads/writes per key
-            Preconditions.checkArgument(!this.serialized.containsKey(key));
-            this.serialized.put(key, serialize(item));
+            j = this.keys.findNext(keys.get(i), j);
+            serialized[i] = this.serialized[j++];
         }
+        return constructor.apply(keys, serialized);
     }
 
-    public AbstractKeyIndexed(NavigableMap<PartitionKey, ByteBuffer> serialized)
+    public <V> V merge(AbstractKeyIndexed<?> that, BiFunction<Keys, ByteBuffer[], V> constructor)
     {
-        this.serialized = serialized;
+        // TODO: special method for linear merging keyed and non-keyed lists simultaneously
+        Keys keys = this.keys.union(that.keys);
+        ByteBuffer[] serialized = new ByteBuffer[keys.size()];
+        int i = 0, j = 0, o = 0;
+        while (i < this.keys.size() && j < that.keys.size())
+        {
+            int c = this.keys.get(i).compareTo(that.keys.get(j));
+            if (c < 0) serialized[o++] = this.serialized[i++];
+            else if (c > 0) serialized[o++] = that.serialized[j++];
+            else { serialized[o++] = this.serialized[i++]; j++; }
+        }
+        while (i < this.keys.size())
+            serialized[o++] = this.serialized[i++];
+        while (j < that.keys.size())
+            serialized[o++] = that.serialized[j++];
+        return constructor.apply(keys, serialized);
     }
 
     public T getDeserialized(PartitionKey key)
     {
-        ByteBuffer bytes = serialized.get(key);
-        if (bytes == null)
-            return null;
-        return deserialize(bytes);
+        int i = keys.indexOf(key);
+        if (i < 0) return null;
+        return deserialize(serialized[i]);
     }
 
     public long estimatedSizeOnHeap()
     {
-        long size = emptySizeOnHeap();
-        for (Map.Entry<PartitionKey, ByteBuffer> entry : serialized.entrySet())
-        {
-            size += entry.getKey().estimatedSizeOnHeap();
-            size += ByteBufferUtil.EMPTY_SIZE_ON_HEAP + ByteBufferAccessor.instance.size(entry.getValue());
-        }
+        long size = emptySizeOnHeap() + AccordObjectSizes.keys(keys);
+        for (ByteBuffer buffer : serialized) size += ByteBufferUtil.estimatedSizeOnHeap(buffer);
         return size;
     }
 
     static class Serializer<V, S extends AbstractKeyIndexed<V>> implements IVersionedSerializer<S>
     {
-        private final Function<NavigableMap<PartitionKey, ByteBuffer>, S> factory;
+        private final BiFunction<Keys, ByteBuffer[], S> factory;
 
-        Serializer(Function<NavigableMap<PartitionKey, ByteBuffer>, S> factory)
+        Serializer(BiFunction<Keys, ByteBuffer[], S> factory)
         {
             this.factory = factory;
         }
@@ -155,33 +208,31 @@ public abstract class AbstractKeyIndexed<T>
         @Override
         public void serialize(S items, DataOutputPlus out, int version) throws IOException
         {
-            out.writeUnsignedVInt(items.serialized.size());
-            for (Map.Entry<PartitionKey, ByteBuffer> entry : items.serialized.entrySet())
-            {
-                PartitionKey.serializer.serialize(entry.getKey(), out, version);
-                ByteBufferUtil.writeWithVIntLength(entry.getValue(), out);
-            }
+            out.writeUnsignedVInt(items.serialized.length);
+            for (Key key : items.keys) PartitionKey.serializer.serialize((PartitionKey) key, out, version);
+            for (ByteBuffer buffer : items.serialized) ByteBufferUtil.writeWithVIntLength(buffer, out);
         }
 
         @Override
         public S deserialize(DataInputPlus in, int version) throws IOException
         {
             int size = (int) in.readUnsignedVInt();
-            NavigableMap<PartitionKey, ByteBuffer> items = new TreeMap<>();
+            Key[] keys = new Key[size];
+            for (int i=0; i<size; i++)
+                keys[i] = PartitionKey.serializer.deserialize(in, version);
+
+            ByteBuffer[] serialized = new ByteBuffer[size];
             for (int i=0; i<size; i++)
-                items.put(PartitionKey.serializer.deserialize(in, version), ByteBufferUtil.readWithVIntLength(in));
-            return factory.apply(items);
+                serialized[i] = ByteBufferUtil.readWithVIntLength(in);
+            return factory.apply(Keys.ofSorted(keys), serialized);
         }
 
         @Override
         public long serializedSize(S items, int version)
         {
-            long size = TypeSizes.sizeofUnsignedVInt(items.serialized.size());
-            for (Map.Entry<PartitionKey, ByteBuffer> entry : items.serialized.entrySet())
-            {
-                size += PartitionKey.serializer.serializedSize(entry.getKey());
-                size += ByteBufferUtil.serializedSizeWithVIntLength(entry.getValue());
-            }
+            long size = TypeSizes.sizeofUnsignedVInt(items.serialized.length);
+            for (Key key : items.keys) size += PartitionKey.serializer.serializedSize((PartitionKey) key, version);
+            for (ByteBuffer buffer : items.serialized) size += ByteBufferUtil.serializedSizeWithVIntLength(buffer);
             return size;
         }
     }
diff --git a/src/java/org/apache/cassandra/service/accord/db/AccordData.java b/src/java/org/apache/cassandra/service/accord/db/AccordData.java
index 40de220e7b..b5feba8901 100644
--- a/src/java/org/apache/cassandra/service/accord/db/AccordData.java
+++ b/src/java/org/apache/cassandra/service/accord/db/AccordData.java
@@ -20,15 +20,17 @@ package org.apache.cassandra.service.accord.db;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
-import java.util.NavigableMap;
 import java.util.TreeMap;
 
 import com.google.common.base.Preconditions;
 
 import accord.api.Data;
 import accord.api.Result;
+import accord.primitives.Keys;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.partitions.FilteredPartition;
 import org.apache.cassandra.db.rows.DeserializationHelper;
@@ -47,22 +49,16 @@ import org.apache.cassandra.utils.ObjectSizes;
 
 public class AccordData extends AbstractKeyIndexed<FilteredPartition> implements Data, Result, Iterable<FilteredPartition>
 {
-    private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new AccordData());
+    private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new AccordData(Collections.emptyList()));
 
     private static PartitionKey getKey(FilteredPartition partition)
     {
         return new PartitionKey(partition.metadata().id, partition.partitionKey());
     }
 
-    public AccordData()
-    {
-        this(new TreeMap<>());
-    }
-
     public AccordData(FilteredPartition partition)
     {
-        this();
-        put(partition);
+        this(Keys.of(AccordKey.of(partition)), new ByteBuffer[] { serialize(partition, partitionSerializer) });
     }
 
     public AccordData(List<FilteredPartition> items)
@@ -70,12 +66,11 @@ public class AccordData extends AbstractKeyIndexed<FilteredPartition> implements
         super(items, AccordData::getKey);
     }
 
-    public AccordData(NavigableMap<PartitionKey, ByteBuffer> serialized)
+    public AccordData(Keys keys, ByteBuffer[] serialized)
     {
-        super(serialized);
+        super(keys, serialized);
     }
 
-    @Override
     void serialize(FilteredPartition partition, DataOutputPlus out, int version) throws IOException
     {
         partitionSerializer.serialize(partition, out, version);
@@ -99,23 +94,6 @@ public class AccordData extends AbstractKeyIndexed<FilteredPartition> implements
         return EMPTY_SIZE;
     }
 
-    private void put(PartitionKey key, ByteBuffer bytes)
-    {
-        // TODO: support multiple partitions (ie: read commands) per partition
-        Preconditions.checkArgument(!serialized.containsKey(key) || serialized.get(key).equals(bytes));
-        serialized.put(key, bytes);
-    }
-
-    void put(PartitionKey key, FilteredPartition partition)
-    {
-        put(key, serialize(partition));
-    }
-
-    void put(FilteredPartition partition)
-    {
-        put(AccordKey.of(partition), partition);
-    }
-
     FilteredPartition get(PartitionKey key)
     {
         return getDeserialized(key);
@@ -124,18 +102,13 @@ public class AccordData extends AbstractKeyIndexed<FilteredPartition> implements
     @Override
     public Iterator<FilteredPartition> iterator()
     {
-        return serialized.values().stream().map(this::deserialize).iterator();
+        return Arrays.stream(serialized).map(this::deserialize).iterator();
     }
 
     @Override
     public Data merge(Data data)
     {
-        AccordData that = (AccordData) data;
-        AccordData merged = new AccordData();
-        //TODO on conflict should we "merge" the partition rather than override?
-        this.serialized.forEach(merged::put);
-        that.serialized.forEach(merged::put);
-        return merged;
+        return super.merge((AccordData) data, AccordData::new);
     }
 
     public static Data merge(Data left, Data right)
diff --git a/src/java/org/apache/cassandra/service/accord/db/AccordQuery.java b/src/java/org/apache/cassandra/service/accord/db/AccordQuery.java
index fe79a353d3..5ec9271e7a 100644
--- a/src/java/org/apache/cassandra/service/accord/db/AccordQuery.java
+++ b/src/java/org/apache/cassandra/service/accord/db/AccordQuery.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.service.accord.db;
 
 import java.io.IOException;
+import java.util.Collections;
 
 import javax.annotation.Nullable;
 
@@ -29,6 +30,7 @@ import accord.api.Query;
 import accord.api.Read;
 import accord.api.Result;
 import accord.api.Update;
+import accord.primitives.TxnId;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -40,18 +42,18 @@ public abstract class AccordQuery implements Query
     public static final AccordQuery ALL = new AccordQuery()
     {
         @Override
-        public Result compute(Data data, @Nullable Read read, @Nullable Update update)
+        public Result compute(TxnId txnId, Data data, @Nullable Read read, @Nullable Update update)
         {
-            return data != null ? (AccordData) data : new AccordData();
+            return data != null ? (AccordData) data : new AccordData(Collections.emptyList());
         }
     };
 
     public static final AccordQuery NONE = new AccordQuery()
     {
         @Override
-        public Result compute(Data data, @Nullable Read read, @Nullable Update update)
+        public Result compute(TxnId txnId, Data data, @Nullable Read read, @Nullable Update update)
         {
-            return new AccordData();
+            return new AccordData(Collections.emptyList());
         }
     };
 
@@ -69,21 +71,27 @@ public abstract class AccordQuery implements Query
         @Override
         public void serialize(AccordQuery query, DataOutputPlus out, int version) throws IOException
         {
-            Preconditions.checkArgument(query == ALL || query == NONE);
-            out.writeBoolean(query == ALL);
+            Preconditions.checkArgument(query == null || query == ALL || query == NONE);
+            out.writeByte(query == null ? 0 : query == ALL ? 1 : 2);
         }
 
         @Override
         public AccordQuery deserialize(DataInputPlus in, int version) throws IOException
         {
-            return in.readBoolean() ? ALL : NONE;
+            switch (in.readByte())
+            {
+                default: throw new AssertionError();
+                case 0: return null;
+                case 1: return ALL;
+                case 2: return NONE;
+            }
         }
 
         @Override
         public long serializedSize(AccordQuery query, int version)
         {
-            Preconditions.checkArgument(query == ALL || query == NONE);
-            return TypeSizes.sizeof(query == ALL);
+            Preconditions.checkArgument(query == null || query == ALL || query == NONE);
+            return TypeSizes.sizeof((byte)2);
         }
     };
 }
diff --git a/src/java/org/apache/cassandra/service/accord/db/AccordRead.java b/src/java/org/apache/cassandra/service/accord/db/AccordRead.java
index 55fe14d665..a0a7532df3 100644
--- a/src/java/org/apache/cassandra/service/accord/db/AccordRead.java
+++ b/src/java/org/apache/cassandra/service/accord/db/AccordRead.java
@@ -22,20 +22,20 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-import com.google.common.annotations.VisibleForTesting;
 
 import accord.api.Data;
 import accord.api.DataStore;
 import accord.api.Key;
 import accord.api.Read;
 import accord.local.CommandStore;
+import accord.local.SafeCommandStore;
+import accord.primitives.KeyRanges;
 import accord.primitives.Keys;
 import accord.primitives.Timestamp;
+import accord.primitives.Txn;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.db.ReadExecutionController;
 import org.apache.cassandra.db.SinglePartitionReadCommand;
@@ -56,16 +56,16 @@ import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 
 public class AccordRead extends AbstractKeyIndexed<SinglePartitionReadCommand> implements Read
 {
-    private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new AccordRead(new TreeMap<>()));
+    private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new AccordRead(Collections.emptyList()));
 
     public AccordRead(List<SinglePartitionReadCommand> items)
     {
         super(items, AccordKey::of);
     }
 
-    public AccordRead(NavigableMap<PartitionKey, ByteBuffer> serialized)
+    public AccordRead(Keys keys, ByteBuffer[] serialized)
     {
-        super(serialized);
+        super(keys, serialized);
     }
 
     @Override
@@ -95,7 +95,7 @@ public class AccordRead extends AbstractKeyIndexed<SinglePartitionReadCommand> i
     @Override
     public Keys keys()
     {
-        return new Keys(serialized.keySet());
+        return keys;
     }
 
     public String toString()
@@ -104,14 +104,14 @@ public class AccordRead extends AbstractKeyIndexed<SinglePartitionReadCommand> i
     }
 
     @Override
-    public Future<Data> read(Key key, boolean isForWriteTxn, CommandStore commandStore, Timestamp executeAt, DataStore store)
+    public Future<Data> read(Key key, Txn.Kind kind, SafeCommandStore safeStore, Timestamp executeAt, DataStore store)
     {
         SinglePartitionReadCommand command = getDeserialized((PartitionKey) key);
         if (command == null)
-            return ImmediateFuture.success(new AccordData());
+            return ImmediateFuture.success(new AccordData(Collections.emptyList()));
 
-        AccordCommandsForKey cfk = (AccordCommandsForKey) commandStore.commandsForKey(key);
-        int nowInSeconds = cfk.nowInSecondsFor(executeAt, isForWriteTxn);
+        AccordCommandsForKey cfk = (AccordCommandsForKey) safeStore.commandsForKey(key);
+        int nowInSeconds = cfk.nowInSecondsFor(executeAt, kind.isWrite());
         Future<Data> future = Stage.READ.submit(() -> {
             SinglePartitionReadCommand read = command.withNowInSec(nowInSeconds);
             try (ReadExecutionController controller = read.executionController();
@@ -127,6 +127,18 @@ public class AccordRead extends AbstractKeyIndexed<SinglePartitionReadCommand> i
         return future;
     }
 
+    @Override
+    public Read slice(KeyRanges ranges)
+    {
+        return super.slice(ranges, AccordRead::new);
+    }
+
+    @Override
+    public Read merge(Read read)
+    {
+        return super.merge((AccordRead)read, AccordRead::new);
+    }
+
     public static AccordRead forCommands(Collection<SinglePartitionReadCommand> commands)
     {
         List<SinglePartitionReadCommand> reads = new ArrayList<>(commands);
diff --git a/src/java/org/apache/cassandra/service/accord/db/AccordUpdate.java b/src/java/org/apache/cassandra/service/accord/db/AccordUpdate.java
index 7abaeb261b..6ea2c4de41 100644
--- a/src/java/org/apache/cassandra/service/accord/db/AccordUpdate.java
+++ b/src/java/org/apache/cassandra/service/accord/db/AccordUpdate.java
@@ -22,14 +22,16 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Objects;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.function.Function;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
@@ -40,6 +42,7 @@ import accord.api.Data;
 import accord.api.Key;
 import accord.api.Update;
 import accord.api.Write;
+import accord.primitives.KeyRanges;
 import accord.primitives.Keys;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.Clustering;
@@ -74,14 +77,16 @@ import org.apache.cassandra.service.accord.serializers.KeySerializers;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 
+import static java.lang.Math.toIntExact;
 import static org.apache.cassandra.utils.CollectionSerializer.deserializeMap;
 import static org.apache.cassandra.utils.CollectionSerializer.serializeMap;
 import static org.apache.cassandra.utils.CollectionSerializer.serializedSizeMap;
 
 public class AccordUpdate implements Update
 {
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new AccordUpdate(Keys.EMPTY, (ByteBuffer[]) null, (ByteBuffer[]) null));;
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new AccordUpdate(Keys.EMPTY, null, null));
 
+    // TODO (soon): extends AbstractKeyIndexed; pack updates+predicates into one ByteBuffer
     private final Keys keys;
     private final ByteBuffer[] updates;
     private final ByteBuffer[] predicates;
@@ -200,7 +205,6 @@ public class AccordUpdate implements Update
         @Override
         protected void serializeBody(DataOutputPlus out, int version)
         {
-
         }
 
         @Override
@@ -338,17 +342,18 @@ public class AccordUpdate implements Update
 
         public static final IVersionedSerializer<AbstractUpdate> serializer = new IVersionedSerializer<AbstractUpdate>()
         {
+            final Kind[] kinds = Kind.values();
             @Override
             public void serialize(AbstractUpdate updater, DataOutputPlus out, int version) throws IOException
             {
-                out.writeInt(updater.kind().ordinal());
+                out.writeUnsignedVInt(updater.kind().ordinal());
                 updater.serializer().serializeBody(updater, out, version);
             }
 
             @Override
             public AbstractUpdate deserialize(DataInputPlus in, int version) throws IOException
             {
-                Kind kind = Kind.values()[in.readInt()];
+                Kind kind = kinds[(int)in.readUnsignedVInt()];
                 switch (kind)
                 {
                     case SIMPLE:
@@ -364,7 +369,8 @@ public class AccordUpdate implements Update
             @Override
             public long serializedSize(AbstractUpdate updater, int version)
             {
-                return TypeSizes.INT_SIZE + updater.serializer().serializedBodySize(updater, version);
+                return TypeSizes.sizeofUnsignedVInt(updater.kind().ordinal())
+                       + updater.serializer().serializedBodySize(updater, version);
             }
         };
     }
@@ -689,22 +695,13 @@ public class AccordUpdate implements Update
         };
     }
 
-    private static Keys keysFrom(List<AbstractUpdate> updates, List<UpdatePredicate> predicates)
-    {
-        Set<Key> keys = new HashSet<>();
-        for (AbstractUpdate update : updates)
-            keys.add(update.partitionKey());
-        for (UpdatePredicate predicate : predicates)
-            keys.add(new PartitionKey(predicate.table.id, predicate.key));
-
-        return new Keys(keys);
-    }
-
     public AccordUpdate(List<AbstractUpdate> updates, List<UpdatePredicate> predicates)
     {
-        this.keys = keysFrom(updates, predicates);
-        this.updates = serialize(updates, AbstractUpdate.serializer);
-        this.predicates = serialize(predicates, predicateSerializer);
+        updates.sort(Comparator.comparing(AbstractUpdate::partitionKey));
+        predicates.sort(Comparator.comparing(UpdatePredicate::partitionKey));
+        this.keys = Keys.of(updates, AbstractUpdate::partitionKey).union(Keys.of(predicates, UpdatePredicate::partitionKey));
+        this.updates = toSerializedValuesArray(keys, updates, AbstractUpdate::partitionKey, AbstractUpdate.serializer);
+        this.predicates = toSerializedValuesArray(keys, predicates, UpdatePredicate::partitionKey, predicateSerializer);
     }
 
     public AccordUpdate(Keys keys, ByteBuffer[] updates, ByteBuffer[] predicates)
@@ -747,12 +744,18 @@ public class AccordUpdate implements Update
     @Override
     public Write apply(Data data)
     {
-        AccordData read = data != null ? (AccordData) data : new AccordData();
-        for (ByteBuffer bytes : predicates)
+        AccordData read = data != null ? (AccordData) data : new AccordData(Collections.emptyList());
+        for (int i = 0 ; i < predicates.length ; ++i)
         {
-            UpdatePredicate predicate = deserialize(bytes, predicateSerializer);
-            if (!predicate.applies(read.get(predicate.partitionKey())))
-                return AccordWrite.EMPTY;
+            if (predicates[i] == null)
+                continue;
+
+            List<UpdatePredicate> test = deserialize(predicates[i], predicateSerializer);
+            for (int j = 0, mj = test.size() ; j < mj ; ++j)
+            {
+                if (!test.get(j).applies(read.get((PartitionKey) keys.get(i))))
+                    return AccordWrite.EMPTY;
+            }
         }
         NavigableMap<PartitionKey, PartitionUpdate> updateMap = new TreeMap<>();
         for (AbstractUpdate updater : deserialize(updates, AbstractUpdate.serializer))
@@ -766,7 +769,54 @@ public class AccordUpdate implements Update
         return new AccordWrite(new ArrayList<>(updateMap.values()));
     }
 
-    UpdatePredicate getPredicate(int i)
+    @Override
+    public Update slice(KeyRanges ranges)
+    {
+        Keys keys = this.keys.slice(ranges);
+        return new AccordUpdate(keys, select(this.keys, keys, updates), select(this.keys, keys, predicates));
+    }
+
+    private static ByteBuffer[] select(Keys in, Keys out, ByteBuffer[] from)
+    {
+        ByteBuffer[] result = new ByteBuffer[out.size()];
+        int j = 0;
+        for (int i = 0 ; i < in.size() ; ++i)
+        {
+            j = in.findNext(out.get(i), j);
+            result[i] = from[j];
+        }
+        return result;
+    }
+
+    @Override
+    public Update merge(Update update)
+    {
+        // TODO: special method for linear merging keyed and non-keyed lists simultaneously
+        AccordUpdate that = (AccordUpdate) update;
+        Keys keys = this.keys.union(that.keys);
+        ByteBuffer[] updates = merge(this.keys, that.keys, this.updates, that.updates, keys.size());
+        ByteBuffer[] predicates = merge(this.keys, that.keys, this.predicates, that.predicates, keys.size());
+        return new AccordUpdate(keys, updates, predicates);
+    }
+
+    private static ByteBuffer[] merge(Keys leftKeys, Keys rightKeys, ByteBuffer[] left, ByteBuffer[] right, int outputSize)
+    {
+        ByteBuffer[] out = new ByteBuffer[outputSize];
+        int l = 0, r = 0, o = 0;
+        while (l < leftKeys.size() && r < rightKeys.size())
+        {
+            int c = leftKeys.get(l).compareTo(rightKeys.get(r));
+            if (c < 0) { out[o++] = left[l++]; }
+            else if (c > 0) { out[o++] = right[r++]; }
+            else if (ByteBufferUtil.compareUnsigned(left[l], right[r]) != 0) { throw new IllegalStateException("The same keys have different values in each input"); }
+            else { out[o++] = left[l++]; r++; }
+        }
+        while (l < leftKeys.size()) { out[o++] = left[l]; }
+        while (r < rightKeys.size()) { out[o++] = right[r++]; }
+        return out;
+    }
+
+    List<UpdatePredicate> getPredicate(int i)
     {
         return deserialize(predicates[i], predicateSerializer);
     }
@@ -837,11 +887,8 @@ public class AccordUpdate implements Update
         public void serialize(AccordUpdate update, DataOutputPlus out, int version) throws IOException
         {
             KeySerializers.keys.serialize(update.keys, out, version);
-            out.writeInt(update.updates.length);
             for (ByteBuffer buffer : update.updates)
                 ByteBufferUtil.writeWithVIntLength(buffer, out);
-
-            out.writeInt(update.predicates.length);
             for (ByteBuffer buffer : update.predicates)
                 ByteBufferUtil.writeWithVIntLength(buffer, out);
         }
@@ -850,14 +897,13 @@ public class AccordUpdate implements Update
         public AccordUpdate deserialize(DataInputPlus in, int version) throws IOException
         {
             Keys keys = KeySerializers.keys.deserialize(in, version);
-            int numUpdate = in.readInt();
-            ByteBuffer[] updates = new ByteBuffer[numUpdate];
-            for (int i=0; i<numUpdate; i++)
+
+            ByteBuffer[] updates = new ByteBuffer[keys.size()];
+            for (int i=0; i<keys.size(); i++)
                 updates[i] = ByteBufferUtil.readWithVIntLength(in);
 
-            int numPredicate = in.readInt();
-            ByteBuffer[] predicates = new ByteBuffer[numPredicate];
-            for (int i=0; i<numPredicate; i++)
+            ByteBuffer[] predicates = new ByteBuffer[keys.size()];
+            for (int i=0; i<keys.size(); i++)
                 predicates[i] = ByteBufferUtil.readWithVIntLength(in);
 
             return new AccordUpdate(keys, updates, predicates);
@@ -867,27 +913,26 @@ public class AccordUpdate implements Update
         public long serializedSize(AccordUpdate update, int version)
         {
             long size = KeySerializers.keys.serializedSize(update.keys, version);
-
-            size += TypeSizes.sizeof(update.updates.length);
             for (ByteBuffer buffer : update.updates)
                 size += ByteBufferUtil.serializedSizeWithVIntLength(buffer);
-
-            size += TypeSizes.sizeof(update.predicates.length);
             for (ByteBuffer buffer : update.predicates)
                 size += ByteBufferUtil.serializedSizeWithVIntLength(buffer);
-
             return size;
         }
     };
 
-    private static <T> ByteBuffer serialize(T item, IVersionedSerializer<T> serializer)
+    private static <T> ByteBuffer toSerializedValues(List<T> items, int start, int end, IVersionedSerializer<T> serializer, int version)
     {
-        int version = MessagingService.current_version;
-        long size = serializer.serializedSize(item, version) + TypeSizes.INT_SIZE;
+        long size = TypeSizes.sizeofUnsignedVInt(version) + TypeSizes.sizeofUnsignedVInt(end - start);
+        for (int i = start ; i < end ; ++i)
+            size += serializer.serializedSize(items.get(i), version);
+
         try (DataOutputBuffer out = new DataOutputBuffer((int) size))
         {
-            out.writeInt(version);
-            serializer.serialize(item, out, version);
+            out.writeUnsignedVInt(version);
+            out.writeUnsignedVInt(end - start);
+            for (int i = start ; i < end ; ++i)
+                serializer.serialize(items.get(i), out, version);
             return out.buffer(false);
         }
         catch (IOException e)
@@ -896,20 +941,46 @@ public class AccordUpdate implements Update
         }
     }
 
-    private static <T> ByteBuffer[] serialize(List<T> items, IVersionedSerializer<T> serializer)
+    private static <T> ByteBuffer[] toSerializedValuesArray(Keys keys, List<T> items, Function<? super T, ? extends Key> toKey, IVersionedSerializer<T> serializer)
     {
-        ByteBuffer[] result = new ByteBuffer[items.size()];
-        for (int i=0,mi=items.size(); i<mi; i++)
-            result[i] = serialize(items.get(i), serializer);
+        ByteBuffer[] result = new ByteBuffer[keys.size()];
+        int i = 0, mi = items.size(), ki = 0;
+        while (i < mi)
+        {
+            Key key = toKey.apply(items.get(i));
+            int j = i + 1;
+            while (j < mi && toKey.apply(items.get(j)).equals(key))
+                ++j;
+
+            int nextki = keys.findNext(key, ki);
+            Arrays.fill(result, ki, nextki, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            ki = nextki;
+            result[ki++] = toSerializedValues(items, i, j, serializer, MessagingService.current_version);
+            i = j;
+        }
+        Arrays.fill(result, ki, result.length, ByteBufferUtil.EMPTY_BYTE_BUFFER);
         return result;
     }
 
-    private static <T> T deserialize(ByteBuffer bytes, IVersionedSerializer<T> serializer)
+    private static <T> List<T> deserialize(ByteBuffer bytes, IVersionedSerializer<T> serializer)
     {
+        if (!bytes.hasRemaining())
+            return Collections.emptyList();
+
         try (DataInputBuffer in = new DataInputBuffer(bytes, true))
         {
-            int version = in.readInt();
-            return serializer.deserialize(in, version);
+            int version = toIntExact(in.readUnsignedVInt());
+            int count = toIntExact(in.readUnsignedVInt());
+            switch (count)
+            {
+                case 0: throw new IllegalStateException();
+                case 1: return Collections.singletonList(serializer.deserialize(in, version));
+                default:
+                    List<T> result = new ArrayList<>();
+                    for (int i = 0 ; i < count ; ++i)
+                        result.add(serializer.deserialize(in, version));
+                    return result;
+            }
         }
         catch (IOException e)
         {
@@ -921,7 +992,7 @@ public class AccordUpdate implements Update
     {
         List<T> result = new ArrayList<>(buffers.length);
         for (ByteBuffer bytes : buffers)
-            result.add(deserialize(bytes, serializer));
+            result.addAll(deserialize(bytes, serializer));
         return result;
     }
 }
diff --git a/src/java/org/apache/cassandra/service/accord/db/AccordWrite.java b/src/java/org/apache/cassandra/service/accord/db/AccordWrite.java
index 1b8526a4b7..5584b7c3cc 100644
--- a/src/java/org/apache/cassandra/service/accord/db/AccordWrite.java
+++ b/src/java/org/apache/cassandra/service/accord/db/AccordWrite.java
@@ -21,14 +21,14 @@ package org.apache.cassandra.service.accord.db;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
-import java.util.NavigableMap;
 
 import com.google.common.collect.ImmutableList;
 
 import accord.api.DataStore;
 import accord.api.Key;
 import accord.api.Write;
-import accord.local.CommandStore;
+import accord.local.SafeCommandStore;
+import accord.primitives.Keys;
 import accord.primitives.Timestamp;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.db.Mutation;
@@ -55,9 +55,9 @@ public class AccordWrite extends AbstractKeyIndexed<PartitionUpdate> implements
         super(items, AccordKey::of);
     }
 
-    public AccordWrite(NavigableMap<PartitionKey, ByteBuffer> items)
+    public AccordWrite(Keys keys, ByteBuffer[] serialized)
     {
-        super(items);
+        super(keys, serialized);
     }
 
     @Override
@@ -85,12 +85,12 @@ public class AccordWrite extends AbstractKeyIndexed<PartitionUpdate> implements
     }
 
     @Override
-    public Future<Void> apply(Key key, CommandStore commandStore, Timestamp executeAt, DataStore store)
+    public Future<Void> apply(Key key, SafeCommandStore safeStore, Timestamp executeAt, DataStore store)
     {
         PartitionUpdate update = getDeserialized((PartitionKey) key);
         if (update == null)
             return SUCCESS;
-        AccordCommandsForKey cfk = (AccordCommandsForKey) commandStore.commandsForKey(key);
+        AccordCommandsForKey cfk = (AccordCommandsForKey) safeStore.commandsForKey(key);
         long timestamp = cfk.timestampMicrosFor(executeAt, true);
         int nowInSeconds = cfk.nowInSecondsFor(executeAt, true);
         update = new PartitionUpdate.Builder(update, 0).updateAllTimestampAndLocalDeletionTime(timestamp, nowInSeconds).build();
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java
index 7760c81e7d..c2d5d88928 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java
@@ -24,13 +24,14 @@ import accord.messages.Accept;
 import accord.messages.Accept.AcceptNack;
 import accord.messages.Accept.AcceptOk;
 import accord.messages.Accept.AcceptReply;
-import accord.primitives.Keys;
+import accord.primitives.PartialRoute;
 import accord.primitives.TxnId;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
+import static accord.local.Command.AcceptOutcome.RejectedBallot;
 import static accord.messages.Accept.SerializerSupport.create;
 
 public class AcceptSerializers
@@ -43,31 +44,31 @@ public class AcceptSerializers
         public void serializeBody(Accept accept, DataOutputPlus out, int version) throws IOException
         {
             CommandSerializers.ballot.serialize(accept.ballot, out, version);
-            KeySerializers.key.serialize(accept.homeKey, out, version);
-            CommandSerializers.txn.serialize(accept.txn, out, version);
             CommandSerializers.timestamp.serialize(accept.executeAt, out, version);
-            CommandSerializers.deps.serialize(accept.deps, out, version);
+            KeySerializers.keys.serialize(accept.keys, out, version);
+            DepsSerializer.partialDeps.serialize(accept.partialDeps, out, version);
+            CommandSerializers.kind.serialize(accept.kind, out, version);
         }
 
         @Override
-        public Accept deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch, TxnId txnId, long minEpoch) throws IOException
+        public Accept deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey) throws IOException
         {
-            return create(scope, waitForEpoch, txnId,
+            return create(txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey,
                           CommandSerializers.ballot.deserialize(in, version),
-                          KeySerializers.key.deserialize(in, version),
-                          CommandSerializers.txn.deserialize(in, version),
                           CommandSerializers.timestamp.deserialize(in, version),
-                          CommandSerializers.deps.deserialize(in, version));
+                          KeySerializers.keys.deserialize(in, version),
+                          DepsSerializer.partialDeps.deserialize(in, version),
+                          CommandSerializers.kind.deserialize(in, version));
         }
 
         @Override
         public long serializedBodySize(Accept accept, int version)
         {
             return CommandSerializers.ballot.serializedSize(accept.ballot, version)
-                   + KeySerializers.key.serializedSize(accept.homeKey, version)
-                   + CommandSerializers.txn.serializedSize(accept.txn, version)
                    + CommandSerializers.timestamp.serializedSize(accept.executeAt, version)
-                   + CommandSerializers.deps.serializedSize(accept.deps, version);
+                   + KeySerializers.keys.serializedSize(accept.keys, version)
+                   + DepsSerializer.partialDeps.serializedSize(accept.partialDeps, version)
+                   + CommandSerializers.kind.serializedSize(accept.kind, version);
         }
     };
 
@@ -78,7 +79,7 @@ public class AcceptSerializers
         {
             CommandSerializers.ballot.serialize(invalidate.ballot, out, version);
             CommandSerializers.txnId.serialize(invalidate.txnId, out, version);
-            KeySerializers.key.serialize(invalidate.someKey, out, version);
+            KeySerializers.routingKey.serialize(invalidate.someKey, out, version);
         }
 
         @Override
@@ -86,7 +87,7 @@ public class AcceptSerializers
         {
             return new Accept.Invalidate(CommandSerializers.ballot.deserialize(in, version),
                                          CommandSerializers.txnId.deserialize(in, version),
-                                         KeySerializers.key.deserialize(in, version));
+                                         KeySerializers.routingKey.deserialize(in, version));
         }
 
         @Override
@@ -94,56 +95,7 @@ public class AcceptSerializers
         {
             return CommandSerializers.ballot.serializedSize(invalidate.ballot, version)
                    + CommandSerializers.txnId.serializedSize(invalidate.txnId, version)
-                   + KeySerializers.key.serializedSize(invalidate.someKey, version);
-        }
-    };
-
-    private static final IVersionedSerializer<AcceptOk> acceptOk = new IVersionedSerializer<AcceptOk>()
-    {
-        @Override
-        public void serialize(AcceptOk acceptOk, DataOutputPlus out, int version) throws IOException
-        {
-            CommandSerializers.txnId.serialize(acceptOk.txnId, out, version);
-            NullableSerializer.serializeNullable(acceptOk.deps, out, version, CommandSerializers.deps);
-
-        }
-
-        @Override
-        public AcceptOk deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return new AcceptOk(CommandSerializers.txnId.deserialize(in, version),
-                                NullableSerializer.deserializeNullable(in, version, CommandSerializers.deps));
-        }
-
-        @Override
-        public long serializedSize(AcceptOk acceptOk, int version)
-        {
-            return CommandSerializers.txnId.serializedSize(acceptOk.txnId, version)
-                   + NullableSerializer.serializedSizeNullable(acceptOk.deps, version, CommandSerializers.deps);
-        }
-    };
-
-    private static final IVersionedSerializer<AcceptNack> acceptNack = new IVersionedSerializer<AcceptNack>()
-    {
-        @Override
-        public void serialize(AcceptNack acceptNack, DataOutputPlus out, int version) throws IOException
-        {
-            CommandSerializers.txnId.serialize(acceptNack.txnId, out, version);
-            CommandSerializers.timestamp.serialize(acceptNack.reject, out, version);
-        }
-
-        @Override
-        public AcceptNack deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return new AcceptNack(CommandSerializers.txnId.deserialize(in, version),
-                                  CommandSerializers.timestamp.deserialize(in, version));
-        }
-
-        @Override
-        public long serializedSize(AcceptNack acceptNack, int version)
-        {
-            return CommandSerializers.txnId.serializedSize(acceptNack.txnId, version)
-                 + CommandSerializers.timestamp.serializedSize(acceptNack.reject, version);
+                   + KeySerializers.routingKey.serializedSize(invalidate.someKey, version);
         }
     };
 
@@ -152,27 +104,53 @@ public class AcceptSerializers
         @Override
         public void serialize(AcceptReply reply, DataOutputPlus out, int version) throws IOException
         {
-            out.writeBoolean(reply.isOK());
-            if (reply.isOK())
-                acceptOk.serialize((AcceptOk) reply, out, version);
-            else
-                acceptNack.serialize((AcceptNack) reply, out, version);
+            switch (reply.outcome())
+            {
+                default: throw new AssertionError();
+                case Success:
+                    out.writeByte(1);
+                    DepsSerializer.partialDeps.serialize(((AcceptOk)reply).deps, out, version);
+                    break;
+                case Redundant:
+                    out.writeByte(2);
+                    break;
+                case RejectedBallot:
+                    out.writeByte(3);
+                    CommandSerializers.ballot.serialize(((AcceptNack) reply).supersededBy, out, version);
+            }
         }
 
         @Override
         public AcceptReply deserialize(DataInputPlus in, int version) throws IOException
         {
-            return in.readBoolean() ? acceptOk.deserialize(in, version) : acceptNack.deserialize(in, version);
+            int type = in.readByte();
+            switch (type)
+            {
+                default: throw new IllegalStateException("Unexpected AcceptNack type: " + type);
+                case 1:
+                    return new AcceptOk(DepsSerializer.partialDeps.deserialize(in, version));
+                case 2:
+                    return AcceptNack.REDUNDANT;
+                case 3:
+                    return new AcceptNack(RejectedBallot, CommandSerializers.ballot.deserialize(in, version));
+            }
         }
 
         @Override
         public long serializedSize(AcceptReply reply, int version)
         {
-            long size = TypeSizes.sizeof(reply.isOK());
-            if (reply.isOK())
-                size += acceptOk.serializedSize((AcceptOk) reply, version);
-            else
-                size += acceptNack.serializedSize((AcceptNack) reply, version);
+            long size = TypeSizes.BYTE_SIZE;
+            switch (reply.outcome())
+            {
+                default: throw new AssertionError();
+                case Success:
+                    size += DepsSerializer.partialDeps.serializedSize(((AcceptOk)reply).deps, version);
+                    break;
+                case Redundant:
+                    break;
+                case RejectedBallot:
+                    size += CommandSerializers.ballot.serializedSize(((AcceptNack) reply).supersededBy, version);
+            }
             return size;
         }
     };
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/ApplyAndCheckSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/ApplyAndCheckSerializers.java
deleted file mode 100644
index ee63735cc3..0000000000
--- a/src/java/org/apache/cassandra/service/accord/serializers/ApplyAndCheckSerializers.java
+++ /dev/null
@@ -1,99 +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.service.accord.serializers;
-
-import java.io.IOException;
-
-import com.google.common.collect.Sets;
-
-import accord.impl.SimpleProgressLog.ApplyAndCheck;
-import accord.impl.SimpleProgressLog.ApplyAndCheckOk;
-import accord.primitives.Keys;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.service.accord.db.AccordData;
-import org.apache.cassandra.utils.CollectionSerializer;
-
-public class ApplyAndCheckSerializers
-{
-    public static final IVersionedSerializer<ApplyAndCheck> request = new TxnRequestSerializer<ApplyAndCheck>()
-    {
-        @Override
-        public void serializeBody(ApplyAndCheck apply, DataOutputPlus out, int version) throws IOException
-        {
-            CommandSerializers.txnId.serialize(apply.txnId, out, version);
-            CommandSerializers.txn.serialize(apply.txn, out, version);
-            KeySerializers.key.serialize(apply.homeKey, out, version);
-            CommandSerializers.timestamp.serialize(apply.executeAt, out, version);
-            CommandSerializers.deps.serialize(apply.deps, out, version);
-            CommandSerializers.writes.serialize(apply.writes, out, version);
-            AccordData.serializer.serialize((AccordData) apply.result, out, version);
-            CollectionSerializer.serializeCollection(TopologySerializers.nodeId, apply.notPersisted, out, version);
-        }
-
-        @Override
-        public ApplyAndCheck deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch) throws IOException
-        {
-            return new ApplyAndCheck(scope, waitForEpoch,
-                                     CommandSerializers.txnId.deserialize(in, version),
-                                     CommandSerializers.txn.deserialize(in, version),
-                                     KeySerializers.key.deserialize(in, version),
-                                     CommandSerializers.timestamp.deserialize(in, version),
-                                     CommandSerializers.deps.deserialize(in, version),
-                                     CommandSerializers.writes.deserialize(in, version),
-                                     AccordData.serializer.deserialize(in, version),
-                                     CollectionSerializer.deserializeCollection(TopologySerializers.nodeId, Sets::newHashSetWithExpectedSize, in, version));
-        }
-
-        @Override
-        public long serializedBodySize(ApplyAndCheck apply, int version)
-        {
-            return CommandSerializers.txnId.serializedSize(apply.txnId, version)
-                   + CommandSerializers.txn.serializedSize(apply.txn, version)
-                   + KeySerializers.key.serializedSize(apply.homeKey, version)
-                   + CommandSerializers.timestamp.serializedSize(apply.executeAt, version)
-                   + CommandSerializers.deps.serializedSize(apply.deps, version)
-                   + CommandSerializers.writes.serializedSize(apply.writes, version)
-                   + AccordData.serializer.serializedSize((AccordData) apply.result, version)
-                   + CollectionSerializer.serializedSizeCollection(TopologySerializers.nodeId, apply.notPersisted, version);
-        }
-    };
-
-    public static final IVersionedSerializer<ApplyAndCheckOk> reply = new IVersionedSerializer<ApplyAndCheckOk>()
-    {
-        @Override
-        public void serialize(ApplyAndCheckOk reply, DataOutputPlus out, int version) throws IOException
-        {
-            CollectionSerializer.serializeCollection(TopologySerializers.nodeId, reply.notPersisted, out, version);
-        }
-
-        @Override
-        public ApplyAndCheckOk deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return new ApplyAndCheckOk(CollectionSerializer.deserializeCollection(TopologySerializers.nodeId, Sets::newHashSetWithExpectedSize, in, version));
-        }
-
-        @Override
-        public long serializedSize(ApplyAndCheckOk reply, int version)
-        {
-            return CollectionSerializer.serializedSizeCollection(TopologySerializers.nodeId, reply.notPersisted, version);
-        }
-    };
-}
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/ApplySerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/ApplySerializers.java
index 8282e61c19..8e2a0cdf49 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/ApplySerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/ApplySerializers.java
@@ -21,7 +21,9 @@ package org.apache.cassandra.service.accord.serializers;
 import java.io.IOException;
 
 import accord.messages.Apply;
-import accord.primitives.Keys;
+import accord.primitives.PartialRoute;
+import accord.primitives.TxnId;
+import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -34,24 +36,19 @@ public class ApplySerializers
         @Override
         public void serializeBody(Apply apply, DataOutputPlus out, int version) throws IOException
         {
-            CommandSerializers.txnId.serialize(apply.txnId, out, version);
-            CommandSerializers.txn.serialize(apply.txn, out, version);
-            KeySerializers.key.serialize(apply.homeKey, out, version);
+            out.writeUnsignedVInt(apply.untilEpoch);
             CommandSerializers.timestamp.serialize(apply.executeAt, out, version);
-            CommandSerializers.deps.serialize(apply.deps, out, version);
+            DepsSerializer.partialDeps.serialize(apply.deps, out, version);
             CommandSerializers.writes.serialize(apply.writes, out, version);
             AccordData.serializer.serialize((AccordData) apply.result, out, version);
         }
 
         @Override
-        public Apply deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch) throws IOException
+        public Apply deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch) throws IOException
         {
-            return Apply.SerializationSupport.create(scope, waitForEpoch,
-                                                     CommandSerializers.txnId.deserialize(in, version),
-                                                     CommandSerializers.txn.deserialize(in, version),
-                                                     KeySerializers.key.deserialize(in, version),
+            return Apply.SerializationSupport.create(txnId, scope, waitForEpoch, in.readUnsignedVInt(),
                                                      CommandSerializers.timestamp.deserialize(in, version),
-                                                     CommandSerializers.deps.deserialize(in, version),
+                                                     DepsSerializer.partialDeps.deserialize(in, version),
                                                      CommandSerializers.writes.deserialize(in, version),
                                                      AccordData.serializer.deserialize(in, version));
         }
@@ -59,34 +56,34 @@ public class ApplySerializers
         @Override
         public long serializedBodySize(Apply apply, int version)
         {
-            return CommandSerializers.txnId.serializedSize(apply.txnId, version)
-                   + CommandSerializers.txn.serializedSize(apply.txn, version)
-                   + KeySerializers.key.serializedSize(apply.homeKey, version)
+            return TypeSizes.sizeofUnsignedVInt(apply.untilEpoch)
                    + CommandSerializers.timestamp.serializedSize(apply.executeAt, version)
-                   + CommandSerializers.deps.serializedSize(apply.deps, version)
+                   + DepsSerializer.partialDeps.serializedSize(apply.deps, version)
                    + CommandSerializers.writes.serializedSize(apply.writes, version)
                    + AccordData.serializer.serializedSize((AccordData) apply.result, version);
         }
     };
 
-    public static final IVersionedSerializer<Apply.ApplyOk> reply = new IVersionedSerializer<Apply.ApplyOk>()
+    public static final IVersionedSerializer<Apply.ApplyReply> reply = new IVersionedSerializer<Apply.ApplyReply>()
     {
+        private final Apply.ApplyReply[] replies = Apply.ApplyReply.values();
+
         @Override
-        public void serialize(Apply.ApplyOk t, DataOutputPlus out, int version) throws IOException
+        public void serialize(Apply.ApplyReply t, DataOutputPlus out, int version) throws IOException
         {
-
+            out.writeByte(t.ordinal());
         }
 
         @Override
-        public Apply.ApplyOk deserialize(DataInputPlus in, int version) throws IOException
+        public Apply.ApplyReply deserialize(DataInputPlus in, int version) throws IOException
         {
-            return Apply.ApplyOk.INSTANCE;
+            return replies[in.readByte()];
         }
 
         @Override
-        public long serializedSize(Apply.ApplyOk t, int version)
+        public long serializedSize(Apply.ApplyReply t, int version)
         {
-            return 0;
+            return 1;
         }
     };
 }
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidateSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidateSerializers.java
deleted file mode 100644
index e57743d095..0000000000
--- a/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidateSerializers.java
+++ /dev/null
@@ -1,126 +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.service.accord.serializers;
-
-import java.io.IOException;
-
-import com.google.common.base.Preconditions;
-
-import accord.api.Result;
-import accord.local.Status;
-import accord.messages.BeginInvalidate;
-import accord.messages.BeginInvalidate.InvalidateNack;
-import accord.messages.BeginInvalidate.InvalidateOk;
-import accord.messages.BeginRecovery.RecoverReply;
-import accord.primitives.Ballot;
-import accord.primitives.Deps;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
-import accord.txn.Writes;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-
-import static org.apache.cassandra.utils.NullableSerializer.deserializeNullable;
-import static org.apache.cassandra.utils.NullableSerializer.serializeNullable;
-import static org.apache.cassandra.utils.NullableSerializer.serializedSizeNullable;
-
-public class BeginInvalidateSerializers
-{
-    public static final IVersionedSerializer<BeginInvalidate> request = new IVersionedSerializer<BeginInvalidate>()
-    {
-        @Override
-        public void serialize(BeginInvalidate begin, DataOutputPlus out, int version) throws IOException
-        {
-            CommandSerializers.txnId.serialize(begin.txnId, out, version);
-            KeySerializers.key.serialize(begin.someKey, out, version);
-            CommandSerializers.ballot.serialize(begin.ballot, out, version);
-        }
-
-        @Override
-        public BeginInvalidate deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return new BeginInvalidate(CommandSerializers.txnId.deserialize(in, version),
-                                       KeySerializers.key.deserialize(in, version),
-                                       CommandSerializers.ballot.deserialize(in, version));
-        }
-
-        @Override
-        public long serializedSize(BeginInvalidate begin, int version)
-        {
-            return CommandSerializers.txnId.serializedSize(begin.txnId, version)
-                   + KeySerializers.key.serializedSize(begin.someKey, version)
-                   + CommandSerializers.ballot.serializedSize(begin.ballot, version);
-        }
-    };
-
-    public static final IVersionedSerializer<RecoverReply> reply = new RecoverySerializers.RecoverReplySerializer<InvalidateOk, InvalidateNack>()
-    {
-        @Override
-        void serializeNack(InvalidateNack recoverNack, DataOutputPlus out, int version) throws IOException
-        {
-            super.serializeNack(recoverNack, out, version);
-            serializeNullable(CommandSerializers.txn, recoverNack.txn, out, version);
-            serializeNullable(KeySerializers.key, recoverNack.homeKey, out, version);
-        }
-
-        @Override
-        void serializeOk(InvalidateOk recoverOk, DataOutputPlus out, int version) throws IOException
-        {
-            super.serializeOk(recoverOk, out, version);
-            serializeNullable(CommandSerializers.txn, recoverOk.txn, out, version);
-            serializeNullable(KeySerializers.key, recoverOk.homeKey, out, version);
-        }
-
-        @Override
-        InvalidateNack deserializeNack(Ballot supersededBy, DataInputPlus in, int version) throws IOException
-        {
-            return new InvalidateNack(supersededBy,
-                                      deserializeNullable(CommandSerializers.txn, in, version),
-                                      deserializeNullable(KeySerializers.key, in, version));
-        }
-
-        @Override
-        InvalidateOk deserializeOk(TxnId txnId, Status status, Ballot accepted, Timestamp executeAt, Deps deps, Deps earlierCommittedWitness, Deps earlierAcceptedNoWitness, boolean rejectsFastPath, Writes writes, Result result, DataInputPlus in, int version) throws IOException
-        {
-            Preconditions.checkArgument(earlierCommittedWitness == null);
-            Preconditions.checkArgument(earlierAcceptedNoWitness == null);
-            Preconditions.checkArgument(!rejectsFastPath);
-            return new InvalidateOk(txnId, status, accepted, executeAt, deps, writes, result,
-                                    deserializeNullable(CommandSerializers.txn, in, version),
-                                    deserializeNullable(KeySerializers.key, in, version));
-        }
-
-        @Override
-        long serializedNackSize(InvalidateNack recoverNack, int version)
-        {
-            return super.serializedNackSize(recoverNack, version)
-                   + serializedSizeNullable(CommandSerializers.txn, recoverNack.txn, version)
-                   + serializedSizeNullable(KeySerializers.key, recoverNack.homeKey, version);
-        }
-
-        @Override
-        long serializedOkSize(InvalidateOk recoverOk, int version)
-        {
-            return super.serializedOkSize(recoverOk, version)
-                   + serializedSizeNullable(CommandSerializers.txn, recoverOk.txn, version)
-                   + serializedSizeNullable(KeySerializers.key, recoverOk.homeKey, version);
-        }
-    };
-}
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java
new file mode 100644
index 0000000000..9596761f35
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.serializers;
+
+import java.io.IOException;
+
+import accord.api.RoutingKey;
+import accord.local.Status;
+import accord.messages.BeginInvalidation;
+import accord.messages.BeginInvalidation.InvalidateNack;
+import accord.messages.BeginInvalidation.InvalidateOk;
+import accord.messages.BeginInvalidation.InvalidateReply;
+import accord.primitives.AbstractRoute;
+import accord.primitives.Ballot;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static org.apache.cassandra.utils.NullableSerializer.deserializeNullable;
+import static org.apache.cassandra.utils.NullableSerializer.serializeNullable;
+import static org.apache.cassandra.utils.NullableSerializer.serializedSizeNullable;
+
+public class BeginInvalidationSerializers
+{
+    public static final IVersionedSerializer<BeginInvalidation> request = new IVersionedSerializer<BeginInvalidation>()
+    {
+        @Override
+        public void serialize(BeginInvalidation begin, DataOutputPlus out, int version) throws IOException
+        {
+            CommandSerializers.txnId.serialize(begin.txnId, out, version);
+            KeySerializers.routingKey.serialize(begin.someKey, out, version);
+            CommandSerializers.ballot.serialize(begin.ballot, out, version);
+        }
+
+        @Override
+        public BeginInvalidation deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return new BeginInvalidation(CommandSerializers.txnId.deserialize(in, version),
+                                       KeySerializers.routingKey.deserialize(in, version),
+                                       CommandSerializers.ballot.deserialize(in, version));
+        }
+
+        @Override
+        public long serializedSize(BeginInvalidation begin, int version)
+        {
+            return CommandSerializers.txnId.serializedSize(begin.txnId, version)
+                   + KeySerializers.routingKey.serializedSize(begin.someKey, version)
+                   + CommandSerializers.ballot.serializedSize(begin.ballot, version);
+        }
+    };
+
+    public static final IVersionedSerializer<InvalidateReply> reply = new IVersionedSerializer<InvalidateReply>()
+    {
+        void serializeOk(InvalidateOk ok, DataOutputPlus out, int version) throws IOException
+        {
+            CommandSerializers.status.serialize(ok.status, out, version);
+            serializeNullable(KeySerializers.abstractRoute, ok.route, out, version);
+            serializeNullable(KeySerializers.routingKey, ok.homeKey, out, version);
+        }
+
+        InvalidateOk deserializeOk(DataInputPlus in, int version) throws IOException
+        {
+            Status status = CommandSerializers.status.deserialize(in, version);
+            AbstractRoute route = deserializeNullable(KeySerializers.abstractRoute, in, version);
+            RoutingKey homeKey = deserializeNullable(KeySerializers.routingKey, in, version);
+            return new InvalidateOk(status, route, homeKey);
+        }
+
+        long serializedOkSize(InvalidateOk ok, int version)
+        {
+            return CommandSerializers.status.serializedSize(ok.status, version)
+                   + serializedSizeNullable(KeySerializers.abstractRoute, ok.route, version)
+                   + serializedSizeNullable(KeySerializers.routingKey, ok.homeKey, version);
+        }
+
+        void serializeNack(InvalidateNack nack, DataOutputPlus out, int version) throws IOException
+        {
+            CommandSerializers.ballot.serialize(nack.supersededBy, out, version);
+            serializeNullable(KeySerializers.routingKey, nack.homeKey, out, version);
+        }
+
+        InvalidateNack deserializeNack(DataInputPlus in, int version) throws IOException
+        {
+            Ballot supersededBy = CommandSerializers.ballot.deserialize(in, version);
+            RoutingKey homeKey = deserializeNullable(KeySerializers.routingKey, in, version);
+            return new InvalidateNack(supersededBy, homeKey);
+        }
+
+        long serializedNackSize(InvalidateNack nack, int version)
+        {
+            return CommandSerializers.ballot.serializedSize(nack.supersededBy, version)
+                   + serializedSizeNullable(KeySerializers.routingKey, nack.homeKey, version);
+        }
+
+        @Override
+        public void serialize(InvalidateReply reply, DataOutputPlus out, int version) throws IOException
+        {
+            out.writeBoolean(reply.isOk());
+            if (!reply.isOk())
+                serializeNack((InvalidateNack) reply, out, version);
+            else
+                serializeOk((InvalidateOk) reply, out, version);
+        }
+
+        @Override
+        public InvalidateReply deserialize(DataInputPlus in, int version) throws IOException
+        {
+            boolean isOk = in.readBoolean();
+            if (!isOk)
+                return deserializeNack(in, version);
+
+            return deserializeOk(in, version);
+        }
+
+        @Override
+        public long serializedSize(InvalidateReply reply, int version)
+        {
+            return TypeSizes.sizeof(reply.isOk())
+                   + (reply.isOk() ? serializedOkSize((InvalidateOk) reply, version)
+                                   : serializedNackSize((InvalidateNack) reply, version));
+        }
+    };
+}
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java
index 318592e026..7076dcd1cb 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java
@@ -20,17 +20,30 @@ package org.apache.cassandra.service.accord.serializers;
 
 import java.io.IOException;
 
+import accord.api.Result;
+import accord.api.RoutingKey;
+import accord.local.SaveStatus;
+import accord.local.Status.Durability;
 import accord.messages.CheckStatus;
 import accord.messages.CheckStatus.CheckStatusNack;
 import accord.messages.CheckStatus.CheckStatusOk;
 import accord.messages.CheckStatus.CheckStatusOkFull;
 import accord.messages.CheckStatus.CheckStatusReply;
+import accord.primitives.AbstractRoute;
+import accord.primitives.Ballot;
+import accord.primitives.PartialDeps;
+import accord.primitives.PartialTxn;
+import accord.primitives.RoutingKeys;
+import accord.primitives.Timestamp;
+import accord.primitives.TxnId;
+import accord.primitives.Writes;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.service.accord.db.AccordData;
 
+import static accord.messages.CheckStatus.SerializationSupport.createOk;
 import static org.apache.cassandra.service.accord.serializers.NullableSerializer.deserializeNullable;
 import static org.apache.cassandra.service.accord.serializers.NullableSerializer.serializeNullable;
 import static org.apache.cassandra.service.accord.serializers.NullableSerializer.serializedSizeNullable;
@@ -39,31 +52,37 @@ public class CheckStatusSerializers
 {
     public static final IVersionedSerializer<CheckStatus> request = new IVersionedSerializer<CheckStatus>()
     {
+        final CheckStatus.IncludeInfo[] infos = CheckStatus.IncludeInfo.values();
+
         @Override
         public void serialize(CheckStatus check, DataOutputPlus out, int version) throws IOException
         {
             CommandSerializers.txnId.serialize(check.txnId, out, version);
-            KeySerializers.key.serialize(check.key, out, version);
-            out.writeLong(check.epoch);
-            out.writeInt(check.includeInfo.ordinal());
+            KeySerializers.routingKeys.serialize(check.someKeys, out, version);
+            out.writeUnsignedVInt(check.startEpoch);
+            out.writeUnsignedVInt(check.endEpoch - check.startEpoch);
+            out.writeByte(check.includeInfo.ordinal());
         }
 
         @Override
         public CheckStatus deserialize(DataInputPlus in, int version) throws IOException
         {
-            return new CheckStatus(CommandSerializers.txnId.deserialize(in, version),
-                                   KeySerializers.key.deserialize(in, version),
-                                   in.readLong(),
-                                   CheckStatus.IncludeInfo.values()[in.readInt()]);
+            TxnId txnId = CommandSerializers.txnId.deserialize(in, version);
+            RoutingKeys someKeys = KeySerializers.routingKeys.deserialize(in, version);
+            long startEpoch = in.readUnsignedVInt();
+            long endEpoch = in.readUnsignedVInt() + startEpoch;
+            CheckStatus.IncludeInfo info = infos[in.readByte()];
+            return new CheckStatus(txnId, someKeys, startEpoch, endEpoch, info);
         }
 
         @Override
         public long serializedSize(CheckStatus check, int version)
         {
             return CommandSerializers.txnId.serializedSize(check.txnId, version)
-                   + KeySerializers.key.serializedSize(check.key, version)
-                   + TypeSizes.LONG_SIZE
-                   + TypeSizes.INT_SIZE;
+                   + KeySerializers.routingKeys.serializedSize(check.someKeys, version)
+                   + TypeSizes.sizeofUnsignedVInt(check.startEpoch)
+                   + TypeSizes.sizeofUnsignedVInt(check.endEpoch - check.startEpoch)
+                   + TypeSizes.BYTE_SIZE;
         }
     };
 
@@ -82,22 +101,23 @@ public class CheckStatusSerializers
                 return;
             }
 
-            CheckStatusOk statusOk = (CheckStatusOk) reply;
+            CheckStatusOk ok = (CheckStatusOk) reply;
             out.write(reply instanceof CheckStatusOkFull ? FULL : OK);
-            CommandSerializers.status.serialize(statusOk.status, out, version);
-            CommandSerializers.ballot.serialize(statusOk.promised, out, version);
-            CommandSerializers.ballot.serialize(statusOk.accepted, out, version);
-            out.writeBoolean(statusOk.isCoordinating);
-            out.writeBoolean(statusOk.hasExecutedOnAllShards);
+            CommandSerializers.saveStatus.serialize(ok.saveStatus, out, version);
+            CommandSerializers.ballot.serialize(ok.promised, out, version);
+            CommandSerializers.ballot.serialize(ok.accepted, out, version);
+            serializeNullable(ok.executeAt, out, version, CommandSerializers.timestamp);
+            out.writeBoolean(ok.isCoordinating);
+            CommandSerializers.durability.serialize(ok.durability, out, version);
+            serializeNullable(ok.route, out, version, KeySerializers.abstractRoute);
+            serializeNullable(ok.homeKey, out, version, KeySerializers.routingKey);
 
             if (!(reply instanceof CheckStatusOkFull))
                 return;
 
-            CheckStatusOkFull okFull = (CheckStatusOkFull) statusOk;
-            serializeNullable(okFull.txn, out, version, CommandSerializers.txn);
-            serializeNullable(okFull.homeKey, out, version, KeySerializers.key);
-            serializeNullable(okFull.executeAt, out, version, CommandSerializers.timestamp);
-            CommandSerializers.deps.serialize(okFull.deps, out, version);
+            CheckStatusOkFull okFull = (CheckStatusOkFull) ok;
+            serializeNullable(okFull.partialTxn, out, version, CommandSerializers.partialTxn);
+            serializeNullable(okFull.committedDeps, out, version, DepsSerializer.partialDeps);
             serializeNullable(okFull.writes, out, version, CommandSerializers.writes);
             serializeNullable((AccordData) okFull.result, out, version, AccordData.serializer);
         }
@@ -112,23 +132,25 @@ public class CheckStatusSerializers
                 case NACK:
                     return CheckStatusNack.nack();
                 case OK:
-                    return new CheckStatusOk(CommandSerializers.status.deserialize(in, version),
-                                             CommandSerializers.ballot.deserialize(in, version),
-                                             CommandSerializers.ballot.deserialize(in, version),
-                                             in.readBoolean(),
-                                             in.readBoolean());
                 case FULL:
-                    return new CheckStatusOkFull(CommandSerializers.status.deserialize(in, version),
-                                                 CommandSerializers.ballot.deserialize(in, version),
-                                                 CommandSerializers.ballot.deserialize(in, version),
-                                                 in.readBoolean(),
-                                                 in.readBoolean(),
-                                                 deserializeNullable(in, version, CommandSerializers.txn),
-                                                 deserializeNullable(in, version, KeySerializers.key),
-                                                 deserializeNullable(in, version, CommandSerializers.timestamp),
-                                                 CommandSerializers.deps.deserialize(in, version),
-                                                 deserializeNullable(in, version, CommandSerializers.writes),
-                                                 deserializeNullable(in, version, AccordData.serializer));
+                    SaveStatus status = CommandSerializers.saveStatus.deserialize(in, version);
+                    Ballot promised = CommandSerializers.ballot.deserialize(in, version);
+                    Ballot accepted = CommandSerializers.ballot.deserialize(in, version);
+                    Timestamp executeAt = deserializeNullable(in, version, CommandSerializers.timestamp);
+                    boolean isCoordinating = in.readBoolean();
+                    Durability durability = CommandSerializers.durability.deserialize(in, version);
+                    AbstractRoute route = deserializeNullable(in, version, KeySerializers.abstractRoute);
+                    RoutingKey homeKey = deserializeNullable(in, version, KeySerializers.routingKey);
+
+                    if (kind == OK)
+                        return createOk(status, promised, accepted, executeAt, isCoordinating, durability, route, homeKey);
+
+                    PartialTxn partialTxn = deserializeNullable(in, version, CommandSerializers.partialTxn);
+                    PartialDeps committedDeps = deserializeNullable(in, version, DepsSerializer.partialDeps);
+                    Writes writes = deserializeNullable(in, version, CommandSerializers.writes);
+                    Result result = deserializeNullable(in, version, AccordData.serializer);
+                    return createOk(status, promised, accepted, executeAt, isCoordinating, durability, route, homeKey,
+                                    partialTxn, committedDeps, writes, result);
             }
         }
 
@@ -139,24 +161,24 @@ public class CheckStatusSerializers
             if (!reply.isOk())
                 return size;
 
-            CheckStatusOk statusOk = (CheckStatusOk) reply;
-            size += CommandSerializers.status.serializedSize(statusOk.status, version);
-            size += CommandSerializers.ballot.serializedSize(statusOk.promised, version);
-            size += CommandSerializers.ballot.serializedSize(statusOk.accepted, version);
-            size += TypeSizes.BOOL_SIZE;
+            CheckStatusOk ok = (CheckStatusOk) reply;
+            size += CommandSerializers.saveStatus.serializedSize(ok.saveStatus, version);
+            size += CommandSerializers.ballot.serializedSize(ok.promised, version);
+            size += CommandSerializers.ballot.serializedSize(ok.accepted, version);
+            size += serializedSizeNullable(ok.executeAt, version, CommandSerializers.timestamp);
             size += TypeSizes.BOOL_SIZE;
+            size += CommandSerializers.durability.serializedSize(ok.durability, version);
+            size += serializedSizeNullable(ok.homeKey, version, KeySerializers.routingKey);
+            size += serializedSizeNullable(ok.route, version, KeySerializers.abstractRoute);
 
             if (!(reply instanceof CheckStatusOkFull))
                 return size;
 
-            CheckStatusOkFull okFull = (CheckStatusOkFull) statusOk;
-            size += serializedSizeNullable(okFull.txn, version, CommandSerializers.txn);
-            size += serializedSizeNullable(okFull.homeKey, version, KeySerializers.key);
-            size += serializedSizeNullable(okFull.executeAt, version, CommandSerializers.timestamp);
-            size += CommandSerializers.deps.serializedSize(okFull.deps, version);
+            CheckStatusOkFull okFull = (CheckStatusOkFull) ok;
+            size += serializedSizeNullable(okFull.partialTxn, version, CommandSerializers.partialTxn);
+            size += serializedSizeNullable(okFull.committedDeps, version, DepsSerializer.partialDeps);
             size += serializedSizeNullable(okFull.writes, version, CommandSerializers.writes);
             size += serializedSizeNullable((AccordData) okFull.result, version, AccordData.serializer);
-
             return size;
         }
     };
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CommandSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CommandSerializers.java
index 4364569aa4..ef19dbe2d0 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/CommandSerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/CommandSerializers.java
@@ -23,14 +23,17 @@ import java.io.IOException;
 import com.google.common.base.Preconditions;
 
 import accord.local.Node;
+import accord.local.SaveStatus;
 import accord.local.Status;
+import accord.local.Status.Durability;
 import accord.primitives.Ballot;
-import accord.primitives.Deps;
+import accord.primitives.KeyRanges;
 import accord.primitives.Keys;
+import accord.primitives.PartialTxn;
 import accord.primitives.Timestamp;
+import accord.primitives.Txn;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
-import accord.txn.Writes;
+import accord.primitives.Writes;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.io.IVersionedSerializer;
@@ -41,9 +44,6 @@ import org.apache.cassandra.service.accord.db.AccordRead;
 import org.apache.cassandra.service.accord.db.AccordUpdate;
 import org.apache.cassandra.service.accord.db.AccordWrite;
 
-import static accord.primitives.Deps.SerializerSupport.keyToTxnId;
-import static accord.primitives.Deps.SerializerSupport.keyToTxnIdCount;
-
 public class CommandSerializers
 {
     private CommandSerializers() {}
@@ -51,6 +51,7 @@ public class CommandSerializers
     public static final TimestampSerializer<TxnId> txnId = new TimestampSerializer<>(TxnId::new);
     public static final TimestampSerializer<Timestamp> timestamp = new TimestampSerializer<>(Timestamp::new);
     public static final TimestampSerializer<Ballot> ballot = new TimestampSerializer<>(Ballot::new);
+    public static final EnumSerializer<Txn.Kind> kind = new EnumSerializer<>(Txn.Kind.class);
 
     public static class TimestampSerializer<T extends Timestamp> implements IVersionedSerializer<T>
     {
@@ -123,36 +124,39 @@ public class CommandSerializers
         }
     }
 
-    public static final IVersionedSerializer<Txn> txn = new IVersionedSerializer<Txn>()
+    public static final IVersionedSerializer<PartialTxn> partialTxn = new IVersionedSerializer<PartialTxn>()
     {
         @Override
-        public void serialize(Txn txn, DataOutputPlus out, int version) throws IOException
+        public void serialize(PartialTxn txn, DataOutputPlus out, int version) throws IOException
         {
+            CommandSerializers.kind.serialize(txn.kind(), out, version);
+            KeySerializers.ranges.serialize(txn.covering(), out, version);
             KeySerializers.keys.serialize(txn.keys(), out, version);
             AccordRead.serializer.serialize((AccordRead) txn.read(), out, version);
             AccordQuery.serializer.serialize((AccordQuery) txn.query(), out, version);
             out.writeBoolean(txn.update() != null);
             if (txn.update() != null)
                 AccordUpdate.serializer.serialize((AccordUpdate) txn.update(), out, version);
-
         }
 
         @Override
-        public Txn deserialize(DataInputPlus in, int version) throws IOException
+        public PartialTxn deserialize(DataInputPlus in, int version) throws IOException
         {
+            Txn.Kind kind = CommandSerializers.kind.deserialize(in, version);
+            KeyRanges covering = KeySerializers.ranges.deserialize(in, version);
             Keys keys = KeySerializers.keys.deserialize(in, version);
             AccordRead read = AccordRead.serializer.deserialize(in, version);
             AccordQuery query = AccordQuery.serializer.deserialize(in, version);
-            if (in.readBoolean())
-                return new Txn.InMemory(keys, read, query, AccordUpdate.serializer.deserialize(in, version));
-            else
-                return new Txn.InMemory(keys, read, query);
+            AccordUpdate update = in.readBoolean() ? AccordUpdate.serializer.deserialize(in, version) : null;
+            return new PartialTxn.InMemory(covering, kind, keys, read, query, update);
         }
 
         @Override
-        public long serializedSize(Txn txn, int version)
+        public long serializedSize(PartialTxn txn, int version)
         {
-            long size = KeySerializers.keys.serializedSize(txn.keys(), version);
+            long size = CommandSerializers.kind.serializedSize(txn.kind(), version);
+            size += KeySerializers.ranges.serializedSize(txn.covering(), version);
+            size += KeySerializers.keys.serializedSize(txn.keys(), version);
             size += AccordRead.serializer.serializedSize((AccordRead) txn.read(), version);
             size += AccordQuery.serializer.serializedSize((AccordQuery) txn.query(), version);
             size += TypeSizes.sizeof(txn.update() != null);
@@ -162,79 +166,9 @@ public class CommandSerializers
         }
     };
 
-    public static final IVersionedSerializer<Status> status = new IVersionedSerializer<Status>()
-    {
-        @Override
-        public void serialize(Status status, DataOutputPlus out, int version) throws IOException
-        {
-            out.writeUnsignedVInt(status.ordinal());
-
-        }
-
-        @Override
-        public Status deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return Status.values()[(int) in.readUnsignedVInt()];
-        }
-
-        @Override
-        public long serializedSize(Status status, int version)
-        {
-            return TypeSizes.sizeofUnsignedVInt(status.ordinal());
-        }
-    };
-
-    public static final IVersionedSerializer<Deps> deps = new IVersionedSerializer<Deps>()
-    {
-        @Override
-        public void serialize(Deps deps, DataOutputPlus out, int version) throws IOException
-        {
-            Keys keys = deps.keys();
-            KeySerializers.keys.serialize(keys, out, version);
-
-            int txnIdCount = deps.txnIdCount();
-            out.writeUnsignedVInt(txnIdCount);
-            for (int i=0; i<txnIdCount; i++)
-                CommandSerializers.txnId.serialize(deps.txnId(i), out, version);
-
-            int keyToTxnIdCount = keyToTxnIdCount(deps);
-            out.writeUnsignedVInt(keyToTxnIdCount);
-            for (int i=0; i<keyToTxnIdCount; i++)
-                out.writeUnsignedVInt(keyToTxnId(deps, i));
-
-        }
-
-        @Override
-        public Deps deserialize(DataInputPlus in, int version) throws IOException
-        {
-            Keys keys = KeySerializers.keys.deserialize(in, version);
-            TxnId[] txnIds = new TxnId[(int) in.readUnsignedVInt()];
-            for (int i=0; i<txnIds.length; i++)
-                txnIds[i] = CommandSerializers.txnId.deserialize(in, version);
-            int[] keyToTxnIds = new int[(int) in.readUnsignedVInt()];
-            for (int i=0; i<keyToTxnIds.length; i++)
-                keyToTxnIds[i] = (int) in.readUnsignedVInt();
-            return Deps.SerializerSupport.create(keys, txnIds, keyToTxnIds);
-        }
-
-        @Override
-        public long serializedSize(Deps deps, int version)
-        {
-            Keys keys = deps.keys();
-            long size = KeySerializers.keys.serializedSize(keys, version);
-
-            int txnIdCount = deps.txnIdCount();
-            size += TypeSizes.sizeofUnsignedVInt(txnIdCount);
-            for (int i=0; i<txnIdCount; i++)
-                size += CommandSerializers.txnId.serializedSize(deps.txnId(i), version);
-
-            int keyToTxnIdCount = keyToTxnIdCount(deps);
-            size += TypeSizes.sizeofUnsignedVInt(keyToTxnIdCount);
-            for (int i=0; i<keyToTxnIdCount; i++)
-                size += TypeSizes.sizeofUnsignedVInt(keyToTxnId(deps, i));
-            return size;
-        }
-    };
+    public static final IVersionedSerializer<SaveStatus> saveStatus = new EnumSerializer<>(SaveStatus.class);
+    public static final IVersionedSerializer<Status> status = new EnumSerializer<>(Status.class);
+    public static final IVersionedSerializer<Durability> durability = new EnumSerializer<>(Durability.class);
 
     public static final IVersionedSerializer<Writes> writes = new IVersionedSerializer<Writes>()
     {
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java
index bbd722117b..0093e8095e 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java
@@ -21,12 +21,18 @@ package org.apache.cassandra.service.accord.serializers;
 import java.io.IOException;
 
 import accord.messages.Commit;
-import accord.primitives.Keys;
+import accord.primitives.PartialRoute;
+import accord.primitives.RoutingKeys;
+import accord.primitives.TxnId;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
+import static org.apache.cassandra.service.accord.serializers.NullableSerializer.deserializeNullable;
+import static org.apache.cassandra.service.accord.serializers.NullableSerializer.serializeNullable;
+import static org.apache.cassandra.service.accord.serializers.NullableSerializer.serializedSizeNullable;
+
 public class CommitSerializers
 {
     public static final IVersionedSerializer<Commit> request = new TxnRequestSerializer<Commit>()
@@ -34,60 +40,64 @@ public class CommitSerializers
         @Override
         public void serializeBody(Commit msg, DataOutputPlus out, int version) throws IOException
         {
-            CommandSerializers.txnId.serialize(msg.txnId, out, version);
-            CommandSerializers.txn.serialize(msg.txn, out, version);
-            CommandSerializers.deps.serialize(msg.deps, out, version);
-            KeySerializers.key.serialize(msg.homeKey, out, version);
             CommandSerializers.timestamp.serialize(msg.executeAt, out, version);
-            out.writeBoolean(msg.read);
+            serializeNullable(msg.partialTxn, out, version, CommandSerializers.partialTxn);
+            DepsSerializer.partialDeps.serialize(msg.partialDeps, out, version);
+            serializeNullable(msg.route, out, version, KeySerializers.route);
+            serializeNullable(msg.read, out, version, ReadDataSerializers.request);
         }
 
         @Override
-        public Commit deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch) throws IOException
+        public Commit deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch) throws IOException
         {
-            return new Commit(scope, waitForEpoch,
-                              CommandSerializers.txnId.deserialize(in, version),
-                              CommandSerializers.txn.deserialize(in, version),
-                              CommandSerializers.deps.deserialize(in, version),
-                              KeySerializers.key.deserialize(in, version),
-                              CommandSerializers.timestamp.deserialize(in, version),
-                              in.readBoolean());
+            return Commit.SerializerSupport.create(txnId, scope, waitForEpoch,
+                                                   CommandSerializers.timestamp.deserialize(in, version),
+                                                   deserializeNullable(in, version, CommandSerializers.partialTxn),
+                                                   DepsSerializer.partialDeps.deserialize(in, version),
+                                                   deserializeNullable(in, version, KeySerializers.route),
+                                                   deserializeNullable(in, version, ReadDataSerializers.request)
+            );
         }
 
         @Override
         public long serializedBodySize(Commit msg, int version)
         {
-            return CommandSerializers.txnId.serializedSize(msg.txnId, version)
-                   + CommandSerializers.txn.serializedSize(msg.txn, version)
-                   + CommandSerializers.deps.serializedSize(msg.deps, version)
-                   + KeySerializers.key.serializedSize(msg.homeKey, version)
-                   + CommandSerializers.timestamp.serializedSize(msg.executeAt, version)
-                   + TypeSizes.BOOL_SIZE;
+            return CommandSerializers.timestamp.serializedSize(msg.executeAt, version)
+                   + serializedSizeNullable(msg.partialTxn, version, CommandSerializers.partialTxn)
+                   + DepsSerializer.partialDeps.serializedSize(msg.partialDeps, version)
+                   + serializedSizeNullable(msg.route, version, KeySerializers.route)
+                   + serializedSizeNullable(msg.read, version, ReadDataSerializers.request);
         }
     };
 
-    public static final IVersionedSerializer<Commit.Invalidate> invalidate = new TxnRequestSerializer<Commit.Invalidate>()
+    public static final IVersionedSerializer<Commit.Invalidate> invalidate = new IVersionedSerializer<Commit.Invalidate>()
     {
         @Override
-        public void serializeBody(Commit.Invalidate invalidate, DataOutputPlus out, int version) throws IOException
+        public void serialize(Commit.Invalidate invalidate, DataOutputPlus out, int version) throws IOException
         {
             CommandSerializers.txnId.serialize(invalidate.txnId, out, version);
-            KeySerializers.keys.serialize(invalidate.txnKeys, out, version);
+            KeySerializers.routingKeys.serialize(invalidate.scope, out, version);
+            out.writeUnsignedVInt(invalidate.waitForEpoch);
+            out.writeUnsignedVInt(invalidate.invalidateUntilEpoch - invalidate.waitForEpoch);
         }
 
         @Override
-        public Commit.Invalidate deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch) throws IOException
+        public Commit.Invalidate deserialize(DataInputPlus in, int version) throws IOException
         {
-            return Commit.Invalidate.SerializerSupport.create(scope, waitForEpoch,
-                                                              CommandSerializers.txnId.deserialize(in, version),
-                                                              KeySerializers.keys.deserialize(in, version));
+            TxnId txnId = CommandSerializers.txnId.deserialize(in, version);
+            RoutingKeys routingKeys = KeySerializers.routingKeys.deserialize(in, version);
+            long waitForEpoch = in.readUnsignedVInt();
+            long invalidateUntilEpoch = in.readUnsignedVInt() + waitForEpoch;
+            return Commit.Invalidate.SerializerSupport.create(txnId, routingKeys, waitForEpoch, invalidateUntilEpoch);
         }
 
         @Override
-        public long serializedBodySize(Commit.Invalidate invalidate, int version)
+        public long serializedSize(Commit.Invalidate invalidate, int version)
         {
             return CommandSerializers.txnId.serializedSize(invalidate.txnId, version)
-                   + KeySerializers.keys.serializedSize(invalidate.txnKeys, version);
+                   + KeySerializers.routingKeys.serializedSize(invalidate.scope, version)
+                   + TypeSizes.sizeofUnsignedVInt(invalidate.waitForEpoch)
+                   + TypeSizes.sizeofUnsignedVInt(invalidate.invalidateUntilEpoch - invalidate.waitForEpoch);
         }
     };
 }
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/DepsSerializer.java b/src/java/org/apache/cassandra/service/accord/serializers/DepsSerializer.java
new file mode 100644
index 0000000000..2917862c13
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/accord/serializers/DepsSerializer.java
@@ -0,0 +1,120 @@
+/*
+ * 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.service.accord.serializers;
+
+import java.io.IOException;
+
+import accord.primitives.Deps;
+import accord.primitives.KeyRanges;
+import accord.primitives.Keys;
+import accord.primitives.PartialDeps;
+import accord.primitives.TxnId;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static accord.primitives.Deps.SerializerSupport.keyToTxnId;
+import static accord.primitives.Deps.SerializerSupport.keyToTxnIdCount;
+
+public abstract class DepsSerializer<D extends Deps> implements IVersionedSerializer<D>
+{
+    public static final DepsSerializer<Deps> deps = new DepsSerializer<Deps>()
+    {
+        @Override
+        Deps deserialize(Keys keys, TxnId[] txnIds, int[] keyToTxnIds, DataInputPlus in, int version)
+        {
+            return Deps.SerializerSupport.create(keys, txnIds, keyToTxnIds);
+        }
+    };
+
+    public static final DepsSerializer<PartialDeps> partialDeps = new DepsSerializer<PartialDeps>()
+    {
+        @Override
+        PartialDeps deserialize(Keys keys, TxnId[] txnIds, int[] keyToTxnIds, DataInputPlus in, int version) throws IOException
+        {
+            KeyRanges covering = KeySerializers.ranges.deserialize(in, version);
+            return PartialDeps.SerializerSupport.create(covering, keys, txnIds, keyToTxnIds);
+        }
+
+        @Override
+        public void serialize(PartialDeps partialDeps, DataOutputPlus out, int version) throws IOException
+        {
+            super.serialize(partialDeps, out, version);
+            KeySerializers.ranges.serialize(partialDeps.covering, out, version);
+        }
+
+        @Override
+        public long serializedSize(PartialDeps partialDeps, int version)
+        {
+            return KeySerializers.ranges.serializedSize(partialDeps.covering, version)
+                   + super.serializedSize(partialDeps, version);
+        }
+    };
+
+    @Override
+    public void serialize(D deps, DataOutputPlus out, int version) throws IOException
+    {
+        Keys keys = deps.keys();
+        KeySerializers.keys.serialize(keys, out, version);
+
+        int txnIdCount = deps.txnIdCount();
+        out.writeUnsignedVInt(txnIdCount);
+        for (int i=0; i<txnIdCount; i++)
+            CommandSerializers.txnId.serialize(deps.txnId(i), out, version);
+
+        int keyToTxnIdCount = keyToTxnIdCount(deps);
+        out.writeUnsignedVInt(keyToTxnIdCount);
+        for (int i=0; i<keyToTxnIdCount; i++)
+            out.writeUnsignedVInt(keyToTxnId(deps, i));
+    }
+
+    @Override
+    public D deserialize(DataInputPlus in, int version) throws IOException
+    {
+        Keys keys = KeySerializers.keys.deserialize(in, version);
+        TxnId[] txnIds = new TxnId[(int) in.readUnsignedVInt()];
+        for (int i=0; i<txnIds.length; i++)
+            txnIds[i] = CommandSerializers.txnId.deserialize(in, version);
+        int[] keyToTxnIds = new int[(int) in.readUnsignedVInt()];
+        for (int i=0; i<keyToTxnIds.length; i++)
+            keyToTxnIds[i] = (int) in.readUnsignedVInt();
+        return deserialize(keys, txnIds, keyToTxnIds, in, version);
+    }
+
+    abstract D deserialize(Keys keys, TxnId[] txnIds, int[] keyToTxnIds, DataInputPlus in, int version) throws IOException;
+
+    @Override
+    public long serializedSize(D deps, int version)
+    {
+        Keys keys = deps.keys();
+        long size = KeySerializers.keys.serializedSize(keys, version);
+
+        int txnIdCount = deps.txnIdCount();
+        size += TypeSizes.sizeofUnsignedVInt(txnIdCount);
+        for (int i=0; i<txnIdCount; i++)
+            size += CommandSerializers.txnId.serializedSize(deps.txnId(i), version);
+
+        int keyToTxnIdCount = keyToTxnIdCount(deps);
+        size += TypeSizes.sizeofUnsignedVInt(keyToTxnIdCount);
+        for (int i=0; i<keyToTxnIdCount; i++)
+            size += TypeSizes.sizeofUnsignedVInt(keyToTxnId(deps, i));
+        return size;
+    }
+}
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/EnumSerializer.java b/src/java/org/apache/cassandra/service/accord/serializers/EnumSerializer.java
new file mode 100644
index 0000000000..2d66c2433b
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/accord/serializers/EnumSerializer.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.service.accord.serializers;
+
+import java.io.IOException;
+
+import accord.messages.SimpleReply;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+public class EnumSerializer<E extends Enum<E>> implements IVersionedSerializer<E>
+{
+    public static final EnumSerializer<SimpleReply> simpleReply = new EnumSerializer<>(SimpleReply.class);
+
+    // TODO: should use something other than ordinal for ser/deser
+    final E[] values;
+
+    public EnumSerializer(Class<E> clazz)
+    {
+        this.values = clazz.getEnumConstants();
+    }
+
+    @Override
+    public void serialize(E t, DataOutputPlus out, int version) throws IOException
+    {
+        out.writeUnsignedVInt(t.ordinal());
+    }
+
+    @Override
+    public E deserialize(DataInputPlus in, int version) throws IOException
+    {
+        return values[(int)in.readUnsignedVInt()];
+    }
+
+    @Override
+    public long serializedSize(E t, int version)
+    {
+        return TypeSizes.sizeofUnsignedVInt(t.ordinal());
+    }
+}
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/GetDepsSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/GetDepsSerializers.java
new file mode 100644
index 0000000000..eb8acf1a67
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/accord/serializers/GetDepsSerializers.java
@@ -0,0 +1,84 @@
+/*
+ * 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.service.accord.serializers;
+
+import java.io.IOException;
+
+import accord.messages.GetDeps;
+import accord.messages.GetDeps.GetDepsOk;
+import accord.primitives.Keys;
+import accord.primitives.PartialRoute;
+import accord.primitives.Timestamp;
+import accord.primitives.Txn;
+import accord.primitives.TxnId;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+public class GetDepsSerializers
+{
+    public static final IVersionedSerializer<GetDeps> request = new TxnRequestSerializer.WithUnsyncedSerializer<GetDeps>()
+    {
+        @Override
+        public void serializeBody(GetDeps msg, DataOutputPlus out, int version) throws IOException
+        {
+            KeySerializers.keys.serialize(msg.keys, out, version);
+            CommandSerializers.timestamp.serialize(msg.executeAt, out, version);
+            CommandSerializers.kind.serialize(msg.kind, out, version);
+        }
+
+        @Override
+        public GetDeps deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey) throws IOException
+        {
+            Keys keys = KeySerializers.keys.deserialize(in, version);
+            Timestamp executeAt = CommandSerializers.timestamp.deserialize(in, version);
+            Txn.Kind kind = CommandSerializers.kind.deserialize(in, version);
+            return GetDeps.SerializationSupport.create(txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey, keys, executeAt, kind);
+        }
+
+        @Override
+        public long serializedBodySize(GetDeps msg, int version)
+        {
+            return KeySerializers.keys.serializedSize(msg.keys, version)
+                   + CommandSerializers.timestamp.serializedSize(msg.executeAt, version)
+                   + CommandSerializers.kind.serializedSize(msg.kind, version);
+        }
+    };
+
+    public static final IVersionedSerializer<GetDepsOk> reply = new IVersionedSerializer<GetDepsOk>()
+    {
+        @Override
+        public void serialize(GetDepsOk reply, DataOutputPlus out, int version) throws IOException
+        {
+            DepsSerializer.partialDeps.serialize(reply.deps, out, version);
+        }
+
+        @Override
+        public GetDepsOk deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return new GetDepsOk(DepsSerializer.partialDeps.deserialize(in, version));
+        }
+
+        @Override
+        public long serializedSize(GetDepsOk reply, int version)
+        {
+            return DepsSerializer.partialDeps.serializedSize(reply.deps, version);
+        }
+    };
+}
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/InformDurableSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/InformDurableSerializers.java
new file mode 100644
index 0000000000..66c649eb92
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/accord/serializers/InformDurableSerializers.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.service.accord.serializers;
+
+import java.io.IOException;
+
+import accord.local.Status;
+import accord.messages.InformDurable;
+import accord.primitives.PartialRoute;
+import accord.primitives.Timestamp;
+import accord.primitives.TxnId;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+public class InformDurableSerializers
+{
+    public static final IVersionedSerializer<InformDurable> request = new TxnRequestSerializer<InformDurable>()
+    {
+        @Override
+        public void serializeBody(InformDurable msg, DataOutputPlus out, int version) throws IOException
+        {
+            CommandSerializers.timestamp.serialize(msg.executeAt, out, version);
+            CommandSerializers.durability.serialize(msg.durability, out, version);
+        }
+
+        @Override
+        public InformDurable deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch) throws IOException
+        {
+            Timestamp executeAt = CommandSerializers.timestamp.deserialize(in, version);
+            Status.Durability durability = CommandSerializers.durability.deserialize(in, version);
+            return InformDurable.SerializationSupport.create(txnId, scope, waitForEpoch, executeAt, durability);
+        }
+
+        @Override
+        public long serializedBodySize(InformDurable msg, int version)
+        {
+            return CommandSerializers.timestamp.serializedSize(msg.executeAt, version)
+            + CommandSerializers.durability.serializedSize(msg.durability, version);
+        }
+    };
+}
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/InformOfPersistenceSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/InformHomeDurableSerializers.java
similarity index 55%
rename from src/java/org/apache/cassandra/service/accord/serializers/InformOfPersistenceSerializers.java
rename to src/java/org/apache/cassandra/service/accord/serializers/InformHomeDurableSerializers.java
index 07b3d148c1..a79f57b915 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/InformOfPersistenceSerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/InformHomeDurableSerializers.java
@@ -22,42 +22,50 @@ import java.io.IOException;
 
 import com.google.common.collect.Sets;
 
-import accord.messages.InformOfPersistence;
+import accord.messages.InformDurable;
+import accord.messages.InformHomeDurable;
+import accord.primitives.PartialRoute;
+import accord.primitives.Timestamp;
+import accord.primitives.TxnId;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.CollectionSerializer;
 
-public class InformOfPersistenceSerializers
+public class InformHomeDurableSerializers
 {
-    public static final IVersionedSerializer<InformOfPersistence> request = new IVersionedSerializer<InformOfPersistence>()
+    public static final IVersionedSerializer<InformHomeDurable> request = new IVersionedSerializer<InformHomeDurable>()
     {
         @Override
-        public void serialize(InformOfPersistence inform, DataOutputPlus out, int version) throws IOException
+        public void serialize(InformHomeDurable inform, DataOutputPlus out, int version) throws IOException
         {
             CommandSerializers.txnId.serialize(inform.txnId, out, version);
-            KeySerializers.key.serialize(inform.homeKey, out, version);
+            KeySerializers.routingKey.serialize(inform.homeKey, out, version);
             CommandSerializers.timestamp.serialize(inform.executeAt, out, version);
+            CommandSerializers.durability.serialize(inform.durability, out, version);
             CollectionSerializer.serializeCollection(TopologySerializers.nodeId, inform.persistedOn, out, version);
 
         }
 
         @Override
-        public InformOfPersistence deserialize(DataInputPlus in, int version) throws IOException
+        public InformHomeDurable deserialize(DataInputPlus in, int version) throws IOException
         {
-            return new InformOfPersistence(CommandSerializers.txnId.deserialize(in, version),
-                                           KeySerializers.key.deserialize(in, version),
-                                           CommandSerializers.timestamp.deserialize(in, version),
-                                           CollectionSerializer.deserializeCollection(TopologySerializers.nodeId, Sets::newHashSetWithExpectedSize, in, version));
+            return new InformHomeDurable(CommandSerializers.txnId.deserialize(in, version),
+                                         KeySerializers.routingKey.deserialize(in, version),
+                                         CommandSerializers.timestamp.deserialize(in, version),
+                                         CommandSerializers.durability.deserialize(in, version),
+                                         CollectionSerializer.deserializeCollection(TopologySerializers.nodeId, Sets::newHashSetWithExpectedSize, in, version));
         }
 
         @Override
-        public long serializedSize(InformOfPersistence inform, int version)
+        public long serializedSize(InformHomeDurable inform, int version)
         {
             return CommandSerializers.txnId.serializedSize(inform.txnId, version)
-                   + KeySerializers.key.serializedSize(inform.homeKey, version)
+                   + KeySerializers.routingKey.serializedSize(inform.homeKey, version)
                    + CommandSerializers.timestamp.serializedSize(inform.executeAt, version)
+                   + CommandSerializers.durability.serializedSize(inform.durability, version)
                    + CollectionSerializer.serializedSizeCollection(TopologySerializers.nodeId, inform.persistedOn, version);
         }
+
     };
 }
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/InformOfTxnIdSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/InformOfTxnIdSerializers.java
new file mode 100644
index 0000000000..e773a405e2
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/accord/serializers/InformOfTxnIdSerializers.java
@@ -0,0 +1,53 @@
+/*
+ * 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.service.accord.serializers;
+
+import java.io.IOException;
+
+import accord.messages.InformOfTxnId;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+public class InformOfTxnIdSerializers
+{
+    public static final IVersionedSerializer<InformOfTxnId> request = new IVersionedSerializer<InformOfTxnId>()
+    {
+        @Override
+        public void serialize(InformOfTxnId inform, DataOutputPlus out, int version) throws IOException
+        {
+            CommandSerializers.txnId.serialize(inform.txnId, out, version);
+            KeySerializers.routingKey.serialize(inform.homeKey, out, version);
+        }
+
+        @Override
+        public InformOfTxnId deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return new InformOfTxnId(CommandSerializers.txnId.deserialize(in, version),
+                                     KeySerializers.routingKey.deserialize(in, version));
+        }
+
+        @Override
+        public long serializedSize(InformOfTxnId inform, int version)
+        {
+            return CommandSerializers.txnId.serializedSize(inform.txnId, version)
+                   + KeySerializers.routingKey.serializedSize(inform.homeKey, version);
+        }
+    };
+}
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/InformOfTxnSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/InformOfTxnSerializers.java
deleted file mode 100644
index 50e3fa8909..0000000000
--- a/src/java/org/apache/cassandra/service/accord/serializers/InformOfTxnSerializers.java
+++ /dev/null
@@ -1,82 +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.service.accord.serializers;
-
-import java.io.IOException;
-
-import accord.messages.InformOfTxn;
-import accord.messages.InformOfTxn.InformOfTxnNack;
-import accord.messages.InformOfTxn.InformOfTxnOk;
-import accord.messages.InformOfTxn.InformOfTxnReply;
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-
-public class InformOfTxnSerializers
-{
-    public static final IVersionedSerializer<InformOfTxn> request = new IVersionedSerializer<InformOfTxn>()
-    {
-        @Override
-        public void serialize(InformOfTxn inform, DataOutputPlus out, int version) throws IOException
-        {
-            CommandSerializers.txnId.serialize(inform.txnId, out, version);
-            KeySerializers.key.serialize(inform.homeKey, out, version);
-            CommandSerializers.txn.serialize(inform.txn, out, version);
-
-        }
-
-        @Override
-        public InformOfTxn deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return new InformOfTxn(CommandSerializers.txnId.deserialize(in, version),
-                                   KeySerializers.key.deserialize(in, version),
-                                   CommandSerializers.txn.deserialize(in, version));
-        }
-
-        @Override
-        public long serializedSize(InformOfTxn inform, int version)
-        {
-            return CommandSerializers.txnId.serializedSize(inform.txnId, version)
-                   + KeySerializers.key.serializedSize(inform.homeKey, version)
-                   + CommandSerializers.txn.serializedSize(inform.txn, version);
-        }
-    };
-
-    public static final IVersionedSerializer<InformOfTxnReply> reply = new IVersionedSerializer<InformOfTxnReply>()
-    {
-        @Override
-        public void serialize(InformOfTxnReply reply, DataOutputPlus out, int version) throws IOException
-        {
-            out.writeBoolean(reply.isOk());
-        }
-
-        @Override
-        public InformOfTxnReply deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return in.readBoolean() ? InformOfTxnOk.ok() : InformOfTxnNack.nack();
-        }
-
-        @Override
-        public long serializedSize(InformOfTxnReply reply, int version)
-        {
-            return TypeSizes.BOOL_SIZE;
-        }
-    };
-}
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/KeySerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/KeySerializers.java
index f35336cc0f..58f62254aa 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/KeySerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/KeySerializers.java
@@ -19,79 +19,204 @@
 package org.apache.cassandra.service.accord.serializers;
 
 import java.io.IOException;
+import java.util.function.IntFunction;
 
 import accord.api.Key;
+import accord.api.RoutingKey;
+import accord.primitives.AbstractKeys;
+import accord.primitives.AbstractRoute;
 import accord.primitives.KeyRange;
 import accord.primitives.KeyRanges;
 import accord.primitives.Keys;
+import accord.primitives.PartialRoute;
+import accord.primitives.Route;
+import accord.primitives.RoutingKeys;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.accord.TokenRange;
 import org.apache.cassandra.service.accord.api.AccordKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey;
 
 public class KeySerializers
 {
     private KeySerializers() {}
 
     public static final IVersionedSerializer<Key> key = (IVersionedSerializer<Key>) (IVersionedSerializer<?>) AccordKey.serializer;
+    public static final IVersionedSerializer<RoutingKey> routingKey = (IVersionedSerializer<RoutingKey>) (IVersionedSerializer<?>) AccordRoutingKey.serializer;
 
-    public static final IVersionedSerializer<Keys> keys = new IVersionedSerializer<Keys>()
+
+    public static final IVersionedSerializer<KeyRanges> ranges = new IVersionedSerializer<KeyRanges>()
     {
         @Override
-        public void serialize(Keys keys, DataOutputPlus out, int version) throws IOException
+        public void serialize(KeyRanges ranges, DataOutputPlus out, int version) throws IOException
         {
-            out.writeInt(keys.size());
-            for (int i=0, mi=keys.size(); i<mi; i++)
-                key.serialize(keys.get(i), out, version);
+            out.writeUnsignedVInt(ranges.size());
+            for (int i=0, mi=ranges.size(); i<mi; i++)
+                TokenRange.serializer.serialize((TokenRange) ranges.get(i), out, version);
         }
 
         @Override
-        public Keys deserialize(DataInputPlus in, int version) throws IOException
+        public KeyRanges deserialize(DataInputPlus in, int version) throws IOException
         {
-            Key[] keys = new Key[in.readInt()];
-            for (int i=0; i<keys.length; i++)
-                keys[i] = key.deserialize(in, version);
-            return Keys.of(keys);
+            KeyRange[] ranges = new KeyRange[(int)in.readUnsignedVInt()];
+            for (int i=0; i<ranges.length; i++)
+                ranges[i] = TokenRange.serializer.deserialize(in, version);
+            return KeyRanges.ofSortedAndDeoverlapped(ranges);
         }
 
         @Override
-        public long serializedSize(Keys keys, int version)
+        public long serializedSize(KeyRanges ranges, int version)
         {
-            long size = TypeSizes.sizeof(keys.size());
-            for (int i=0, mi=keys.size(); i<mi; i++)
-                size += key.serializedSize(keys.get(i), version);
+            long size = TypeSizes.sizeofUnsignedVInt(ranges.size());
+            for (int i=0, mi=ranges.size(); i<mi; i++)
+                size += TokenRange.serializer.serializedSize((TokenRange) ranges.get(i), version);
             return size;
         }
     };
 
-    public static final IVersionedSerializer<KeyRanges> ranges = new IVersionedSerializer<KeyRanges>()
+    public static final IVersionedSerializer<Keys> keys = new AbstractKeysSerializer<Key, Keys>(key, Key[]::new)
     {
+        @Override Keys deserialize(DataInputPlus in, int version, Key[] keys)
+        {
+            return Keys.SerializationSupport.create(keys);
+        }
+    };
+    public static final IVersionedSerializer<RoutingKeys> routingKeys = new AbstractKeysSerializer<RoutingKey, RoutingKeys>(routingKey, RoutingKey[]::new)
+    {
+        @Override RoutingKeys deserialize(DataInputPlus in, int version, RoutingKey[] keys)
+        {
+            return RoutingKeys.SerializationSupport.create(keys);
+        }
+    };
+    public static final IVersionedSerializer<PartialRoute> partialRoute = new AbstractKeysSerializer<RoutingKey, PartialRoute>(routingKey, RoutingKey[]::new)
+    {
+        @Override PartialRoute deserialize(DataInputPlus in, int version, RoutingKey[] keys) throws IOException
+        {
+            KeyRanges covering = ranges.deserialize(in, version);
+            RoutingKey homeKey = routingKey.deserialize(in, version);
+            return PartialRoute.SerializationSupport.create(covering, homeKey, keys);
+        }
+
         @Override
-        public void serialize(KeyRanges ranges, DataOutputPlus out, int version) throws IOException
+        public void serialize(PartialRoute keys, DataOutputPlus out, int version) throws IOException
         {
-            out.writeInt(ranges.size());
-            for (int i=0, mi=ranges.size(); i<mi; i++)
-                TokenRange.serializer.serialize((TokenRange) ranges.get(i), out, version);
+            super.serialize(keys, out, version);
+            ranges.serialize(keys.covering, out, version);
+            routingKey.serialize(keys.homeKey, out, version);
         }
 
         @Override
-        public KeyRanges deserialize(DataInputPlus in, int version) throws IOException
+        public long serializedSize(PartialRoute keys, int version)
         {
-            KeyRange[] ranges = new KeyRange[in.readInt()];
-            for (int i=0; i<ranges.length; i++)
-                ranges[i] = TokenRange.serializer.deserialize(in, version);
-            return KeyRanges.ofSortedAndDeoverlapped(ranges);
+            return super.serializedSize(keys, version)
+                   + ranges.serializedSize(keys.covering, version)
+                   + routingKey.serializedSize(keys.homeKey, version);
+        }
+    };
+    public static final IVersionedSerializer<Route> route = new AbstractKeysSerializer<RoutingKey, Route>(routingKey, RoutingKey[]::new)
+    {
+        @Override Route deserialize(DataInputPlus in, int version, RoutingKey[] keys) throws IOException
+        {
+            RoutingKey homeKey = routingKey.deserialize(in, version);
+            return Route.SerializationSupport.create(homeKey, keys);
         }
 
         @Override
-        public long serializedSize(KeyRanges ranges, int version)
+        public void serialize(Route keys, DataOutputPlus out, int version) throws IOException
         {
-            long size = TypeSizes.sizeof(ranges.size());
-            for (int i=0, mi=ranges.size(); i<mi; i++)
-                size += TokenRange.serializer.serializedSize((TokenRange) ranges.get(i), version);
+            super.serialize(keys, out, version);
+            routingKey.serialize(keys.homeKey, out, version);
+        }
+
+        @Override
+        public long serializedSize(Route keys, int version)
+        {
+            return super.serializedSize(keys, version)
+                 + routingKey.serializedSize(keys.homeKey, version);
+        }
+    };
+
+    public static final IVersionedSerializer<AbstractRoute> abstractRoute = new IVersionedSerializer<AbstractRoute>()
+    {
+        @Override
+        public void serialize(AbstractRoute t, DataOutputPlus out, int version) throws IOException
+        {
+            if (t instanceof Route)
+            {
+                out.writeByte(1);
+                route.serialize((Route)t, out, version);
+            }
+            else
+            {
+                out.writeByte(2);
+                partialRoute.serialize((PartialRoute)t, out, version);
+            }
+        }
+
+        @Override
+        public AbstractRoute deserialize(DataInputPlus in, int version) throws IOException
+        {
+            byte b = in.readByte();
+            switch (b)
+            {
+                default: throw new IOException("Corrupted input: expected byte 1 or 2, received " + b);
+                case 1: return route.deserialize(in, version);
+                case 2: return partialRoute.deserialize(in, version);
+            }
+        }
+
+        @Override
+        public long serializedSize(AbstractRoute t, int version)
+        {
+            if (t instanceof Route)
+            {
+                return 1 + route.serializedSize((Route)t, version);
+            }
+            else
+            {
+                return 1 + partialRoute.serializedSize((PartialRoute)t, version);
+            }
+        }
+    };
+
+    public static abstract class AbstractKeysSerializer<K extends RoutingKey, KS extends AbstractKeys<K, ?>> implements IVersionedSerializer<KS>
+    {
+        final IVersionedSerializer<K> keySerializer;
+        final IntFunction<K[]> allocate;
+
+        public AbstractKeysSerializer(IVersionedSerializer<K> keySerializer, IntFunction<K[]> allocate)
+        {
+            this.keySerializer = keySerializer;
+            this.allocate = allocate;
+        }
+
+        @Override
+        public void serialize(KS keys, DataOutputPlus out, int version) throws IOException
+        {
+            out.writeUnsignedVInt(keys.size());
+            for (int i=0, mi=keys.size(); i<mi; i++)
+                keySerializer.serialize(keys.get(i), out, version);
+        }
+
+        abstract KS deserialize(DataInputPlus in, int version, K[] keys) throws IOException;
+
+        @Override
+        public KS deserialize(DataInputPlus in, int version) throws IOException
+        {
+            K[] keys = allocate.apply((int)in.readUnsignedVInt());
+            for (int i=0; i<keys.length; i++)
+                keys[i] = keySerializer.deserialize(in, version);
+            return deserialize(in, version, keys);
+        }
+
+        @Override
+        public long serializedSize(KS keys, int version)
+        {
+            long size = TypeSizes.sizeofUnsignedVInt(keys.size());
+            for (int i=0, mi=keys.size(); i<mi; i++)
+                size += keySerializer.serializedSize(keys.get(i), version);
             return size;
         }
     };
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/PreacceptSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/PreacceptSerializers.java
index 625fa923ef..43dd3f8c90 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/PreacceptSerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/PreacceptSerializers.java
@@ -20,10 +20,14 @@ package org.apache.cassandra.service.accord.serializers;
 
 import java.io.IOException;
 
+import javax.annotation.Nullable;
+
 import accord.messages.PreAccept;
 import accord.messages.PreAccept.PreAcceptOk;
 import accord.messages.PreAccept.PreAcceptReply;
-import accord.primitives.Keys;
+import accord.primitives.PartialRoute;
+import accord.primitives.PartialTxn;
+import accord.primitives.Route;
 import accord.primitives.TxnId;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
@@ -31,6 +35,10 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.service.accord.serializers.TxnRequestSerializer.WithUnsyncedSerializer;
 
+import static org.apache.cassandra.service.accord.serializers.NullableSerializer.deserializeNullable;
+import static org.apache.cassandra.service.accord.serializers.NullableSerializer.serializeNullable;
+import static org.apache.cassandra.service.accord.serializers.NullableSerializer.serializedSizeNullable;
+
 public class PreacceptSerializers
 {
     private PreacceptSerializers() {}
@@ -40,23 +48,27 @@ public class PreacceptSerializers
         @Override
         public void serializeBody(PreAccept msg, DataOutputPlus out, int version) throws IOException
         {
-            CommandSerializers.txn.serialize(msg.txn, out, version);
-            KeySerializers.key.serialize(msg.homeKey, out, version);
+            CommandSerializers.partialTxn.serialize(msg.partialTxn, out, version);
+            serializeNullable(msg.route, out, version, KeySerializers.route);
+            out.writeUnsignedVInt(msg.maxEpoch - msg.minEpoch);
         }
 
         @Override
-        public PreAccept deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch, TxnId txnId, long minEpoch) throws IOException
+        public PreAccept deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey) throws IOException
         {
-            return new PreAccept(scope, waitForEpoch, txnId,
-                                 CommandSerializers.txn.deserialize(in, version),
-                                 KeySerializers.key.deserialize(in, version));
+            PartialTxn partialTxn = CommandSerializers.partialTxn.deserialize(in, version);
+            @Nullable Route route = deserializeNullable(in, version, KeySerializers.route);
+            long maxEpoch = in.readUnsignedVInt() + minEpoch;
+            return PreAccept.SerializerSupport.create(txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey,
+                                                      maxEpoch, partialTxn, route);
         }
 
         @Override
         public long serializedBodySize(PreAccept msg, int version)
         {
-            return CommandSerializers.txn.serializedSize(msg.txn, version)
-                   + KeySerializers.key.serializedSize(msg.homeKey, version);
+            return CommandSerializers.partialTxn.serializedSize(msg.partialTxn, version)
+                   + serializedSizeNullable(msg.route, version, KeySerializers.route)
+                   + TypeSizes.sizeofUnsignedVInt(msg.maxEpoch - msg.minEpoch);
         }
     };
 
@@ -65,14 +77,14 @@ public class PreacceptSerializers
         @Override
         public void serialize(PreAcceptReply reply, DataOutputPlus out, int version) throws IOException
         {
-            out.writeBoolean(reply.isOK());
-            if (!reply.isOK())
+            out.writeBoolean(reply.isOk());
+            if (!reply.isOk())
                 return;
 
             PreAcceptOk preAcceptOk = (PreAcceptOk) reply;
             CommandSerializers.txnId.serialize(preAcceptOk.txnId, out, version);
             CommandSerializers.timestamp.serialize(preAcceptOk.witnessedAt, out, version);
-            CommandSerializers.deps.serialize(preAcceptOk.deps, out, version);
+            DepsSerializer.partialDeps.serialize(preAcceptOk.deps, out, version);
         }
 
         @Override
@@ -83,20 +95,20 @@ public class PreacceptSerializers
 
             return new PreAcceptOk(CommandSerializers.txnId.deserialize(in, version),
                                    CommandSerializers.timestamp.deserialize(in, version),
-                                   CommandSerializers.deps.deserialize(in, version));
+                                   DepsSerializer.partialDeps.deserialize(in, version));
         }
 
         @Override
         public long serializedSize(PreAcceptReply reply, int version)
         {
-            long size = TypeSizes.sizeof(reply.isOK());
-            if (!reply.isOK())
+            long size = TypeSizes.sizeof(reply.isOk());
+            if (!reply.isOk())
                 return size;
 
             PreAcceptOk preAcceptOk = (PreAcceptOk) reply;
             size += CommandSerializers.txnId.serializedSize(preAcceptOk.txnId, version);
             size += CommandSerializers.timestamp.serializedSize(preAcceptOk.witnessedAt, version);
-            size += CommandSerializers.deps.serializedSize(preAcceptOk.deps, version);
+            size += DepsSerializer.partialDeps.serializedSize(preAcceptOk.deps, version);
 
             return size;
         }
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/ReadDataSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/ReadDataSerializers.java
index f1952971d2..85e633e47e 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/ReadDataSerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/ReadDataSerializers.java
@@ -25,6 +25,7 @@ import accord.messages.ReadData.ReadNack;
 import accord.messages.ReadData.ReadOk;
 import accord.messages.ReadData.ReadReply;
 import accord.primitives.Keys;
+import accord.primitives.TxnId;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -33,51 +34,52 @@ import org.apache.cassandra.service.accord.db.AccordData;
 
 public class ReadDataSerializers
 {
-    public static final IVersionedSerializer<ReadData> request = new TxnRequestSerializer<ReadData>()
+    public static final IVersionedSerializer<ReadData> request = new IVersionedSerializer<ReadData>()
     {
         @Override
-        public void serializeBody(ReadData read, DataOutputPlus out, int version) throws IOException
+        public void serialize(ReadData read, DataOutputPlus out, int version) throws IOException
         {
             CommandSerializers.txnId.serialize(read.txnId, out, version);
-            CommandSerializers.txn.serialize(read.txn, out, version);
-            CommandSerializers.deps.serialize(read.deps, out, version);
-            KeySerializers.key.serialize(read.homeKey, out, version);
-            CommandSerializers.timestamp.serialize(read.executeAt, out, version);
+            KeySerializers.keys.serialize(read.readScope, out, version);
+            out.writeUnsignedVInt(read.waitForEpoch());
+            out.writeUnsignedVInt(read.executeAtEpoch - read.waitForEpoch());
         }
 
         @Override
-        public ReadData deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch) throws IOException
+        public ReadData deserialize(DataInputPlus in, int version) throws IOException
         {
-            return ReadData.SerializerSupport.create(scope, waitForEpoch,
-                                                     CommandSerializers.txnId.deserialize(in, version),
-                                                     CommandSerializers.txn.deserialize(in, version),
-                                                     CommandSerializers.deps.deserialize(in, version),
-                                                     KeySerializers.key.deserialize(in, version),
-                                                     CommandSerializers.timestamp.deserialize(in, version));
+            TxnId txnId = CommandSerializers.txnId.deserialize(in, version);
+            Keys readScope = KeySerializers.keys.deserialize(in, version);
+            long waitForEpoch = in.readUnsignedVInt();
+            long executeAtEpoch = in.readUnsignedVInt() + waitForEpoch;
+            return ReadData.SerializerSupport.create(txnId, readScope, executeAtEpoch, waitForEpoch);
         }
 
         @Override
-        public long serializedBodySize(ReadData read, int version)
+        public long serializedSize(ReadData read, int version)
         {
             return CommandSerializers.txnId.serializedSize(read.txnId, version)
-                   + CommandSerializers.txn.serializedSize(read.txn, version)
-                   + CommandSerializers.deps.serializedSize(read.deps, version)
-                   + KeySerializers.key.serializedSize(read.homeKey, version)
-                   + CommandSerializers.timestamp.serializedSize(read.executeAt, version);
+                   + KeySerializers.keys.serializedSize(read.readScope, version)
+                   + TypeSizes.sizeofUnsignedVInt(read.waitForEpoch())
+                   + TypeSizes.sizeofUnsignedVInt(read.executeAtEpoch - read.waitForEpoch());
         }
     };
 
     public static final IVersionedSerializer<ReadReply> reply = new IVersionedSerializer<ReadReply>()
     {
+        // TODO (now): use something other than ordinal
+        final ReadNack[] nacks = ReadNack.values();
+
         @Override
         public void serialize(ReadReply reply, DataOutputPlus out, int version) throws IOException
         {
-            out.writeBoolean(reply.isOK());
-
-            // ReadNack
-            if (!reply.isOK())
+            if (!reply.isOk())
+            {
+                out.writeByte(1 + ((ReadNack) reply).ordinal());
                 return;
+            }
 
+            out.writeByte(0);
             ReadOk readOk = (ReadOk) reply;
             AccordData.serializer.serialize((AccordData) readOk.data, out, version);
         }
@@ -85,10 +87,9 @@ public class ReadDataSerializers
         @Override
         public ReadReply deserialize(DataInputPlus in, int version) throws IOException
         {
-            boolean isOK = in.readBoolean();
-
-            if (!isOK)
-                return new ReadNack();
+            int id = in.readByte();
+            if (id != 0)
+                return nacks[id - 1];
 
             return new ReadOk(AccordData.serializer.deserialize(in, version));
         }
@@ -96,13 +97,11 @@ public class ReadDataSerializers
         @Override
         public long serializedSize(ReadReply reply, int version)
         {
-            long size = TypeSizes.sizeof(reply.isOK());
-
-            if (!reply.isOK())
-                return size;
+            if (!reply.isOk())
+                return TypeSizes.BYTE_SIZE;
 
             ReadOk readOk = (ReadOk) reply;
-            return size + AccordData.serializer.serializedSize((AccordData) readOk.data, version);
+            return TypeSizes.BYTE_SIZE + AccordData.serializer.serializedSize((AccordData) readOk.data, version);
         }
     };
 }
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java
index 4c2a770f93..3acd76e36e 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java
@@ -20,6 +20,8 @@ package org.apache.cassandra.service.accord.serializers;
 
 import java.io.IOException;
 
+import javax.annotation.Nullable;
+
 import accord.api.Result;
 import accord.local.Status;
 import accord.messages.BeginRecovery;
@@ -28,10 +30,13 @@ import accord.messages.BeginRecovery.RecoverOk;
 import accord.messages.BeginRecovery.RecoverReply;
 import accord.primitives.Ballot;
 import accord.primitives.Deps;
-import accord.primitives.Keys;
+import accord.primitives.PartialDeps;
+import accord.primitives.PartialRoute;
+import accord.primitives.PartialTxn;
+import accord.primitives.Route;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
-import accord.txn.Writes;
+import accord.primitives.Writes;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -49,51 +54,45 @@ public class RecoverySerializers
         @Override
         public void serializeBody(BeginRecovery recover, DataOutputPlus out, int version) throws IOException
         {
-            CommandSerializers.txnId.serialize(recover.txnId, out, version);
-            CommandSerializers.txn.serialize(recover.txn, out, version);
-            KeySerializers.key.serialize(recover.homeKey, out, version);
+            CommandSerializers.partialTxn.serialize(recover.partialTxn, out, version);
             CommandSerializers.ballot.serialize(recover.ballot, out, version);
+            serializeNullable(KeySerializers.route, recover.route, out, version);
         }
 
         @Override
-        public BeginRecovery deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch) throws IOException
+        public BeginRecovery deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch) throws IOException
         {
-            return new BeginRecovery(scope, waitForEpoch,
-                                     CommandSerializers.txnId.deserialize(in, version),
-                                     CommandSerializers.txn.deserialize(in, version),
-                                     KeySerializers.key.deserialize(in, version),
-                                     CommandSerializers.ballot.deserialize(in, version));
+            PartialTxn partialTxn = CommandSerializers.partialTxn.deserialize(in, version);
+            Ballot ballot = CommandSerializers.ballot.deserialize(in, version);
+            @Nullable Route route = deserializeNullable(KeySerializers.route, in, version);
+            return BeginRecovery.SerializationSupport.create(txnId, scope, waitForEpoch, partialTxn, ballot, route);
         }
 
         @Override
         public long serializedBodySize(BeginRecovery recover, int version)
         {
-            return CommandSerializers.txnId.serializedSize(recover.txnId, version)
-                   + CommandSerializers.txn.serializedSize(recover.txn, version)
-                   + KeySerializers.key.serializedSize(recover.homeKey, version)
-                   + CommandSerializers.ballot.serializedSize(recover.ballot, version);
+            return CommandSerializers.partialTxn.serializedSize(recover.partialTxn, version)
+                   + CommandSerializers.ballot.serializedSize(recover.ballot, version)
+                   + serializedSizeNullable(KeySerializers.route, recover.route, version);
         }
     };
 
-    static abstract class RecoverReplySerializer<O extends RecoverOk, N extends RecoverNack> implements IVersionedSerializer<RecoverReply>
+    public static final IVersionedSerializer<RecoverReply> reply = new IVersionedSerializer<RecoverReply>()
     {
-
-        void serializeNack(N recoverNack, DataOutputPlus out, int version) throws IOException
+        void serializeNack(RecoverNack recoverNack, DataOutputPlus out, int version) throws IOException
         {
             CommandSerializers.ballot.serialize(recoverNack.supersededBy, out, version);
         }
 
-
-        void serializeOk(O recoverOk, DataOutputPlus out, int version) throws IOException
+        void serializeOk(RecoverOk recoverOk, DataOutputPlus out, int version) throws IOException
         {
             CommandSerializers.txnId.serialize(recoverOk.txnId, out, version);
             CommandSerializers.status.serialize(recoverOk.status, out, version);
             CommandSerializers.ballot.serialize(recoverOk.accepted, out, version);
             serializeNullable(CommandSerializers.timestamp, recoverOk.executeAt, out, version);
-            CommandSerializers.deps.serialize(recoverOk.deps, out, version);
-            // FIXME: nullable support is only to support InvalidateResponses, where these are always null
-            serializeNullable(CommandSerializers.deps, recoverOk.earlierCommittedWitness, out, version);
-            serializeNullable(CommandSerializers.deps, recoverOk.earlierAcceptedNoWitness, out, version);
+            DepsSerializer.partialDeps.serialize(recoverOk.deps, out, version);
+            DepsSerializer.deps.serialize(recoverOk.earlierCommittedWitness, out, version);
+            DepsSerializer.deps.serialize(recoverOk.earlierAcceptedNoWitness, out, version);
             out.writeBoolean(recoverOk.rejectsFastPath);
             serializeNullable(CommandSerializers.writes, recoverOk.writes, out, version);
             serializeNullable(AccordData.serializer, (AccordData) recoverOk.result, out, version);
@@ -102,29 +101,22 @@ public class RecoverySerializers
         @Override
         public final void serialize(RecoverReply reply, DataOutputPlus out, int version) throws IOException
         {
-            out.writeBoolean(reply.isOK());
-            if (!reply.isOK())
-                serializeNack((N) reply, out, version);
+            out.writeBoolean(reply.isOk());
+            if (!reply.isOk())
+                serializeNack((RecoverNack) reply, out, version);
             else
-                serializeOk((O) reply, out, version);
+                serializeOk((RecoverOk) reply, out, version);
+        }
+
+        RecoverNack deserializeNack(Ballot supersededBy, DataInputPlus in, int version)
+        {
+            return new RecoverNack(supersededBy);
         }
 
-        abstract N deserializeNack(Ballot supersededBy,
-                                   DataInputPlus in,
-                                   int version) throws IOException;
-
-        abstract O deserializeOk(TxnId txnId,
-                                 Status status,
-                                 Ballot accepted,
-                                 Timestamp executeAt,
-                                 Deps deps,
-                                 Deps earlierCommittedWitness,
-                                 Deps earlierAcceptedNoWitness,
-                                 boolean rejectsFastPath,
-                                 Writes writes,
-                                 Result result,
-                                 DataInputPlus in,
-                                 int version) throws IOException;
+        RecoverOk deserializeOk(TxnId txnId, Status status, Ballot accepted, Timestamp executeAt, PartialDeps deps, Deps earlierCommittedWitness, Deps earlierAcceptedNoWitness, boolean rejectsFastPath, Writes writes, Result result, DataInputPlus in, int version)
+        {
+            return new RecoverOk(txnId, status, accepted, executeAt, deps, earlierCommittedWitness, earlierAcceptedNoWitness, rejectsFastPath, writes, result);
+        }
 
         @Override
         public final RecoverReply deserialize(DataInputPlus in, int version) throws IOException
@@ -137,9 +129,9 @@ public class RecoverySerializers
                                  CommandSerializers.status.deserialize(in, version),
                                  CommandSerializers.ballot.deserialize(in, version),
                                  deserializeNullable(CommandSerializers.timestamp, in, version),
-                                 CommandSerializers.deps.deserialize(in, version),
-                                 deserializeNullable(CommandSerializers.deps, in, version),
-                                 deserializeNullable(CommandSerializers.deps, in, version),
+                                 DepsSerializer.partialDeps.deserialize(in, version),
+                                 DepsSerializer.deps.deserialize(in, version),
+                                 DepsSerializer.deps.deserialize(in, version),
                                  in.readBoolean(),
                                  deserializeNullable(CommandSerializers.writes, in, version),
                                  deserializeNullable(AccordData.serializer, in, version),
@@ -147,20 +139,20 @@ public class RecoverySerializers
                                  version);
         }
 
-        long serializedNackSize(N recoverNack, int version)
+        long serializedNackSize(RecoverNack recoverNack, int version)
         {
             return CommandSerializers.ballot.serializedSize(recoverNack.supersededBy, version);
         }
 
-        long serializedOkSize(O recoverOk, int version)
+        long serializedOkSize(RecoverOk recoverOk, int version)
         {
             long size = CommandSerializers.txnId.serializedSize(recoverOk.txnId, version);
             size += CommandSerializers.status.serializedSize(recoverOk.status, version);
             size += CommandSerializers.ballot.serializedSize(recoverOk.accepted, version);
             size += serializedSizeNullable(CommandSerializers.timestamp, recoverOk.executeAt, version);
-            size += CommandSerializers.deps.serializedSize(recoverOk.deps, version);
-            size += serializedSizeNullable(CommandSerializers.deps, recoverOk.earlierCommittedWitness, version);
-            size += serializedSizeNullable(CommandSerializers.deps, recoverOk.earlierAcceptedNoWitness, version);
+            size += DepsSerializer.partialDeps.serializedSize(recoverOk.deps, version);
+            size += DepsSerializer.deps.serializedSize(recoverOk.earlierCommittedWitness, version);
+            size += DepsSerializer.deps.serializedSize(recoverOk.earlierAcceptedNoWitness, version);
             size += TypeSizes.sizeof(recoverOk.rejectsFastPath);
             size += serializedSizeNullable(CommandSerializers.writes, recoverOk.writes, version);
             size += serializedSizeNullable(AccordData.serializer, (AccordData) recoverOk.result, version);
@@ -170,23 +162,8 @@ public class RecoverySerializers
         @Override
         public final long serializedSize(RecoverReply reply, int version)
         {
-            return TypeSizes.sizeof(reply.isOK())
-                   + (reply.isOK() ? serializedOkSize((O) reply, version) : serializedNackSize((N) reply, version));
-        }
-    }
-
-    public static final IVersionedSerializer<RecoverReply> reply = new RecoverReplySerializer()
-    {
-        @Override
-        RecoverNack deserializeNack(Ballot supersededBy, DataInputPlus in, int version)
-        {
-            return new RecoverNack(supersededBy);
-        }
-
-        @Override
-        RecoverOk deserializeOk(TxnId txnId, Status status, Ballot accepted, Timestamp executeAt, Deps deps, Deps earlierCommittedWitness, Deps earlierAcceptedNoWitness, boolean rejectsFastPath, Writes writes, Result result, DataInputPlus in, int version)
-        {
-            return new RecoverOk(txnId, status, accepted, executeAt, deps, earlierCommittedWitness, earlierAcceptedNoWitness, rejectsFastPath, writes, result);
+            return TypeSizes.sizeof(reply.isOk())
+                   + (reply.isOk() ? serializedOkSize((RecoverOk) reply, version) : serializedNackSize((RecoverNack) reply, version));
         }
     };
 }
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/TopologySerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/TopologySerializers.java
index da6d0cfdd3..699fb22be4 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/TopologySerializers.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/TopologySerializers.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.service.accord.serializers;
 import java.io.IOException;
 
 import accord.local.Node;
-import accord.messages.TxnRequest;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.marshal.ValueAccessor;
 import org.apache.cassandra.io.IVersionedSerializer;
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java b/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java
index 6f6586f735..0d133f78af 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java
@@ -21,19 +21,20 @@ package org.apache.cassandra.service.accord.serializers;
 import java.io.IOException;
 
 import accord.messages.TxnRequest;
-import accord.primitives.Keys;
+import accord.primitives.PartialRoute;
 import accord.primitives.TxnId;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
-public abstract class TxnRequestSerializer<T extends TxnRequest> implements IVersionedSerializer<T>
+public abstract class TxnRequestSerializer<T extends TxnRequest<?>> implements IVersionedSerializer<T>
 {
     void serializeHeader(T msg, DataOutputPlus out, int version) throws IOException
     {
-        KeySerializers.keys.serialize(msg.scope(), out, version);
-        out.writeLong(msg.waitForEpoch());
+        CommandSerializers.txnId.serialize(msg.txnId, out, version);
+        KeySerializers.partialRoute.serialize(msg.scope, out, version);
+        out.writeUnsignedVInt(msg.waitForEpoch);
     }
 
     public abstract void serializeBody(T msg, DataOutputPlus out, int version) throws IOException;
@@ -45,19 +46,23 @@ public abstract class TxnRequestSerializer<T extends TxnRequest> implements IVer
         serializeBody(msg, out, version);
     }
 
-    public abstract T deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch) throws IOException;
+    public abstract T deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch) throws IOException;
 
     @Override
     public final T deserialize(DataInputPlus in, int version) throws IOException
     {
-        Keys scope = KeySerializers.keys.deserialize(in, version);
-        long waitForEpoch = in.readLong();
-        return deserializeBody(in, version, scope, waitForEpoch);
+        TxnId txnId = CommandSerializers.txnId.deserialize(in, version);
+        PartialRoute scope = KeySerializers.partialRoute.deserialize(in, version);
+        // TODO: there should be a base epoch
+        long waitForEpoch = in.readUnsignedVInt();
+        return deserializeBody(in, version, txnId, scope, waitForEpoch);
     }
 
     long serializedHeaderSize(T msg, int version)
     {
-        return KeySerializers.keys.serializedSize(msg.scope(), version) + TypeSizes.LONG_SIZE;
+        return CommandSerializers.txnId.serializedSize(msg.txnId, version)
+               + KeySerializers.partialRoute.serializedSize(msg.scope(), version) +
+               TypeSizes.sizeofUnsignedVInt(msg.waitForEpoch);
     }
 
     public abstract long serializedBodySize(T msg, int version);
@@ -68,32 +73,32 @@ public abstract class TxnRequestSerializer<T extends TxnRequest> implements IVer
         return serializedHeaderSize(msg, version) + serializedBodySize(msg, version);
     }
 
-    public static abstract class WithUnsyncedSerializer<T extends TxnRequest.WithUnsynced> extends TxnRequestSerializer<T>
+    public static abstract class WithUnsyncedSerializer<T extends TxnRequest.WithUnsynced<?>> extends TxnRequestSerializer<T>
     {
         @Override
         void serializeHeader(T msg, DataOutputPlus out, int version) throws IOException
         {
             super.serializeHeader(msg, out, version);
-            CommandSerializers.txnId.serialize(msg.txnId, out, version);
-            out.writeLong(msg.minEpoch);
+            out.writeUnsignedVInt(msg.minEpoch);
+            out.writeBoolean(msg.doNotComputeProgressKey);
         }
 
-        public abstract T deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch, TxnId txnId, long minEpoch) throws IOException;
+        public abstract T deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey) throws IOException;
 
         @Override
-        public final T deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch) throws IOException
+        public final T deserializeBody(DataInputPlus in, int version, TxnId txnId, PartialRoute scope, long waitForEpoch) throws IOException
         {
-            TxnId txnId = CommandSerializers.txnId.deserialize(in, version);
-            long minEpoch = in.readLong();
-            return deserializeBody(in, version, scope, waitForEpoch, txnId, minEpoch);
+            long minEpoch = in.readUnsignedVInt();
+            boolean doNotComputeProgressKey = in.readBoolean();
+            return deserializeBody(in, version, txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey);
         }
 
         @Override
         long serializedHeaderSize(T msg, int version)
         {
             long size = super.serializedHeaderSize(msg, version);
-            size += CommandSerializers.txnId.serializedSize(msg.txnId, version);
-            size += TypeSizes.LONG_SIZE;
+            size += TypeSizes.sizeofUnsignedVInt(msg.minEpoch);
+            size += TypeSizes.BOOL_SIZE;
             return size;
         }
     }
diff --git a/src/java/org/apache/cassandra/service/accord/serializers/WaitOnCommitSerializer.java b/src/java/org/apache/cassandra/service/accord/serializers/WaitOnCommitSerializer.java
index 089e70113f..2203914a01 100644
--- a/src/java/org/apache/cassandra/service/accord/serializers/WaitOnCommitSerializer.java
+++ b/src/java/org/apache/cassandra/service/accord/serializers/WaitOnCommitSerializer.java
@@ -22,31 +22,36 @@ import java.io.IOException;
 
 import accord.messages.WaitOnCommit;
 import accord.messages.WaitOnCommit.WaitOnCommitOk;
-import accord.primitives.Keys;
+import accord.primitives.RoutingKeys;
+import accord.primitives.TxnId;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 public class WaitOnCommitSerializer
 {
-    public static final IVersionedSerializer<WaitOnCommit> request = new TxnRequestSerializer<WaitOnCommit>()
+    public static final IVersionedSerializer<WaitOnCommit> request = new IVersionedSerializer<WaitOnCommit>()
     {
         @Override
-        public void serializeBody(WaitOnCommit wait, DataOutputPlus out, int version) throws IOException
+        public void serialize(WaitOnCommit wait, DataOutputPlus out, int version) throws IOException
         {
             CommandSerializers.txnId.serialize(wait.txnId, out, version);
+            KeySerializers.routingKeys.serialize(wait.scope, out, version);
         }
 
         @Override
-        public WaitOnCommit deserializeBody(DataInputPlus in, int version, Keys scope, long waitForEpoch) throws IOException
+        public WaitOnCommit deserialize(DataInputPlus in, int version) throws IOException
         {
-            return WaitOnCommit.SerializerSupport.create(scope, waitForEpoch, CommandSerializers.txnId.deserialize(in, version));
+            TxnId txnId = CommandSerializers.txnId.deserialize(in, version);
+            RoutingKeys scope = KeySerializers.routingKeys.deserialize(in, version);
+            return WaitOnCommit.SerializerSupport.create(txnId, scope);
         }
 
         @Override
-        public long serializedBodySize(WaitOnCommit wait, int version)
+        public long serializedSize(WaitOnCommit wait, int version)
         {
-            return CommandSerializers.txnId.serializedSize(wait.txnId, version);
+            return CommandSerializers.txnId.serializedSize(wait.txnId, version)
+                   + KeySerializers.routingKeys.serializedSize(wait.scope, version);
         }
     };
 
@@ -55,7 +60,6 @@ public class WaitOnCommitSerializer
         @Override
         public void serialize(WaitOnCommitOk ok, DataOutputPlus out, int version) throws IOException
         {
-
         }
 
         @Override
diff --git a/src/java/org/apache/cassandra/service/accord/store/AbstractStoredField.java b/src/java/org/apache/cassandra/service/accord/store/AbstractStoredField.java
index 5fde9c5198..b09bc31827 100644
--- a/src/java/org/apache/cassandra/service/accord/store/AbstractStoredField.java
+++ b/src/java/org/apache/cassandra/service/accord/store/AbstractStoredField.java
@@ -31,12 +31,12 @@ public abstract class AbstractStoredField
 
     private byte flag;
 
-    public AbstractStoredField(AccordState.Kind kind)
+    public AbstractStoredField(AccordState.ReadWrite readWrite)
     {
         this.flag = 0;
-        if (kind == AccordState.Kind.WRITE_ONLY)
+        if (readWrite == AccordState.ReadWrite.WRITE_ONLY)
             set(WRITE_ONLY_FLAG);
-        if (kind == AccordState.Kind.READ_ONLY)
+        if (readWrite == AccordState.ReadWrite.READ_ONLY)
             set(READ_ONLY_FLAG);
     }
 
diff --git a/src/java/org/apache/cassandra/service/accord/store/StoredBoolean.java b/src/java/org/apache/cassandra/service/accord/store/StoredBoolean.java
index a6003c4361..fb81da9df4 100644
--- a/src/java/org/apache/cassandra/service/accord/store/StoredBoolean.java
+++ b/src/java/org/apache/cassandra/service/accord/store/StoredBoolean.java
@@ -25,12 +25,12 @@ import org.apache.cassandra.utils.ObjectSizes;
 
 public class StoredBoolean extends AbstractStoredField
 {
-    public static final long EMPTY_SIZE = ObjectSizes.measure(new StoredBoolean(AccordState.Kind.FULL));
+    public static final long EMPTY_SIZE = ObjectSizes.measure(new StoredBoolean(AccordState.ReadWrite.FULL));
     protected boolean value;
 
-    public StoredBoolean(AccordState.Kind kind)
+    public StoredBoolean(AccordState.ReadWrite readWrite)
     {
-        super(kind);
+        super(readWrite);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/service/accord/store/StoredLong.java b/src/java/org/apache/cassandra/service/accord/store/StoredLong.java
index 656cd5004b..3534d68b93 100644
--- a/src/java/org/apache/cassandra/service/accord/store/StoredLong.java
+++ b/src/java/org/apache/cassandra/service/accord/store/StoredLong.java
@@ -25,13 +25,13 @@ import org.apache.cassandra.utils.ObjectSizes;
 
 public class StoredLong extends AbstractStoredField
 {
-    public static final long EMPTY_SIZE = ObjectSizes.measure(new StoredLong(AccordState.Kind.FULL));
+    public static final long EMPTY_SIZE = ObjectSizes.measure(new StoredLong(AccordState.ReadWrite.FULL));
 
     protected long value;
 
-    public StoredLong(AccordState.Kind kind)
+    public StoredLong(AccordState.ReadWrite readWrite)
     {
-        super(kind);
+        super(readWrite);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/service/accord/store/StoredNavigableMap.java b/src/java/org/apache/cassandra/service/accord/store/StoredNavigableMap.java
index 351dd7c084..22c5e2ec4a 100644
--- a/src/java/org/apache/cassandra/service/accord/store/StoredNavigableMap.java
+++ b/src/java/org/apache/cassandra/service/accord/store/StoredNavigableMap.java
@@ -38,15 +38,15 @@ import org.apache.cassandra.utils.ObjectSizes;
  */
 public class StoredNavigableMap<K extends Comparable<?>, V> extends AbstractStoredField
 {
-    private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new StoredNavigableMap<>(AccordState.Kind.FULL));
+    private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new StoredNavigableMap<>(AccordState.ReadWrite.FULL));
     private NavigableMap<K, V> map = null;
     private NavigableMap<K, V> view = null;
     private NavigableMap<K, V> additions = null;
-    private NavigableSet<K> deletions = null;
+    private NavigableMap<K, V> deletions = null;
 
-    public StoredNavigableMap(AccordState.Kind kind)
+    public StoredNavigableMap(AccordState.ReadWrite readWrite)
     {
-        super(kind);
+        super(readWrite);
     }
 
     @Override
@@ -126,8 +126,27 @@ public class StoredNavigableMap<K extends Comparable<?>, V> extends AbstractStor
         if (!wasCleared())
         {
             if (deletions == null)
-                deletions = new TreeSet<>();
-            deletions.add(key);
+                deletions = new TreeMap<>();
+            deletions.put(key, null);
+        }
+        if (additions != null)
+            additions.remove(key);
+    }
+
+    // TODO: this is a kludge, but will suffice until we can more fully rework efficiency of waitingOn collections
+    // this is semantically equivalent to blindRemove(key) but stores the value we believe was bound to key on removal
+    // so that it can be used by forEachDeletion
+    public void blindRemove(K key, V value)
+    {
+        preBlindChange();
+        if (hasValue())
+            map.remove(key);
+
+        if (!wasCleared())
+        {
+            if (deletions == null)
+                deletions = new TreeMap<>();
+            deletions.put(key, value);
         }
         if (additions != null)
             additions.remove(key);
@@ -180,6 +199,12 @@ public class StoredNavigableMap<K extends Comparable<?>, V> extends AbstractStor
     }
 
     public void forEachDeletion(Consumer<K> consumer)
+    {
+        if (deletions != null)
+            deletions.keySet().forEach(consumer);
+    }
+
+    public void forEachDeletion(BiConsumer<K, V> consumer)
     {
         if (deletions != null)
             deletions.forEach(consumer);
diff --git a/src/java/org/apache/cassandra/service/accord/store/StoredSet.java b/src/java/org/apache/cassandra/service/accord/store/StoredSet.java
index a0be95a8e7..699c8bf48f 100644
--- a/src/java/org/apache/cassandra/service/accord/store/StoredSet.java
+++ b/src/java/org/apache/cassandra/service/accord/store/StoredSet.java
@@ -35,18 +35,18 @@ import org.apache.cassandra.utils.ObjectSizes;
 public abstract class StoredSet<T, S extends Set<T>> extends AbstractStoredField
 {
     private S set = null;
-    private Set<T> view = null;
+    private S view = null;
     private Set<T> additions = null;
     private Set<T> deletions = null;
 
     abstract S createDataSet();
     abstract Set<T> createMetaSet();
-    abstract Set<T> createView(S data);
+    abstract S createView(S data);
     abstract long emptySize();
 
-    public StoredSet(AccordState.Kind kind)
+    public StoredSet(AccordState.ReadWrite readWrite)
     {
-        super(kind);
+        super(readWrite);
     }
 
     @Override
@@ -95,7 +95,7 @@ public abstract class StoredSet<T, S extends Set<T>> extends AbstractStoredField
         setInternal(set);
     }
 
-    public Set<T> getView()
+    public S getView()
     {
         preGet();
         return view;
@@ -179,11 +179,17 @@ public abstract class StoredSet<T, S extends Set<T>> extends AbstractStoredField
         return size;
     }
 
-    public static class Navigable<T extends Comparable<?>> extends StoredSet<T, NavigableSet<T>>
+    public interface Changes<T>
     {
-        private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new Navigable<>(AccordState.Kind.FULL));
+        void forEachAddition(Consumer<T> consumer);
+        void forEachDeletion(Consumer<T> consumer);
+    }
+
+    public static class Navigable<T extends Comparable<?>> extends StoredSet<T, NavigableSet<T>> implements Changes<T>
+    {
+        private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new Navigable<>(AccordState.ReadWrite.FULL));
 
-        public Navigable(AccordState.Kind kind) { super(kind); }
+        public Navigable(AccordState.ReadWrite readWrite) { super(readWrite); }
 
         @Override
         NavigableSet<T> createDataSet()
@@ -210,11 +216,11 @@ public abstract class StoredSet<T, S extends Set<T>> extends AbstractStoredField
         }
     }
 
-    public static class DeterministicIdentity<T> extends StoredSet<T, DeterministicIdentitySet<T>>
+    public static class DeterministicIdentity<T> extends StoredSet<T, Set<T>> implements Changes<T>
     {
-        private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new DeterministicIdentity<>(AccordState.Kind.FULL));
+        private static final long EMPTY_SIZE = ObjectSizes.measureDeep(new DeterministicIdentity<>(AccordState.ReadWrite.FULL));
 
-        public DeterministicIdentity(AccordState.Kind kind) { super(kind); }
+        public DeterministicIdentity(AccordState.ReadWrite readWrite) { super(readWrite); }
 
         @Override
         DeterministicIdentitySet<T> createDataSet()
@@ -229,7 +235,7 @@ public abstract class StoredSet<T, S extends Set<T>> extends AbstractStoredField
         }
 
         @Override
-        Set<T> createView(DeterministicIdentitySet<T> data)
+        Set<T> createView(Set<T> data)
         {
             return Collections.unmodifiableSet(data);
         }
diff --git a/src/java/org/apache/cassandra/service/accord/store/StoredValue.java b/src/java/org/apache/cassandra/service/accord/store/StoredValue.java
index 820d9f3e40..3e08c8b43a 100644
--- a/src/java/org/apache/cassandra/service/accord/store/StoredValue.java
+++ b/src/java/org/apache/cassandra/service/accord/store/StoredValue.java
@@ -26,12 +26,12 @@ import org.apache.cassandra.utils.ObjectSizes;
 
 public class StoredValue<T> extends AbstractStoredField
 {
-    public static final long EMPTY_SIZE = ObjectSizes.measure(new StoredValue<>(AccordState.Kind.FULL));
+    public static final long EMPTY_SIZE = ObjectSizes.measure(new StoredValue<>(AccordState.ReadWrite.FULL));
     protected T value;
 
-    public StoredValue(AccordState.Kind kind)
+    public StoredValue(AccordState.ReadWrite readWrite)
     {
-        super(kind);
+        super(readWrite);
     }
 
     @Override
@@ -94,9 +94,9 @@ public class StoredValue<T> extends AbstractStoredField
     {
         T previous;
 
-        public HistoryPreserving(AccordState.Kind kind)
+        public HistoryPreserving(AccordState.ReadWrite readWrite)
         {
-            super(kind);
+            super(readWrite);
         }
 
         public T previous()
diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java
index b23283ff55..3fb6584d84 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java
@@ -45,9 +45,9 @@ import accord.coordinate.Preempted;
 import accord.local.Status;
 import accord.messages.Commit;
 import accord.primitives.Keys;
+import accord.primitives.Txn;
 import accord.primitives.TxnId;
 import accord.topology.Topologies;
-import accord.txn.Txn;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.partitions.FilteredPartition;
@@ -257,7 +257,7 @@ public class AccordIntegrationTest extends TestBaseImpl
             cluster.filters().verbs(Verb.ACCORD_COMMIT_REQ.id).messagesMatching((from, to, iMessage) -> cluster.get(from).callOnInstance(() -> {
                 Message<?> msg = Instance.deserializeMessage(iMessage);
                 if (msg.payload instanceof Commit)
-                    return ((Commit) msg.payload).read;
+                    return ((Commit) msg.payload).read != null;
                 return false;
             })).drop();
 
@@ -298,7 +298,7 @@ public class AccordIntegrationTest extends TestBaseImpl
         cluster.stream().filter(i -> !i.isShutdown()).forEach(inst -> {
             while (timeout.get() == null)
             {
-                SimpleQueryResult pending = inst.executeInternalWithResult("SELECT store_generation, store_index, txn_id, status FROM system_accord.commands WHERE status < ? ALLOW FILTERING", Status.Executed.ordinal());
+                SimpleQueryResult pending = inst.executeInternalWithResult("SELECT store_generation, store_index, txn_id, status FROM system_accord.commands WHERE status < ? ALLOW FILTERING", Status.PreApplied.ordinal());
                 pending = QueryResultUtil.map(pending, ImmutableMap.of(
                 "txn_id", (ByteBuffer bb) -> AccordKeyspace.deserializeTimestampOrNull(bb, TxnId::new),
                 "status", (Integer ordinal) -> Status.values()[ordinal]
diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java
index 071fd25a3a..21ac6175fb 100644
--- a/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java
+++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java
@@ -62,7 +62,8 @@ public class InterceptClasses implements BiFunction<String, byte[], byte[]>
                                                             "|org[/.]apache[/.]cassandra[/.]distributed[/.]impl[/.]DirectStreamingConnectionFactory.*" +
                                                             "|org[/.]apache[/.]cassandra[/.]db[/.]commitlog[/.].*" +
                                                             "|org[/.]apache[/.]cassandra[/.]service[/.]paxos[/.].*" +
-                                                            "|accord[/.].*");
+                                                            "|accord[/.].*"
+    );
 
     private static final Pattern GLOBAL_METHODS = Pattern.compile("org[/.]apache[/.]cassandra[/.](?!simulator[/.]).*" +
                                                                   "|org[/.]apache[/.]cassandra[/.]simulator[/.]test[/.].*" +
diff --git a/test/simulator/main/org/apache/cassandra/simulator/paxos/PairOfSequencesAccordSimulation.java b/test/simulator/main/org/apache/cassandra/simulator/paxos/PairOfSequencesAccordSimulation.java
index e6e79c290c..c7fa1853fb 100644
--- a/test/simulator/main/org/apache/cassandra/simulator/paxos/PairOfSequencesAccordSimulation.java
+++ b/test/simulator/main/org/apache/cassandra/simulator/paxos/PairOfSequencesAccordSimulation.java
@@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory;
 
 import accord.coordinate.Preempted;
 import accord.coordinate.Timeout;
-import accord.txn.Txn;
+import accord.primitives.Txn;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.Clustering;
diff --git a/test/unit/org/apache/cassandra/service/accord/AccordCommandStoreTest.java b/test/unit/org/apache/cassandra/service/accord/AccordCommandStoreTest.java
index 780366cc78..53b762de2f 100644
--- a/test/unit/org/apache/cassandra/service/accord/AccordCommandStoreTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/AccordCommandStoreTest.java
@@ -32,10 +32,10 @@ import org.slf4j.LoggerFactory;
 import accord.api.Key;
 import accord.local.Command;
 import accord.local.Status;
-import accord.primitives.Deps;
+import accord.primitives.PartialDeps;
+import accord.primitives.PartialTxn;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -45,10 +45,11 @@ import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
 
+import static accord.local.Status.Durability.Durable;
 import static org.apache.cassandra.cql3.statements.schema.CreateTableStatement.parse;
 import static org.apache.cassandra.service.accord.AccordTestUtils.ballot;
 import static org.apache.cassandra.service.accord.AccordTestUtils.createAccordCommandStore;
-import static org.apache.cassandra.service.accord.AccordTestUtils.createTxn;
+import static org.apache.cassandra.service.accord.AccordTestUtils.createPartialTxn;
 import static org.apache.cassandra.service.accord.AccordTestUtils.processCommandResult;
 import static org.apache.cassandra.service.accord.AccordTestUtils.timestamp;
 import static org.apache.cassandra.service.accord.AccordTestUtils.txnId;
@@ -76,36 +77,36 @@ public class AccordCommandStoreTest
     public void commandLoadSave() throws Throwable
     {
         AtomicLong clock = new AtomicLong(0);
-        Txn depTxn = createTxn(0);
+        PartialTxn depTxn = createPartialTxn(0);
         Key key = depTxn.keys().get(0);
         AccordCommandStore commandStore = createAccordCommandStore(clock::incrementAndGet, "ks", "tbl");
 
-        Deps.OrderedBuilder builder = Deps.orderedBuilder(false);
+        PartialDeps.OrderedBuilder builder = PartialDeps.orderedBuilder(depTxn.covering(), false);
         builder.add(key, txnId(1, clock.incrementAndGet(), 0, 1));
-        Deps dependencies = builder.build();
+        PartialDeps dependencies = builder.build();
         QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, c, v) VALUES (0, 0, 1)");
 
         TxnId oldTxnId1 = txnId(1, clock.incrementAndGet(), 0, 1);
         TxnId oldTxnId2 = txnId(1, clock.incrementAndGet(), 0, 1);
         TxnId oldTimestamp = txnId(1, clock.incrementAndGet(), 0, 1);
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
-        AccordCommand command = new AccordCommand(commandStore, txnId).initialize();
-        command.txn(createTxn(0));
-        command.homeKey(key);
-        command.progressKey(key);
-        command.isGloballyPersistent(true);
-        command.promised(ballot(1, clock.incrementAndGet(), 0, 1));
-        command.accepted(ballot(1, clock.incrementAndGet(), 0, 1));
-        command.executeAt(timestamp(1, clock.incrementAndGet(), 0, 1));
-        command.savedDeps(dependencies);
-        command.status(Status.Accepted);
-        command.addWaitingOnCommit(new AccordCommand(commandStore, oldTxnId1).initialize());
-        command.addWaitingOnApplyIfAbsent(new AccordCommand(commandStore, oldTxnId2).initialize());
+        AccordCommand command = new AccordCommand(txnId).initialize();
+        command.setPartialTxn(createPartialTxn(0));
+        command.homeKey(key.toRoutingKey());
+        command.progressKey(key.toRoutingKey());
+        command.setDurability(Durable);
+        command.setPromised(ballot(1, clock.incrementAndGet(), 0, 1));
+        command.setAccepted(ballot(1, clock.incrementAndGet(), 0, 1));
+        command.setExecuteAt(timestamp(1, clock.incrementAndGet(), 0, 1));
+        command.setPartialDeps(dependencies);
+        command.setStatus(Status.Accepted);
+        command.addWaitingOnCommit(oldTxnId1);
+        command.addWaitingOnApplyIfAbsent(oldTxnId2, oldTimestamp);
         command.storedListeners.clear();
-        command.addListener(new AccordCommand(commandStore, oldTxnId1));
-        processCommandResult(command);
+        command.addListener(new AccordCommand(oldTxnId1));
+        processCommandResult(commandStore, command);
 
-        AccordKeyspace.getCommandMutation(command, commandStore.nextSystemTimestampMicros()).apply();
+        AccordKeyspace.getCommandMutation(commandStore, command, commandStore.nextSystemTimestampMicros()).apply();
         logger.info("E: {}", command);
         Command actual = AccordKeyspace.loadCommand(commandStore, txnId);
         logger.info("A: {}", actual);
@@ -120,16 +121,16 @@ public class AccordCommandStoreTest
         AccordCommandStore commandStore = createAccordCommandStore(clock::incrementAndGet, "ks", "tbl");
         Timestamp maxTimestamp = timestamp(1, clock.incrementAndGet(), 0, 1);
 
-        Txn txn = createTxn(1);
+        PartialTxn txn = createPartialTxn(1);
         PartitionKey key = (PartitionKey) Iterables.getOnlyElement(txn.keys());
         TxnId txnId1 = txnId(1, clock.incrementAndGet(), 0, 1);
         TxnId txnId2 = txnId(1, clock.incrementAndGet(), 0, 1);
-        AccordCommand command1 = new AccordCommand(commandStore, txnId1).initialize();
-        AccordCommand command2 = new AccordCommand(commandStore, txnId2).initialize();
-        command1.txn(txn);
-        command2.txn(txn);
-        command1.executeAt(timestamp(1, clock.incrementAndGet(), 0, 1));
-        command2.executeAt(timestamp(1, clock.incrementAndGet(), 0, 1));
+        AccordCommand command1 = new AccordCommand(txnId1).initialize();
+        AccordCommand command2 = new AccordCommand(txnId2).initialize();
+        command1.setPartialTxn(txn);
+        command2.setPartialTxn(txn);
+        command1.setExecuteAt(timestamp(1, clock.incrementAndGet(), 0, 1));
+        command2.setExecuteAt(timestamp(1, clock.incrementAndGet(), 0, 1));
 
         AccordCommandsForKey cfk = new AccordCommandsForKey(commandStore, key).initialize();
         cfk.updateMax(maxTimestamp);
@@ -142,7 +143,7 @@ public class AccordCommandStoreTest
         cfk.register(command1);
         cfk.register(command2);
 
-        AccordKeyspace.getCommandsForKeyMutation(cfk, commandStore.nextSystemTimestampMicros()).apply();
+        AccordKeyspace.getCommandsForKeyMutation(commandStore, cfk, commandStore.nextSystemTimestampMicros()).apply();
         logger.info("E: {}", cfk);
         AccordCommandsForKey actual = AccordKeyspace.loadCommandsForKey(commandStore, key);
         logger.info("A: {}", actual);
@@ -155,7 +156,7 @@ public class AccordCommandStoreTest
     {
         AtomicLong clock = new AtomicLong(0);
         AccordCommandStore commandStore = createAccordCommandStore(clock::incrementAndGet, "ks", "tbl");
-        Txn txn = createTxn(1);
+        PartialTxn txn = createPartialTxn(1);
         PartitionKey key = (PartitionKey) Iterables.getOnlyElement(txn.keys());
 
         AccordCommandsForKey.WriteOnly writeOnlyCfk = new AccordCommandsForKey.WriteOnly(commandStore, key);
@@ -169,7 +170,7 @@ public class AccordCommandStoreTest
             writeOnlyCfk.updateMax(maxTimestamp);
         }
 
-        AccordKeyspace.getCommandsForKeyMutation(writeOnlyCfk, commandStore.nextSystemTimestampMicros()).apply();
+        AccordKeyspace.getCommandsForKeyMutation(commandStore, writeOnlyCfk, commandStore.nextSystemTimestampMicros()).apply();
         AccordCommandsForKey fullCfk = AccordKeyspace.loadCommandsForKey(commandStore, key);
 
         Assert.assertEquals(expected, fullCfk.blindWitnessed.getView());
diff --git a/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java b/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java
index 4600a3005f..e48cda6e84 100644
--- a/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java
@@ -18,28 +18,32 @@
 
 package org.apache.cassandra.service.accord;
 
+import java.util.Collections;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Consumer;
 
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import accord.api.Key;
+import accord.api.RoutingKey;
 import accord.local.Command;
-import accord.local.CommandStore;
 import accord.local.CommandsForKey;
 import accord.local.Node;
+import accord.local.PreLoadContext;
 import accord.local.Status;
 import accord.messages.Accept;
 import accord.messages.Commit;
 import accord.messages.PreAccept;
 import accord.primitives.Ballot;
-import accord.primitives.Deps;
+import accord.primitives.PartialDeps;
+import accord.primitives.PartialRoute;
+import accord.primitives.PartialTxn;
+import accord.primitives.Route;
 import accord.primitives.Timestamp;
+import accord.primitives.Txn;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.Schema;
@@ -54,6 +58,7 @@ import static org.apache.cassandra.service.accord.AccordTestUtils.*;
 public class AccordCommandTest
 {
 
+    static final AtomicLong clock = new AtomicLong(0);
     private static final Node.Id ID1 = new Node.Id(1);
     private static final Node.Id ID2 = new Node.Id(2);
     private static final Node.Id ID3 = new Node.Id(3);
@@ -79,108 +84,111 @@ public class AccordCommandTest
     @Test
     public void basicCycleTest() throws ExecutionException, InterruptedException
     {
-        AtomicLong clock = new AtomicLong(0);
         AccordCommandStore commandStore = createAccordCommandStore(clock::incrementAndGet, "ks", "tbl");
-        commandStore.processSetup(instance -> { ((AccordCommandStore) instance).setCacheSize(0); });
+        commandStore.execute(PreLoadContext.empty(), instance -> { ((AccordCommandStore) instance).setCacheSize(0); }).get();
 
 
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
         Txn txn = createTxn(1);
         Key key = txn.keys().get(0);
-        PreAccept preAccept = new PreAccept(txn.keys(), 1, txnId, txn, key);
+        RoutingKey homeKey = key.toRoutingKey();
+        Route fullRoute = txn.keys().toRoute(homeKey);
+        PartialRoute route = fullRoute.slice(fullRange(txn));
+        PartialTxn partialTxn = txn.slice(route.covering, true);
+        PreAccept preAccept = PreAccept.SerializerSupport.create(txnId, route, 1, 1, false, 1, partialTxn, fullRoute);
 
         // Check preaccept
-        commandStore.process(preAccept, instance -> {
-            PreAccept.PreAcceptReply reply = preAccept.process(instance, key);
-            Assert.assertTrue(reply.isOK());
+        commandStore.execute(preAccept, instance -> {
+            PreAccept.PreAcceptReply reply = preAccept.apply(instance);
+            Assert.assertTrue(reply.isOk());
             PreAccept.PreAcceptOk ok = (PreAccept.PreAcceptOk) reply;
             Assert.assertEquals(txnId, ok.witnessedAt);
             Assert.assertTrue(ok.deps.isEmpty());
         }).get();
 
-        commandStore.process(preAccept, instance -> {
+        commandStore.execute(preAccept, instance -> {
             Command command = instance.command(txnId);
             Assert.assertEquals(txnId, command.executeAt());
             Assert.assertEquals(Status.PreAccepted, command.status());
-            Assert.assertTrue(command.savedDeps().isEmpty());
+            Assert.assertTrue(command.partialDeps().isEmpty());
 
             CommandsForKey cfk = instance.commandsForKey(key(1));
             Assert.assertEquals(txnId, cfk.max());
-            Assert.assertNotNull(cfk.uncommitted().get(txnId));
-            Assert.assertNull(cfk.committedById().get(txnId));
-            Assert.assertNull(cfk.committedByExecuteAt().get(txnId));
+            Assert.assertNotNull(((AccordCommandsForKey.Series<?>)cfk.uncommitted()).get(txnId));
+            Assert.assertNull(((AccordCommandsForKey.Series<?>)cfk.committedById()).get(txnId));
+            Assert.assertNull(((AccordCommandsForKey.Series<?>)cfk.committedByExecuteAt()).get(txnId));
         }).get();
 
         // check accept
         TxnId txnId2 = txnId(1, clock.incrementAndGet(), 0, 1);
         Timestamp executeAt = timestamp(1, clock.incrementAndGet(), 0, 1);
-        Deps.OrderedBuilder builder = Deps.orderedBuilder(false);
+        PartialDeps.OrderedBuilder builder = PartialDeps.orderedBuilder(route.covering, false);
         builder.add(key, txnId2);
-        Deps deps = builder.build();
-        Accept accept = Accept.SerializerSupport.create(txn.keys(), 1, txnId, Ballot.ZERO, key, txn, executeAt, deps);
+        PartialDeps deps = builder.build();
+        Accept accept = Accept.SerializerSupport.create(txnId, route, 1, 1, false, Ballot.ZERO, executeAt, partialTxn.keys(), deps, partialTxn.kind());
 
-        commandStore.process(accept, instance -> {
-            Accept.AcceptReply reply = accept.process(instance, key);
-            Assert.assertTrue(reply.isOK());
+        commandStore.execute(accept, instance -> {
+            Accept.AcceptReply reply = accept.apply(instance);
+            Assert.assertTrue(reply.isOk());
             Accept.AcceptOk ok = (Accept.AcceptOk) reply;
             Assert.assertTrue(ok.deps.isEmpty());
         }).get();
 
-        commandStore.process(accept, instance -> {
+        commandStore.execute(accept, instance -> {
             Command command = instance.command(txnId);
             Assert.assertEquals(executeAt, command.executeAt());
             Assert.assertEquals(Status.Accepted, command.status());
-            Assert.assertEquals(deps, command.savedDeps());
+            Assert.assertEquals(deps, command.partialDeps());
 
             CommandsForKey cfk = instance.commandsForKey(key(1));
             Assert.assertEquals(executeAt, cfk.max());
-            Assert.assertNotNull(cfk.uncommitted().get(txnId));
-            Assert.assertNull(cfk.committedById().get(txnId));
-            Assert.assertNull(cfk.committedByExecuteAt().get(txnId));
+            Assert.assertNotNull(((AccordCommandsForKey.Series<?>)cfk.uncommitted()).get(txnId));
+            Assert.assertNull(((AccordCommandsForKey.Series<?>)cfk.committedById()).get(txnId));
+            Assert.assertNull(((AccordCommandsForKey.Series<?>)cfk.committedByExecuteAt()).get(txnId));
         }).get();
 
         // check commit
-        Commit commit = new Commit(txn.keys(), 1, txnId, txn, deps, key, executeAt, false);
-        commandStore.process(commit, instance -> {
-            Command command = instance.command(txnId);
-            command.commit(commit.txn, key, key, commit.executeAt, commit.deps);
-        }).get();
+        Commit commit = Commit.SerializerSupport.create(txnId, route, 1, executeAt, partialTxn, deps, fullRoute, null);
+        commandStore.execute(commit, commit::apply).get();
 
-        commandStore.process(commit, instance -> {
+        commandStore.execute(PreLoadContext.contextFor(txnId, Collections.singleton(key)),instance -> {
             Command command = instance.command(txnId);
             Assert.assertEquals(commit.executeAt, command.executeAt());
             Assert.assertTrue(command.hasBeen(Status.Committed));
-            Assert.assertEquals(commit.deps, command.savedDeps());
+            Assert.assertEquals(commit.partialDeps, command.partialDeps());
 
             CommandsForKey cfk = instance.commandsForKey(key(1));
-            Assert.assertNull(cfk.uncommitted().get(txnId));
-            Assert.assertNotNull(cfk.committedById().get(txnId));
-            Assert.assertNotNull(cfk.committedByExecuteAt().get(commit.executeAt));
+            Assert.assertNull(((AccordCommandsForKey.Series<?>)cfk.uncommitted()).get(txnId));
+            Assert.assertNotNull(((AccordCommandsForKey.Series<?>)cfk.committedById()).get(txnId));
+            Assert.assertNotNull(((AccordCommandsForKey.Series<?>)cfk.committedByExecuteAt()).get(commit.executeAt));
         }).get();
     }
 
     @Test
     public void computeDeps() throws Throwable
     {
-        AtomicLong clock = new AtomicLong(0);
         AccordCommandStore commandStore = createAccordCommandStore(clock::incrementAndGet, "ks", "tbl");
-        commandStore.processSetup(instance -> { ((AccordCommandStore) instance).setCacheSize(0); });
+        commandStore.execute(PreLoadContext.empty(), instance -> { ((AccordCommandStore) instance).setCacheSize(0); }).get();
 
         TxnId txnId1 = txnId(1, clock.incrementAndGet(), 0, 1);
         Txn txn = createTxn(2);
         Key key = txn.keys().get(0);
-        PreAccept preAccept1 = new PreAccept(txn.keys(), 1, txnId1, txn, key);
+        RoutingKey homeKey = key.toRoutingKey();
+        Route fullRoute = txn.keys().toRoute(homeKey);
+        PartialRoute route = fullRoute.slice(fullRange(txn));
+        PartialTxn partialTxn = txn.slice(route.covering, true);
+        PreAccept preAccept1 = PreAccept.SerializerSupport.create(txnId1, route, 1, 1, false, 1, partialTxn, fullRoute);
 
-        commandStore.process(preAccept1, (Consumer<CommandStore>) cs -> preAccept1.process(cs, key)).get();
+        commandStore.execute(preAccept1, preAccept1::apply).get();
 
         // second preaccept should identify txnId1 as a dependency
         TxnId txnId2 = txnId(1, clock.incrementAndGet(), 0, 1);
-        PreAccept preAccept2 = new PreAccept(txn.keys(), 1, txnId2, txn, key);
-        commandStore.process(preAccept2, instance -> {
-            PreAccept.PreAcceptReply reply = preAccept2.process(instance, key);
-            Assert.assertTrue(reply.isOK());
+        PreAccept preAccept2 = PreAccept.SerializerSupport.create(txnId2, route, 1, 1, false, 1, partialTxn, fullRoute);
+        commandStore.execute(preAccept2, instance -> {
+            PreAccept.PreAcceptReply reply = preAccept2.apply(instance);
+            Assert.assertTrue(reply.isOk());
             PreAccept.PreAcceptOk ok = (PreAccept.PreAcceptOk) reply;
             Assert.assertTrue(ok.deps.contains(txnId1));
-        });
+        }).get();
     }
 }
diff --git a/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java b/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java
index cd1b84cfd7..2219a89b09 100644
--- a/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java
+++ b/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java
@@ -35,24 +35,28 @@ import com.google.common.collect.Sets;
 
 import accord.api.Data;
 import accord.api.ProgressLog;
+import accord.api.RoutingKey;
 import accord.api.Write;
 import accord.impl.InMemoryCommandStore;
 import accord.local.Command;
 import accord.local.CommandStore;
 import accord.local.Node;
 import accord.local.Node.Id;
-import accord.local.PartialCommand;
+import accord.local.NodeTimeService;
 import accord.local.PreLoadContext;
+import accord.local.Status.Known;
+import accord.primitives.AbstractKeys;
 import accord.primitives.Ballot;
 import accord.primitives.KeyRange;
 import accord.primitives.KeyRanges;
-import accord.primitives.Keys;
+import accord.primitives.PartialTxn;
+import accord.primitives.RoutingKeys;
 import accord.primitives.Timestamp;
+import accord.primitives.Txn;
 import accord.primitives.TxnId;
+import accord.primitives.Writes;
 import accord.topology.Shard;
 import accord.topology.Topology;
-import accord.txn.Txn;
-import accord.txn.Writes;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
@@ -75,14 +79,17 @@ public class AccordTestUtils
 
     public static final ProgressLog NOOP_PROGRESS_LOG = new ProgressLog()
     {
-        @Override public void preaccept(Command command, boolean isProgressShard, boolean isHomeShard) {}
-        @Override public void accept(Command command, boolean isProgressShard, boolean isHomeShard) {}
-        @Override public void commit(Command command, boolean isProgressShard, boolean isHomeShard) {}
-        @Override public void readyToExecute(Command command, boolean isProgressShard, boolean isHomeShard) {}
-        @Override public void execute(Command command, boolean isProgressShard, boolean isHomeShard) {}
-        @Override public void invalidate(Command command, boolean isProgressShard, boolean isHomeShard) {}
-        @Override public void executedOnAllShards(Command command, Set<Id> persistedOn) {}
-        @Override public void waiting(PartialCommand blockedBy, @Nullable Keys someKeys) {}
+        @Override public void unwitnessed(TxnId txnId, RoutingKey homeKey, ProgressShard shard) {}
+        @Override public void preaccepted(Command command, ProgressShard progressShard) {}
+        @Override public void accepted(Command command, ProgressShard progressShard) {}
+        @Override public void committed(Command command, ProgressShard progressShard) {}
+        @Override public void readyToExecute(Command command, ProgressShard progressShard) {}
+        @Override public void executed(Command command, ProgressShard progressShard) {}
+        @Override public void invalidated(Command command, ProgressShard progressShard) {}
+        @Override public void durable(Command command, Set<Id> persistedOn) {}
+        @Override public void durable(TxnId txnId, @Nullable RoutingKeys someKeys, ProgressShard shard) {}
+        @Override public void durableLocal(TxnId txnId) {}
+        @Override public void waiting(TxnId blockedBy, Known blockedUntil, RoutingKeys blockedOnKeys) {}
     };
 
     public static Topology simpleTopology(TableId... tables)
@@ -125,18 +132,18 @@ public class AccordTestUtils
     /**
      * does the reads, writes, and results for a command without the consensus
      */
-    public static void processCommandResult(Command command) throws Throwable
+    public static void processCommandResult(AccordCommandStore commandStore, Command command) throws Throwable
     {
 
-        command.commandStore().process(PreLoadContext.contextFor(Collections.emptyList(), command.txn().keys()),
+        commandStore.execute(PreLoadContext.contextFor(Collections.emptyList(), command.partialTxn().keys()),
                                        instance -> {
-            Txn txn = command.txn();
+            PartialTxn txn = command.partialTxn();
             AccordRead read = (AccordRead) txn.read();
             Data readData = read.keys().stream()
                                 .map(key -> {
                                     try
                                     {
-                                        return read.read(key, command.txn().isWrite(), command.commandStore(), command.executeAt(), null).get();
+                                        return read.read(key, command.kind(), commandStore, command.executeAt(), null).get();
                                     }
                                     catch (InterruptedException e)
                                     {
@@ -149,8 +156,8 @@ public class AccordTestUtils
                                 })
                                 .reduce(null, AccordData::merge);
             Write write = txn.update().apply(readData);
-            command.writes(new Writes(command.executeAt(), txn.keys(), write));
-            command.result(txn.query().compute(readData, txn.read(), txn.update()));
+            ((AccordCommand)command).setWrites(new Writes(command.executeAt(), txn.keys(), write));
+            ((AccordCommand)command).setResult(txn.query().compute(command.txnId(), readData, txn.read(), txn.update()));
         }).get();
     }
 
@@ -168,6 +175,19 @@ public class AccordTestUtils
         return createTxn(key, key);
     }
 
+    public static KeyRanges fullRange(Txn txn)
+    {
+        TableId tableId = ((AccordKey)txn.keys().get(0)).tableId();
+        return KeyRanges.of(TokenRange.fullRange(tableId));
+    }
+
+    public static PartialTxn createPartialTxn(int key)
+    {
+        Txn txn = createTxn(key, key);
+        KeyRanges ranges = fullRange(txn);
+        return new PartialTxn.InMemory(ranges, txn.kind(), txn.keys(), txn.read(), txn.query(), txn.update());
+    }
+
     private static class SingleEpochRanges implements CommandStore.RangesForEpoch
     {
         private final KeyRanges ranges;
@@ -184,6 +204,12 @@ public class AccordTestUtils
             return ranges;
         }
 
+        @Override
+        public KeyRanges between(long fromInclusive, long toInclusive)
+        {
+            return ranges;
+        }
+
         @Override
         public KeyRanges since(long epoch)
         {
@@ -192,7 +218,13 @@ public class AccordTestUtils
         }
 
         @Override
-        public boolean intersects(long epoch, Keys keys)
+        public boolean owns(long epoch, RoutingKey key)
+        {
+            return ranges.contains(key);
+        }
+
+        @Override
+        public boolean intersects(long epoch, AbstractKeys<?, ?> keys)
         {
             assert epoch == 1;
             return ranges.intersects(keys);
@@ -205,9 +237,14 @@ public class AccordTestUtils
         TokenRange range = TokenRange.fullRange(metadata.id);
         Node.Id node = EndpointMapping.endpointToId(FBUtilities.getBroadcastAddressAndPort());
         Topology topology = new Topology(1, new Shard(range, Lists.newArrayList(node), Sets.newHashSet(node), Collections.emptySet()));
-        return new InMemoryCommandStore.Synchronized(0, 1, 8,
-                                                     ts -> new Timestamp(1, now.getAsLong(), 0, node),
-                                                     () -> 1,
+        NodeTimeService time = new NodeTimeService()
+        {
+            @Override public Id id() { return node;}
+            @Override public long epoch() {return 1; }
+            @Override public Timestamp uniqueNow(Timestamp atLeast) { return new Timestamp(1, now.getAsLong(), 0, node); }
+        };
+        return new InMemoryCommandStore.Synchronized(0, 0, 1, 8,
+                                                     time,
                                                      new AccordAgent(),
                                                      null,
                                                      cs -> null,
@@ -221,9 +258,14 @@ public class AccordTestUtils
             thread.setName(CommandStore.class.getSimpleName() + '[' + node + ':' + 0 + ']');
             return thread;
         });
-        return new AccordCommandStore(0, 0, 1,
-                                      ts -> new Timestamp(1, now.getAsLong(), 0, node),
-                                      () -> 1,
+        NodeTimeService time = new NodeTimeService()
+        {
+            @Override public Id id() { return node;}
+            @Override public long epoch() {return 1; }
+            @Override public Timestamp uniqueNow(Timestamp atLeast) { return new Timestamp(1, now.getAsLong(), 0, node); }
+        };
+        return new AccordCommandStore(0, 0, 0, 1,
+                                      time,
                                       new AccordAgent(),
                                       null,
                                       cs -> NOOP_PROGRESS_LOG,
diff --git a/test/unit/org/apache/cassandra/service/accord/AccordTopologyTest.java b/test/unit/org/apache/cassandra/service/accord/AccordTopologyTest.java
index f737372d46..4dd94847a9 100644
--- a/test/unit/org/apache/cassandra/service/accord/AccordTopologyTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/AccordTopologyTest.java
@@ -56,7 +56,7 @@ public class AccordTopologyTest
     {
         IPartitioner partitioner = DatabaseDescriptor.getPartitioner();
         Topology topology = AccordTopologyUtils.createTopology(1);
-        Assert.assertFalse(topology.isEmpty());
+        Assert.assertNotEquals(0, topology.size());
         TableId tableId = Schema.instance.getTableMetadata("ks", "tbl").id;
         Token minToken = partitioner.getMinimumToken();
         Token maxToken = partitioner.getMaximumToken();
diff --git a/test/unit/org/apache/cassandra/service/accord/api/AccordKeyTest.java b/test/unit/org/apache/cassandra/service/accord/api/AccordKeyTest.java
index eda11b761a..81f9b49a22 100644
--- a/test/unit/org/apache/cassandra/service/accord/api/AccordKeyTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/api/AccordKeyTest.java
@@ -29,6 +29,8 @@ import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.accord.api.AccordKey.*;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey.SentinelKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey.TokenKey;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.SerializerTestUtils;
 
@@ -66,7 +68,7 @@ public class AccordKeyTest
     public void tokenKeyTest()
     {
         DecoratedKey dk = partitioner(TABLE1).decorateKey(ByteBufferUtil.bytes(5));
-        TokenKey pk = new TokenKey(TABLE1, dk.getToken().maxKeyBound());
+        TokenKey pk = new TokenKey(TABLE1, dk.getToken());
         SerializerTestUtils.assertSerializerIOEquality(pk, TokenKey.serializer);
     }
 
@@ -75,9 +77,11 @@ public class AccordKeyTest
     {
         DecoratedKey dk = partitioner(TABLE1).decorateKey(ByteBufferUtil.bytes(5));
         PartitionKey pk = new PartitionKey(TABLE1, dk);
-        TokenKey tkLow = new TokenKey(TABLE1, dk.getToken().minKeyBound());
-        TokenKey tkHigh = new TokenKey(TABLE1, dk.getToken().maxKeyBound());
+        TokenKey tk = new TokenKey(TABLE1, dk.getToken());
+        TokenKey tkLow = new TokenKey(TABLE1, dk.getToken().decreaseSlightly());
+        TokenKey tkHigh = new TokenKey(TABLE1, dk.getToken().increaseSlightly());
 
+        Assert.assertTrue(tk.compareTo(pk) == 0);
         Assert.assertTrue(tkLow.compareTo(pk) < 0);
         Assert.assertTrue(pk.compareTo(tkHigh) < 0);
     }
diff --git a/test/unit/org/apache/cassandra/service/accord/async/AsyncLoaderTest.java b/test/unit/org/apache/cassandra/service/accord/async/AsyncLoaderTest.java
index 1351117999..83f3d8058e 100644
--- a/test/unit/org/apache/cassandra/service/accord/async/AsyncLoaderTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/async/AsyncLoaderTest.java
@@ -31,8 +31,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import accord.local.Status;
+import accord.primitives.PartialTxn;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.StorageService;
@@ -50,7 +50,7 @@ import static com.google.common.collect.Iterables.getOnlyElement;
 import static java.util.Collections.singleton;
 import static org.apache.cassandra.cql3.statements.schema.CreateTableStatement.parse;
 import static org.apache.cassandra.service.accord.AccordTestUtils.createAccordCommandStore;
-import static org.apache.cassandra.service.accord.AccordTestUtils.createTxn;
+import static org.apache.cassandra.service.accord.AccordTestUtils.createPartialTxn;
 import static org.apache.cassandra.service.accord.AccordTestUtils.execute;
 import static org.apache.cassandra.service.accord.AccordTestUtils.txnId;
 
@@ -76,12 +76,12 @@ public class AsyncLoaderTest
         AccordStateCache.Instance<TxnId, AccordCommand> commandCache = commandStore.commandCache();
         AccordStateCache.Instance<PartitionKey, AccordCommandsForKey> cfkCacche = commandStore.commandsForKeyCache();
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
-        Txn txn = createTxn(0);
+        PartialTxn txn = createPartialTxn(0);
         PartitionKey key = (PartitionKey) Iterables.getOnlyElement(txn.keys());
 
         // acquire / release
         AccordCommand command = commandCache.getOrCreate(txnId).initialize();
-        command.txn(txn);
+        command.setPartialTxn(txn);
         commandCache.release(command);
         AccordCommandsForKey cfk = cfkCacche.getOrCreate(key).initialize();
         cfkCacche.release(cfk);
@@ -110,15 +110,15 @@ public class AsyncLoaderTest
         AccordStateCache.Instance<TxnId, AccordCommand> commandCache = commandStore.commandCache();
         AccordStateCache.Instance<PartitionKey, AccordCommandsForKey> cfkCacche = commandStore.commandsForKeyCache();
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
-        Txn txn = createTxn(0);
+        PartialTxn txn = createPartialTxn(0);
         PartitionKey key = (PartitionKey) Iterables.getOnlyElement(txn.keys());
 
         // create / persist
-        AccordCommand command = new AccordCommand(commandStore, txnId).initialize();
-        command.txn(txn);
-        AccordKeyspace.getCommandMutation(command, commandStore.nextSystemTimestampMicros()).apply();
+        AccordCommand command = new AccordCommand(txnId).initialize();
+        command.setPartialTxn(txn);
+        AccordKeyspace.getCommandMutation(commandStore, command, commandStore.nextSystemTimestampMicros()).apply();
         AccordCommandsForKey cfk = new AccordCommandsForKey(commandStore, key).initialize();
-        AccordKeyspace.getCommandsForKeyMutation(cfk, commandStore.nextSystemTimestampMicros()).apply();
+        AccordKeyspace.getCommandsForKeyMutation(commandStore, cfk, commandStore.nextSystemTimestampMicros()).apply();
 
         // resources are on disk only, so the loader should suspend...
         AsyncContext context = new AsyncContext();
@@ -152,15 +152,15 @@ public class AsyncLoaderTest
         AccordStateCache.Instance<TxnId, AccordCommand> commandCache = commandStore.commandCache();
         AccordStateCache.Instance<PartitionKey, AccordCommandsForKey> cfkCacche = commandStore.commandsForKeyCache();
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
-        Txn txn = createTxn(0);
+        PartialTxn txn = createPartialTxn(0);
         PartitionKey key = (PartitionKey) Iterables.getOnlyElement(txn.keys());
 
         // acquire /release, create / persist
         AccordCommand command = commandCache.getOrCreate(txnId).initialize();
-        command.txn(txn);
+        command.setPartialTxn(txn);
         commandCache.release(command);
         AccordCommandsForKey cfk = new AccordCommandsForKey(commandStore, key).initialize();
-        AccordKeyspace.getCommandsForKeyMutation(cfk, commandStore.nextSystemTimestampMicros()).apply();
+        AccordKeyspace.getCommandsForKeyMutation(commandStore, cfk, commandStore.nextSystemTimestampMicros()).apply();
 
         // resources are on disk only, so the loader should suspend...
         AsyncContext context = new AsyncContext();
@@ -194,12 +194,12 @@ public class AsyncLoaderTest
         AccordStateCache.Instance<TxnId, AccordCommand> commandCache = commandStore.commandCache();
         AccordStateCache.Instance<PartitionKey, AccordCommandsForKey> cfkCacche = commandStore.commandsForKeyCache();
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
-        Txn txn = createTxn(0);
+        PartialTxn txn = createPartialTxn(0);
         PartitionKey key = (PartitionKey) Iterables.getOnlyElement(txn.keys());
 
         // acquire / release
         AccordCommand command = commandCache.getOrCreate(txnId).initialize();
-        command.txn(txn);
+        command.setPartialTxn(txn);
         commandCache.release(command);
         AccordCommandsForKey cfk = cfkCacche.getOrCreate(key).initialize();
         cfkCacche.release(cfk);
@@ -241,24 +241,24 @@ public class AsyncLoaderTest
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
         TxnId blockApply = txnId(1, clock.incrementAndGet(), 0, 1);
         TxnId blockCommit = txnId(1, clock.incrementAndGet(), 0, 1);
-        Txn txn = createTxn(0);
+        PartialTxn txn = createPartialTxn(0);
         AccordKey.PartitionKey key = (AccordKey.PartitionKey) getOnlyElement(txn.keys());
 
-        AccordCommand command = new AccordCommand(commandStore, txnId).initialize();
-        command.txn(txn);
-        command.executeAt(txnId);
-        command.status(Status.Committed);
-        AccordKeyspace.getCommandMutation(command, commandStore.nextSystemTimestampMicros()).apply();
+        AccordCommand command = new AccordCommand(txnId).initialize();
+        command.setPartialTxn(txn);
+        command.setExecuteAt(txnId);
+        command.setStatus(Status.Committed);
+        AccordKeyspace.getCommandMutation(commandStore, command, commandStore.nextSystemTimestampMicros()).apply();
         command.clearModifiedFlag();
 
         execute(commandStore, () -> {
             AccordStateCache.Instance<TxnId, AccordCommand> cache = commandStore.commandCache();
-            AccordCommand.WriteOnly writeOnly1 = new AccordCommand.WriteOnly(commandStore, txnId);
+            AccordCommand.WriteOnly writeOnly1 = new AccordCommand.WriteOnly(txnId);
             writeOnly1.blockingApplyOn.blindAdd(blockApply);
             writeOnly1.future(new AsyncPromise<>());
             cache.addWriteOnly(writeOnly1);
 
-            AccordCommand.WriteOnly writeOnly2 = new AccordCommand.WriteOnly(commandStore, txnId);
+            AccordCommand.WriteOnly writeOnly2 = new AccordCommand.WriteOnly(txnId);
             writeOnly2.blockingCommitOn.blindAdd(blockCommit);
             writeOnly2.future(new AsyncPromise<>());
             cache.addWriteOnly(writeOnly2);
diff --git a/test/unit/org/apache/cassandra/service/accord/async/AsyncOperationTest.java b/test/unit/org/apache/cassandra/service/accord/async/AsyncOperationTest.java
index 565b25579d..ebf272aac9 100644
--- a/test/unit/org/apache/cassandra/service/accord/async/AsyncOperationTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/async/AsyncOperationTest.java
@@ -32,12 +32,11 @@ import org.junit.Test;
 import accord.local.Command;
 import accord.local.CommandStore;
 import accord.local.CommandsForKey;
-import accord.local.PartialCommand;
-import accord.local.PreLoadContext;
+import accord.local.SafeCommandStore;
 import accord.local.Status;
 import accord.primitives.Timestamp;
+import accord.primitives.Txn;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -59,11 +58,14 @@ import static java.util.Collections.emptyList;
 import static java.util.Collections.singleton;
 import static org.apache.cassandra.cql3.statements.schema.CreateTableStatement.parse;
 import static org.apache.cassandra.service.accord.AccordTestUtils.createAccordCommandStore;
+import static org.apache.cassandra.service.accord.AccordTestUtils.createPartialTxn;
 import static org.apache.cassandra.service.accord.AccordTestUtils.createTxn;
 import static org.apache.cassandra.service.accord.AccordTestUtils.txnId;
 
 public class AsyncOperationTest
 {
+    private static AtomicLong clock = new AtomicLong(0);
+
     @BeforeClass
     public static void beforeClass() throws Throwable
     {
@@ -87,13 +89,12 @@ public class AsyncOperationTest
     @Test
     public void optionalCommandTest() throws Throwable
     {
-        AtomicLong clock = new AtomicLong(0);
         AccordCommandStore commandStore = createAccordCommandStore(clock::incrementAndGet, "ks", "tbl");
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
-        Txn txn = createTxn(0);
+        Txn txn = createTxn((int)clock.incrementAndGet());
         AccordKey.PartitionKey key = (AccordKey.PartitionKey) Iterables.getOnlyElement(txn.keys());
 
-        commandStore.process(contextFor(txnId), instance -> {
+        commandStore.execute(contextFor(txnId), instance -> {
             Command command = instance.ifPresent(txnId);
             Assert.assertNull(command);
         }).get();
@@ -105,12 +106,11 @@ public class AsyncOperationTest
     @Test
     public void optionalCommandsForKeyTest() throws Throwable
     {
-        AtomicLong clock = new AtomicLong(0);
         AccordCommandStore commandStore = createAccordCommandStore(clock::incrementAndGet, "ks", "tbl");
-        Txn txn = createTxn(0);
+        Txn txn = createTxn((int)clock.incrementAndGet());
         AccordKey.PartitionKey key = (AccordKey.PartitionKey) Iterables.getOnlyElement(txn.keys());
 
-        commandStore.process(contextFor(Collections.emptyList(), Collections.singleton(key)), instance -> {
+        commandStore.execute(contextFor(Collections.emptyList(), Collections.singleton(key)), instance -> {
             CommandsForKey cfk = commandStore.maybeCommandsForKey(key);
             Assert.assertNull(cfk);
         }).get();
@@ -124,19 +124,13 @@ public class AsyncOperationTest
         }
     }
 
-
-    private static PartialCommand createPartialCommand(Command command)
-    {
-        return new AccordPartialCommand(command.txnId(), command.txn(), command.executeAt(), command.status());
-    }
-
     private static AccordCommand createCommittedAndPersist(AccordCommandStore commandStore, TxnId txnId, Timestamp executeAt)
     {
-        AccordCommand command = new AccordCommand(commandStore, txnId).initialize();
-        command.txn(createTxn(0));
-        command.executeAt(executeAt);
-        command.status(Status.Committed);
-        AccordKeyspace.getCommandMutation(command, commandStore.nextSystemTimestampMicros()).apply();
+        AccordCommand command = new AccordCommand(txnId).initialize();
+        command.setPartialTxn(createPartialTxn(0));
+        command.setExecuteAt(executeAt);
+        command.setStatus(Status.Committed);
+        AccordKeyspace.getCommandMutation(commandStore, command, commandStore.nextSystemTimestampMicros()).apply();
         command.clearModifiedFlag();
         return command;
     }
@@ -163,14 +157,13 @@ public class AsyncOperationTest
     @Test
     public void testFutureCleanup()
     {
-        AtomicLong clock = new AtomicLong(0);
         AccordCommandStore commandStore = createAccordCommandStore(clock::incrementAndGet, "ks", "tbl");
 
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
 
         AccordCommand command = createCommittedAndPersist(commandStore, txnId);
 
-        Consumer<CommandStore> consumer = instance -> instance.command(txnId).status(Status.Executed);
+        Consumer<SafeCommandStore> consumer = instance -> ((AccordCommand)instance.command(txnId)).setStatus(Status.PreApplied);
         AsyncOperation<Void> operation = new AsyncOperation.ForConsumer(commandStore, singleton(txnId), emptyList(), consumer)
         {
 
diff --git a/test/unit/org/apache/cassandra/service/accord/async/AsyncWriterTest.java b/test/unit/org/apache/cassandra/service/accord/async/AsyncWriterTest.java
index 4d788b6865..28c2508491 100644
--- a/test/unit/org/apache/cassandra/service/accord/async/AsyncWriterTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/async/AsyncWriterTest.java
@@ -26,11 +26,11 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import accord.local.Command;
-import accord.local.PartialCommand;
 import accord.local.Status;
+import accord.primitives.KeyRanges;
 import accord.primitives.Timestamp;
+import accord.primitives.Txn;
 import accord.primitives.TxnId;
-import accord.txn.Txn;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.StorageService;
@@ -47,6 +47,7 @@ import static org.apache.cassandra.cql3.statements.schema.CreateTableStatement.p
 import static org.apache.cassandra.service.accord.AccordTestUtils.createAccordCommandStore;
 import static org.apache.cassandra.service.accord.AccordTestUtils.createTxn;
 import static org.apache.cassandra.service.accord.AccordTestUtils.execute;
+import static org.apache.cassandra.service.accord.AccordTestUtils.fullRange;
 import static org.apache.cassandra.service.accord.AccordTestUtils.timestamp;
 import static org.apache.cassandra.service.accord.AccordTestUtils.txnId;
 
@@ -84,24 +85,23 @@ public class AsyncWriterTest
         TxnId blockingId = txnId(1, clock.incrementAndGet(), 0, 1);
         TxnId waitingId = txnId(1, clock.incrementAndGet(), 0, 1);
         Txn txn = createTxn(0);
-        AccordKey.PartitionKey key = (AccordKey.PartitionKey) getOnlyElement(txn.keys());
-
-        AccordCommand blocking = new AccordCommand(commandStore, blockingId).initialize();
-        blocking.txn(txn);
-        blocking.executeAt(blockingId);
-        blocking.status(Status.Committed);
-        AccordKeyspace.getCommandMutation(blocking, commandStore.nextSystemTimestampMicros()).apply();
+        KeyRanges ranges = fullRange(txn);
+        AccordCommand blocking = new AccordCommand(blockingId).initialize();
+        blocking.setPartialTxn(txn.slice(ranges, true));
+        blocking.setExecuteAt(blockingId);
+        blocking.setStatus(Status.Committed);
+        AccordKeyspace.getCommandMutation(commandStore, blocking, commandStore.nextSystemTimestampMicros()).apply();
         blocking.clearModifiedFlag();
 
-        AccordCommand waiting = new AccordCommand(commandStore, waitingId).initialize();
-        waiting.txn(txn);
-        waiting.executeAt(waitingId);
-        waiting.status(Status.Committed);
-        AccordKeyspace.getCommandMutation(waiting, commandStore.nextSystemTimestampMicros()).apply();
+        AccordCommand waiting = new AccordCommand(waitingId).initialize();
+        waiting.setPartialTxn(txn.slice(ranges, true));
+        waiting.setExecuteAt(waitingId);
+        waiting.setStatus(Status.Committed);
+        AccordKeyspace.getCommandMutation(commandStore, waiting, commandStore.nextSystemTimestampMicros()).apply();
         waiting.clearModifiedFlag();
 
         AsyncContext context = new AsyncContext();
-        waiting.addWaitingOnApplyIfAbsent(blocking);
+        waiting.addWaitingOnApplyIfAbsent(blocking.txnId(), blocking.executeAt());
         context.commands.add(waiting);
         save(commandStore, context);
 
@@ -111,20 +111,17 @@ public class AsyncWriterTest
 
         // now change the blocking command and check it's changes are reflected in the waiting command
         context = new AsyncContext();
-        blocking.status(Status.ReadyToExecute);
+        blocking.setStatus(Status.ReadyToExecute);
         context.commands.add(blocking);
         save(commandStore, context);
 
         waiting = AccordKeyspace.loadCommand(commandStore, waitingId);
         AccordCommand waitingFinal = waiting;
-        AccordCommand blockingFinal = blocking;
         execute(commandStore, () -> {
             AsyncContext ctx = new AsyncContext();
             commandStore.setContext(ctx);
-            AccordPartialCommand blockingSummary = (AccordPartialCommand) waitingFinal.firstWaitingOnApply();
-            Assert.assertNotSame(blockingFinal, blockingSummary);
-            Assert.assertEquals(Status.ReadyToExecute, blockingSummary.status());
-            Assert.assertEquals(blockingId, blockingSummary.executeAt());
+            TxnId blockingSummary = waitingFinal.firstWaitingOnApply();
+            Assert.assertEquals(blockingId, blockingSummary);
             commandStore.unsetContext(ctx);
         });
     }
@@ -138,18 +135,19 @@ public class AsyncWriterTest
         TxnId txnId = txnId(1, clock.incrementAndGet(), 0, 1);
         Timestamp executeAt = timestamp(1, clock.incrementAndGet(), 0, 1);
         Txn txn = createTxn(0);
+        KeyRanges ranges = fullRange(txn);
         AccordKey.PartitionKey key = (AccordKey.PartitionKey) getOnlyElement(txn.keys());
 
         AccordCommandsForKey cfk = new AccordCommandsForKey(commandStore, key).initialize();
-        AccordKeyspace.getCommandsForKeyMutation(cfk, commandStore.nextSystemTimestampMicros()).apply();
+        AccordKeyspace.getCommandsForKeyMutation(commandStore, cfk, commandStore.nextSystemTimestampMicros()).apply();
         Assert.assertTrue(cfk.uncommitted.isEmpty());
         Assert.assertTrue(cfk.committedByExecuteAt.isEmpty());
         Assert.assertTrue(cfk.committedById.isEmpty());
 
-        AccordCommand command = new AccordCommand(commandStore, txnId).initialize();
-        command.txn(txn);
-        command.executeAt(executeAt);
-        command.status(Status.Accepted);
+        AccordCommand command = new AccordCommand(txnId).initialize();
+        command.setPartialTxn(txn.slice(ranges, true));
+        command.setExecuteAt(executeAt);
+        command.setStatus(Status.Accepted);
         AsyncContext context = new AsyncContext();
         context.commands.add(command);
         save(commandStore, context);
@@ -158,9 +156,8 @@ public class AsyncWriterTest
         execute(commandStore, () -> {
             AsyncContext ctx = new AsyncContext();
             commandStore.setContext(ctx);
-            AccordPartialCommand.WithDeps summary = (AccordPartialCommand.WithDeps) getOnlyElement(cfkUncommitted.uncommitted().all().collect(Collectors.toList()));
+            AccordPartialCommand summary = getOnlyElement(cfkUncommitted.uncommitted().all().collect(Collectors.toList()));
             Assert.assertTrue(cfkUncommitted.uncommitted.map.getView().containsKey(txnId));
-            Assert.assertNotSame(command, summary);
             Assert.assertEquals(Status.Accepted, summary.status());
             Assert.assertEquals(executeAt, summary.executeAt());
 
@@ -170,7 +167,7 @@ public class AsyncWriterTest
         });
 
         // commit, summary should be moved to committed maps
-        command.status(Status.Committed);
+        command.setStatus(Status.Committed);
         context = new AsyncContext();
         context.commands.add(command);
         save(commandStore, context);
@@ -179,13 +176,11 @@ public class AsyncWriterTest
         execute(commandStore, () -> {
             AsyncContext ctx = new AsyncContext();
             commandStore.setContext(ctx);
-            AccordPartialCommand.WithDeps idSummary = (AccordPartialCommand.WithDeps) getOnlyElement(cfkCommitted.committedById().all().collect(Collectors.toList()));
-            AccordPartialCommand.WithDeps executeSummary = (AccordPartialCommand.WithDeps) getOnlyElement(cfkCommitted.committedByExecuteAt().all().collect(Collectors.toList()));
+            AccordPartialCommand idSummary = getOnlyElement(cfkCommitted.committedById().all().collect(Collectors.toList()));
+            AccordPartialCommand executeSummary = getOnlyElement(cfkCommitted.committedByExecuteAt().all().collect(Collectors.toList()));
 
             Assert.assertTrue(cfkCommitted.committedById.map.getView().containsKey(txnId));
             Assert.assertTrue(cfkCommitted.committedByExecuteAt.map.getView().containsKey(executeAt));
-            Assert.assertNotEquals(command, idSummary);
-            // we store serialized values, so they will never be the same object as we deserialize
             Assert.assertEquals(idSummary, executeSummary);
 
             Assert.assertEquals(Status.Committed, idSummary.status());
@@ -205,45 +200,44 @@ public class AsyncWriterTest
         TxnId blockingId = txnId(1, clock.incrementAndGet(), 0, 1);
         TxnId waitingId = txnId(1, clock.incrementAndGet(), 0, 1);
         Txn txn = createTxn(0);
-        AccordKey.PartitionKey key = (AccordKey.PartitionKey) getOnlyElement(txn.keys());
+        KeyRanges ranges = fullRange(txn);
 
         {
-            AccordCommand blocking = new AccordCommand(commandStore, blockingId).initialize();
-            blocking.txn(txn);
-            blocking.executeAt(blockingId);
-            blocking.status(Status.Committed);
+            AccordCommand blocking = new AccordCommand(blockingId).initialize();
+            blocking.setPartialTxn(txn.slice(ranges, true));
+            blocking.setExecuteAt(blockingId);
+            blocking.setStatus(Status.Committed);
 
-            AccordCommand waiting = new AccordCommand(commandStore, waitingId).initialize();
-            waiting.txn(txn);
-            waiting.executeAt(waitingId);
-            waiting.status(Status.Committed);
-            waiting.addWaitingOnApplyIfAbsent(blocking);
+            AccordCommand waiting = new AccordCommand(waitingId).initialize();
+            waiting.setPartialTxn(txn.slice(ranges, true));
+            waiting.setExecuteAt(waitingId);
+            waiting.setStatus(Status.Committed);
+            waiting.addWaitingOnApplyIfAbsent(blocking.txnId(), blocking.executeAt());
 
             blocking.addListener(waiting);
 
-            AccordKeyspace.getCommandMutation(blocking, commandStore.nextSystemTimestampMicros()).apply();
-            AccordKeyspace.getCommandMutation(waiting, commandStore.nextSystemTimestampMicros()).apply();
+            AccordKeyspace.getCommandMutation(commandStore, blocking, commandStore.nextSystemTimestampMicros()).apply();
+            AccordKeyspace.getCommandMutation(commandStore, waiting, commandStore.nextSystemTimestampMicros()).apply();
             blocking.clearModifiedFlag();
             waiting.clearModifiedFlag();
         }
 
         // confirm the blocking operation has the waiting one as a listener
-        commandStore.process(contextFor(blockingId), cs -> {
+        commandStore.execute(contextFor(blockingId), cs -> {
             AccordCommand blocking = (AccordCommand) cs.command(blockingId);
             Assert.assertTrue(blocking.hasListenerFor(waitingId));
         });
 
         // remove listener from PartialCommand
-        commandStore.process(contextFor(waitingId), cs -> {
+        commandStore.execute(contextFor(waitingId), cs -> {
             Command waiting = cs.command(waitingId);
-            PartialCommand blocking = waiting.firstWaitingOnApply();
+            TxnId blocking = ((AccordCommand)waiting).firstWaitingOnApply();
             Assert.assertNotNull(blocking);
-            Assert.assertEquals(blockingId, blocking.txnId());
-            blocking.removeListener(waiting);
+            Assert.assertEquals(blockingId, blocking);
         });
 
         // confirm it was propagated to the full command
-        commandStore.process(contextFor(blockingId), cs -> {
+        commandStore.execute(contextFor(blockingId), cs -> {
             AccordCommand blocking = (AccordCommand) cs.command(blockingId);
             Assert.assertFalse(blocking.hasListenerFor(waitingId));
         });
diff --git a/test/unit/org/apache/cassandra/service/accord/db/AccordUpdateTest.java b/test/unit/org/apache/cassandra/service/accord/db/AccordUpdateTest.java
index 49e12ab0f4..b85a3a6c20 100644
--- a/test/unit/org/apache/cassandra/service/accord/db/AccordUpdateTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/db/AccordUpdateTest.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.service.accord.db;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import accord.txn.Txn;
+import accord.primitives.Txn;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.accord.AccordTxnBuilder;
@@ -53,6 +53,6 @@ public class AccordUpdateTest
 
 
         AccordUpdate update = (AccordUpdate) txn.update();
-        assertSerializerIOEquality(update.getPredicate(0), AccordUpdate.predicateSerializer);
+        assertSerializerIOEquality(update.getPredicate(0).get(0), AccordUpdate.predicateSerializer);
     }
 }
diff --git a/test/unit/org/apache/cassandra/service/accord/serializers/CommandSerializersTest.java b/test/unit/org/apache/cassandra/service/accord/serializers/CommandSerializersTest.java
index f4011a4013..2d034b5734 100644
--- a/test/unit/org/apache/cassandra/service/accord/serializers/CommandSerializersTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/serializers/CommandSerializersTest.java
@@ -21,10 +21,17 @@ package org.apache.cassandra.service.accord.serializers;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import accord.txn.Txn;
+import accord.primitives.KeyRange;
+import accord.primitives.KeyRanges;
+import accord.primitives.PartialTxn;
+import accord.primitives.Txn;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.accord.AccordTxnBuilder;
+import org.apache.cassandra.service.accord.TokenRange;
+import org.apache.cassandra.service.accord.api.AccordKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey.SentinelKey;
 import org.apache.cassandra.utils.SerializerTestUtils;
 
 import static org.apache.cassandra.cql3.statements.schema.CreateTableStatement.parse;
@@ -48,7 +55,9 @@ public class CommandSerializersTest
         txnBuilder.withRead("SELECT * FROM ks.tbl WHERE k=0 AND c=0");
         txnBuilder.withWrite("INSERT INTO ks.tbl (k, c, v) VALUES (0, 0, 1)");
         txnBuilder.withCondition("ks", "tbl", 0, 0, NOT_EXISTS);
-        Txn expected = txnBuilder.build();
-        SerializerTestUtils.assertSerializerIOEquality(expected, CommandSerializers.txn);
+        Txn txn = txnBuilder.build();
+        TableId tableId = ((AccordKey.PartitionKey) txn.keys().get(0)).tableId();
+        PartialTxn expected = txn.slice(KeyRanges.of(TokenRange.fullRange(tableId)), true);
+        SerializerTestUtils.assertSerializerIOEquality(expected, CommandSerializers.partialTxn);
     }
 }
diff --git a/test/unit/org/apache/cassandra/service/accord/store/StoredMapTest.java b/test/unit/org/apache/cassandra/service/accord/store/StoredMapTest.java
index 99aac6dc1d..d7062a5e08 100644
--- a/test/unit/org/apache/cassandra/service/accord/store/StoredMapTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/store/StoredMapTest.java
@@ -55,7 +55,7 @@ public class StoredMapTest
         expectedData.put(1, 2);
         expectedData.put(5, 6);
 
-        StoredNavigableMap<Integer, Integer> map = new StoredNavigableMap<>(AccordState.Kind.FULL);
+        StoredNavigableMap<Integer, Integer> map = new StoredNavigableMap<>(AccordState.ReadWrite.FULL);
 
         // no values loaded, getView should fail
         assertISE(map::getView);
@@ -107,7 +107,7 @@ public class StoredMapTest
     @Test
     public void unloadedAddsAndRemoves()
     {
-        StoredNavigableMap<Integer, Integer> map = new StoredNavigableMap<>(AccordState.Kind.FULL);
+        StoredNavigableMap<Integer, Integer> map = new StoredNavigableMap<>(AccordState.ReadWrite.FULL);
         assertISE(map::getView);
 
         // check additions
@@ -144,7 +144,7 @@ public class StoredMapTest
         NavigableMap<Integer, Integer> expectedAdditions = new TreeMap<>();
         Set<Integer> expectedDeletions = new HashSet<>();
 
-        StoredNavigableMap<Integer, Integer> map = new StoredNavigableMap<>(AccordState.Kind.FULL);
+        StoredNavigableMap<Integer, Integer> map = new StoredNavigableMap<>(AccordState.ReadWrite.FULL);
         map.load(new TreeMap<>());
         Assert.assertEquals(expectedData, map.getView());
 
@@ -180,7 +180,7 @@ public class StoredMapTest
         NavigableMap<Integer, Integer> expectedAdditions = new TreeMap<>();
 
         expectedData.put(1, 2);
-        StoredNavigableMap<Integer, Integer> map = new StoredNavigableMap<>(AccordState.Kind.FULL);
+        StoredNavigableMap<Integer, Integer> map = new StoredNavigableMap<>(AccordState.ReadWrite.FULL);
         map.load(new TreeMap<>(expectedData));
         Assert.assertEquals(expectedData, map.getView());
 
diff --git a/test/unit/org/apache/cassandra/service/accord/store/StoredSetTest.java b/test/unit/org/apache/cassandra/service/accord/store/StoredSetTest.java
index efec6fb80c..8099dc4e97 100644
--- a/test/unit/org/apache/cassandra/service/accord/store/StoredSetTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/store/StoredSetTest.java
@@ -54,7 +54,7 @@ public class StoredSetTest
         expected.add(1);
         expected.add(5);
 
-        StoredSet.Navigable<Integer> map = new StoredSet.Navigable<>(AccordState.Kind.FULL);
+        StoredSet.Navigable<Integer> map = new StoredSet.Navigable<>(AccordState.ReadWrite.FULL);
 
         // no values loaded, getView should fail
         assertISE(map::getView);
@@ -106,7 +106,7 @@ public class StoredSetTest
     @Test
     public void unloadedAddsAndRemoves()
     {
-        StoredSet.Navigable<Integer> map = new StoredSet.Navigable<>(AccordState.Kind.FULL);
+        StoredSet.Navigable<Integer> map = new StoredSet.Navigable<>(AccordState.ReadWrite.FULL);
         assertISE(map::getView);
 
         // check additions
@@ -143,7 +143,7 @@ public class StoredSetTest
         NavigableSet<Integer> expectedAdditions = new TreeSet<>();
         Set<Integer> expectedDeletions = new HashSet<>();
 
-        StoredSet.Navigable<Integer> map = new StoredSet.Navigable<>(AccordState.Kind.FULL);
+        StoredSet.Navigable<Integer> map = new StoredSet.Navigable<>(AccordState.ReadWrite.FULL);
         map.load(new TreeSet<>());
         Assert.assertEquals(expectedData, map.getView());
 
@@ -179,7 +179,7 @@ public class StoredSetTest
         NavigableSet<Integer> expectedAdditions = new TreeSet<>();
 
         expectedData.add(1);
-        StoredSet.Navigable<Integer> map = new StoredSet.Navigable<>(AccordState.Kind.FULL);
+        StoredSet.Navigable<Integer> map = new StoredSet.Navigable<>(AccordState.ReadWrite.FULL);
         map.load(new TreeSet<>(expectedData));
         Assert.assertEquals(expectedData, map.getView());
 
diff --git a/test/unit/org/apache/cassandra/service/accord/store/StoredValueTest.java b/test/unit/org/apache/cassandra/service/accord/store/StoredValueTest.java
index 92c847968d..d0250a9827 100644
--- a/test/unit/org/apache/cassandra/service/accord/store/StoredValueTest.java
+++ b/test/unit/org/apache/cassandra/service/accord/store/StoredValueTest.java
@@ -41,7 +41,7 @@ public class StoredValueTest
     @Test
     public void storedValueTest()
     {
-        StoredValue<Integer> value = new StoredValue<>(AccordState.Kind.FULL);
+        StoredValue<Integer> value = new StoredValue<>(AccordState.ReadWrite.FULL);
         // value is unloaded, read should fail
         assertISE(value::get);
 
@@ -68,7 +68,7 @@ public class StoredValueTest
     @Test
     public void historyPreservingTest()
     {
-        StoredValue.HistoryPreserving<Integer> value = new StoredValue.HistoryPreserving<>(AccordState.Kind.FULL);
+        StoredValue.HistoryPreserving<Integer> value = new StoredValue.HistoryPreserving<>(AccordState.ReadWrite.FULL);
         value.load(5);
 
         Assert.assertEquals(Integer.valueOf(5), value.get());


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