Refactor CommandsForKey for efficiency, and to support transitive dependency elision

patch by Benedict; reviewed by Aleksey Yeshchenko for CASSANDRA-19310
diff --git a/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java b/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java
index b49ac43..4e17cf2 100644
--- a/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java
+++ b/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java
@@ -23,12 +23,14 @@
 import java.util.Set;
 import java.util.function.BiConsumer;
 
+import javax.annotation.Nullable;
+
 import accord.coordinate.tracking.QuorumTracker;
 import accord.local.Node;
 import accord.local.Node.Id;
 import accord.messages.Callback;
 import accord.primitives.FullRoute;
-import accord.primitives.Txn;
+import accord.primitives.Seekables;
 import accord.primitives.TxnId;
 import accord.topology.Topologies;
 import accord.utils.Invariants;
@@ -95,7 +97,6 @@
 
     final Node node;
     final TxnId txnId;
-    final Txn txn;
     final FullRoute<?> route;
 
     private Topologies topologies;
@@ -103,16 +104,15 @@
     private ExtraEpochs extraEpochs;
     private Map<Id, Object> debug = Invariants.debug() ? new LinkedHashMap<>() : null;
 
-    AbstractCoordinatePreAccept(Node node, FullRoute<?> route, TxnId txnId, Txn txn)
+    AbstractCoordinatePreAccept(Node node, FullRoute<?> route, TxnId txnId)
     {
-        this(node, txnId, txn, route, node.topology().withUnsyncedEpochs(route, txnId, txnId));
+        this(node, route, txnId, node.topology().withUnsyncedEpochs(route, txnId, txnId));
     }
 
-    AbstractCoordinatePreAccept(Node node, TxnId txnId, Txn txn, FullRoute<?> route, Topologies topologies)
+    AbstractCoordinatePreAccept(Node node, FullRoute<?> route, @Nullable TxnId txnId, Topologies topologies)
     {
         this.node = node;
         this.txnId = txnId;
-        this.txn = txn;
         this.route = route;
         this.topologies = topologies;
     }
@@ -122,6 +122,7 @@
         contact(topologies.nodes(), topologies, this);
     }
 
+    abstract Seekables<?, ?> keysOrRanges();
     abstract void contact(Set<Id> nodes, Topologies topologies, Callback<R> callback);
     abstract void onSuccessInternal(Id from, R reply);
     /**
@@ -163,7 +164,22 @@
     @Override
     public final void setFailure(Throwable failure)
     {
-        Invariants.checkState(!initialRoundIsDone || (extraEpochs != null && !extraEpochs.extraRoundIsDone));
+        super.setFailure(failure);
+        onFailure(failure);
+    }
+
+    @Override
+    public final boolean tryFailure(Throwable failure)
+    {
+        if (!super.tryFailure(failure))
+            return false;
+        onFailure(failure);
+        return true;
+    }
+
+    private void onFailure(Throwable failure)
+    {
+        // we may already be complete, as we may receive a failure from a later phase; but it's fine to redundantly mark done
         initialRoundIsDone = true;
         if (extraEpochs != null)
             extraEpochs.extraRoundIsDone = true;
@@ -178,7 +194,7 @@
             else if (failure instanceof Invalidated)
                 node.agent().metricsEventsListener().onInvalidated(txnId);
         }
-        super.setFailure(failure);
+
     }
 
     final void onPreAcceptedOrNewEpoch()
@@ -203,7 +219,7 @@
         // TODO (desired, efficiency): check if we have already have a valid quorum for the future epoch
         //  (noting that nodes may have adopted new ranges, in which case they should be discounted, and quorums may have changed shape)
         node.withEpoch(latestEpoch, () -> {
-            TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(latestEpoch), txnId, route.homeKey(), txn.keys());
+            TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(latestEpoch), txnId, route.homeKey(), keysOrRanges());
             if (mismatch != null)
             {
                 initialRoundIsDone = true;
@@ -230,23 +246,7 @@
     @Override
     public final void accept(T success, Throwable failure)
     {
-        if (success != null)
-        {
-            trySuccess(success);
-        }
-        else
-        {
-            if (failure instanceof CoordinationFailed)
-            {
-                ((CoordinationFailed) failure).set(txnId, route.homeKey());
-                if (failure instanceof Preempted)
-                    node.agent().metricsEventsListener().onPreempted(txnId);
-                else if (failure instanceof Timeout)
-                    node.agent().metricsEventsListener().onTimeout(txnId);
-                else if (failure instanceof Invalidated)
-                    node.agent().metricsEventsListener().onInvalidated(txnId);
-            }
-            tryFailure(failure);
-        }
+        if (success != null) trySuccess(success);
+        else tryFailure(failure);
     }
 }
diff --git a/accord-core/src/main/java/accord/coordinate/Barrier.java b/accord-core/src/main/java/accord/coordinate/Barrier.java
index ca5d346..8ce342f 100644
--- a/accord-core/src/main/java/accord/coordinate/Barrier.java
+++ b/accord-core/src/main/java/accord/coordinate/Barrier.java
@@ -18,7 +18,6 @@
 
 package accord.coordinate;
 
-import java.util.Objects;
 import javax.annotation.Nonnull;
 
 import accord.local.*;
@@ -30,7 +29,7 @@
 import accord.api.Key;
 import accord.api.RoutingKey;
 import accord.local.SafeCommandStore.TestDep;
-import accord.local.SafeCommandStore.TestTimestamp;
+import accord.local.SafeCommandStore.TestStartedAt;
 import accord.primitives.Routable.Domain;
 import accord.primitives.Seekables;
 import accord.primitives.SyncPoint;
@@ -42,6 +41,7 @@
 
 import static accord.local.PreLoadContext.contextFor;
 import static accord.primitives.Txn.Kind.Kinds.AnyGloballyVisible;
+import static accord.local.SafeCommandStore.TestStatus.IS_STABLE;
 import static accord.utils.Invariants.checkArgument;
 import static accord.utils.Invariants.checkState;
 import static accord.utils.Invariants.illegalState;
@@ -109,15 +109,7 @@
                     return;
                 }
 
-                if (barrierTxn != null)
-                {
-                    if (barrierTxn.status.equals(Status.Applied))
-                    {
-                        doBarrierSuccess(barrierTxn.executeAt);
-                    }
-                    // A listener was added to the transaction already
-                }
-                else
+                if (barrierTxn == null)
                 {
                     createSyncPoint();
                 }
@@ -143,7 +135,8 @@
 
     private void createSyncPoint()
     {
-        coordinateSyncPoint = CoordinateSyncPoint.inclusive(node, seekables, barrierType.async);
+        coordinateSyncPoint = barrierType.async ? CoordinateSyncPoint.inclusive(node, seekables)
+                                                : CoordinateSyncPoint.inclusiveAndAwaitQuorum(node, seekables);
         coordinateSyncPoint.addCallback((syncPoint, syncPointFailure) -> {
             if (syncPointFailure != null)
             {
@@ -153,7 +146,7 @@
 
             // Need to wait for the local transaction to finish since coordinate sync point won't wait on anything
             // if async was requested or there were no deps found
-            if (syncPoint.finishedAsync)
+            if (barrierType.async)
             {
                 TxnId txnId = syncPoint.syncId;
                 long epoch = txnId.epoch();
@@ -201,7 +194,7 @@
         ExistingTransactionCheck check = new ExistingTransactionCheck();
         Key k = seekables.get(0).asKey();
         node.commandStores().mapReduceConsume(
-                contextFor(k, KeyHistory.ALL),
+                contextFor(k, KeyHistory.COMMANDS),
                 k.toUnseekable(),
                 minEpoch,
                 Long.MAX_VALUE,
@@ -217,23 +210,13 @@
         @Nonnull
         public final Timestamp executeAt;
         @Nonnull
-        public final Status status;
-        @Nonnull
         public final Key key;
-        public BarrierTxn(@Nonnull TxnId txnId, @Nonnull Timestamp executeAt, @Nonnull Status status, Key key)
+        public BarrierTxn(@Nonnull TxnId txnId, @Nonnull Timestamp executeAt, Key key)
         {
             this.txnId = txnId;
             this.executeAt = executeAt;
-            this.status = status;
             this.key = key;
         }
-
-        public BarrierTxn max(BarrierTxn other)
-        {
-            if (other == null)
-                return this;
-            return status.compareTo(other.status) >= 0 ? this : other;
-        }
     }
 
     /*
@@ -248,31 +231,29 @@
         @Override
         public BarrierTxn apply(SafeCommandStore safeStore)
         {
-            BarrierTxn found = safeStore.mapReduce(
-                    seekables,
-                    safeStore.ranges().allAfter(minEpoch),
-                    KeyHistory.ALL,
-                    // Barriers are trying to establish that committed transactions are applied before the barrier (or in this case just minEpoch)
-                    // so all existing transaction types should ensure that at this point. An earlier txnid may have an executeAt that is after
-                    // this barrier or the transaction we listen on and that is fine
-                    AnyGloballyVisible,
-                    TestTimestamp.EXECUTES_AFTER,
-                    TxnId.minForEpoch(minEpoch),
-                    TestDep.ANY_DEPS,
-                    null,
-                    Status.Committed,
-                    Status.Applied,
-                    (p1, keyOrRange, txnId, executeAt, status, deps, barrierTxn) -> {
-                        if (keyOrRange.domain() == Domain.Key)
-                            return new BarrierTxn(txnId, executeAt, status, keyOrRange.asKey());
-                        return null;
-                    },
-                    null,
-                    null,
-                    // Take the first one we find, and call it good enough to wait on
-                    Objects::nonNull);
+            // TODO (required): consider these semantics
+            BarrierTxn found = safeStore.mapReduceFull(
+            seekables,
+            safeStore.ranges().allAfter(minEpoch),
+            // Barriers are trying to establish that committed transactions are applied before the barrier (or in this case just minEpoch)
+            // so all existing transaction types should ensure that at this point. An earlier txnid may have an executeAt that is after
+            // this barrier or the transaction we listen on and that is fine
+            TxnId.minForEpoch(minEpoch),
+            AnyGloballyVisible,
+            TestStartedAt.STARTED_AFTER,
+            TestDep.ANY_DEPS,
+            IS_STABLE,
+            (p1, keyOrRange, txnId, executeAt, barrierTxn) -> {
+                if (barrierTxn != null)
+                    return barrierTxn;
+                if (keyOrRange.domain() == Domain.Key)
+                    return new BarrierTxn(txnId, executeAt, keyOrRange.asKey());
+                return null;
+            },
+            null,
+            null);
             // It's not applied so add a listener to find out when it is applied
-            if (found != null && !found.status.equals(Status.Applied))
+            if (found != null)
             {
                 safeStore.commandStore().execute(
                         contextFor(found.txnId),
diff --git a/accord-core/src/main/java/accord/coordinate/BlockOnDeps.java b/accord-core/src/main/java/accord/coordinate/BlockOnDeps.java
deleted file mode 100644
index 65abe73..0000000
--- a/accord-core/src/main/java/accord/coordinate/BlockOnDeps.java
+++ /dev/null
@@ -1,135 +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 accord.coordinate;
-
-import java.util.function.BiConsumer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import accord.api.Result;
-import accord.coordinate.tracking.QuorumTracker;
-import accord.coordinate.tracking.RequestStatus;
-import accord.local.Node;
-import accord.local.Node.Id;
-import accord.messages.Callback;
-import accord.messages.Commit;
-import accord.messages.ReadData.CommitOrReadNack;
-import accord.messages.ReadData.ReadReply;
-import accord.primitives.Deps;
-import accord.primitives.FullRoute;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
-import accord.topology.Topologies;
-
-import static accord.coordinate.tracking.RequestStatus.Failed;
-import static accord.utils.Invariants.illegalState;
-
-/**
- * Block on deps at quorum for a sync point transaction, and then move the transaction to the applied state
- */
-public class BlockOnDeps implements Callback<ReadReply>
-{
-    @SuppressWarnings("unused")
-    private static final Logger logger = LoggerFactory.getLogger(BlockOnDeps.class);
-
-    final Node node;
-    final TxnId txnId;
-    final Txn txn;
-    final FullRoute<?> route;
-    final Deps deps;
-    final Topologies blockOn;
-    final QuorumTracker tracker;
-    final BiConsumer<? super Result, Throwable> callback;
-
-    private boolean isDone = false;
-
-    private BlockOnDeps(Node node, TxnId txnId, Txn txn, FullRoute<?> route,  Deps deps, BiConsumer<? super Result, Throwable> callback)
-    {
-        this.node = node;
-        this.txnId = txnId;
-        this.txn = txn;
-        this.route = route;
-        this.deps = deps;
-        this.blockOn = node.topology().forEpoch(route, txnId.epoch());
-        this.tracker = new QuorumTracker(blockOn);
-        this.callback = callback;
-    }
-
-    public static void blockOnDeps(Node node, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, BiConsumer<? super Result, Throwable> callback)
-    {
-        BlockOnDeps blockOnDeps = new BlockOnDeps(node, txnId, txn, route, deps, callback);
-        blockOnDeps.start();
-    }
-
-    void start()
-    {
-        Commit.stableMaximalAndBlockOnDeps(node, tracker.topologies(), txnId, txn, route, txnId, deps, this);
-    }
-
-    @Override
-    public void onSuccess(Id from, ReadReply reply)
-    {
-        if (isDone)
-            return;
-
-        if (reply.isOk())
-        {
-            if (tracker.recordSuccess(from) == RequestStatus.Success)
-            {
-                isDone = true;
-                callback.accept(txn.result(txnId, txnId, null), null);
-            }
-            return;
-        }
-
-        CommitOrReadNack nack = (CommitOrReadNack) reply;
-        switch (nack)
-        {
-            default: throw illegalState();
-            case Redundant:
-                // WaitUntilApplied only sends Redundant on truncation which implies durable and applied
-                isDone = true;
-                callback.accept(txn.result(txnId, txnId, null), null);
-                break;
-            case Insufficient:
-                throw illegalState("Received `NotCommitted` response after sending maximal stable as part of `BlockOnDeps`");
-            case Invalid:
-                onFailure(from, illegalState("Submitted a read command to a replica that did not own the range"));
-                break;
-        }
-    }
-
-    @Override
-    public void onFailure(Id from, Throwable failure)
-    {
-        if (tracker.recordFailure(from) == Failed)
-        {
-            isDone = true;
-            callback.accept(null, failure);
-        }
-    }
-
-    @Override
-    public void onCallbackFailure(Id from, Throwable failure)
-    {
-        isDone = true;
-        callback.accept(null, failure);
-    }
-}
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java b/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java
index b07d677..fa94afb 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java
@@ -31,6 +31,7 @@
 import accord.messages.GetEphemeralReadDeps.GetEphemeralReadDepsOk;
 import accord.primitives.Deps;
 import accord.primitives.FullRoute;
+import accord.primitives.Seekables;
 import accord.primitives.Txn;
 import accord.primitives.TxnId;
 import accord.topology.Topologies;
@@ -67,19 +68,28 @@
         return coordinate;
     }
 
+    private final Txn txn;
+
     private final QuorumTracker tracker;
     private final List<GetEphemeralReadDepsOk> oks;
     private long executeAtEpoch;
 
     CoordinateEphemeralRead(Node node, Topologies topologies, FullRoute<?> route, TxnId txnId, Txn txn)
     {
-        super(node, route, txnId, txn);
+        super(node, route, txnId);
+        this.txn = txn;
         this.tracker = new QuorumTracker(topologies);
         this.executeAtEpoch = txnId.epoch();
         this.oks = new ArrayList<>(topologies.estimateUniqueNodes());
     }
 
     @Override
+    Seekables<?, ?> keysOrRanges()
+    {
+        return txn.keys();
+    }
+
+    @Override
     void contact(Set<Node.Id> nodes, Topologies topologies, Callback<GetEphemeralReadDepsOk> callback)
     {
         CommandStore commandStore = CommandStore.maybeCurrent();
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateGloballyDurable.java b/accord-core/src/main/java/accord/coordinate/CoordinateGloballyDurable.java
index 5186d5d..94528bb 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateGloballyDurable.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateGloballyDurable.java
@@ -31,11 +31,15 @@
 import accord.utils.async.AsyncResult;
 import accord.utils.async.AsyncResults.SettableResult;
 
+import static accord.primitives.Routable.Domain.Range;
+import static accord.primitives.Txn.Kind.LocalOnly;
+
 // TODO (expected): this does not need to query every shard; can disseminate globally any sub-range of the ring
 //  (indeed, we could slice both the query and dissemination only so that they always overlap)
 public class CoordinateGloballyDurable extends SettableResult<Void> implements Callback<DurableBeforeReply>
 {
     final Node node;
+    final TxnId txnId; // only used for key into journal
     // TODO (expected): this can be a ReadTracker, we only need one response from each shard
     final QuorumTracker tracker;
     private DurableBefore durableBefore = DurableBefore.EMPTY;
@@ -44,6 +48,7 @@
     {
         Topologies topologies = node.topology().preciseEpochs(epoch);
         this.node = node;
+        this.txnId = node.nextTxnId(LocalOnly, Range);
         this.tracker = new QuorumTracker(topologies);
     }
 
@@ -65,8 +70,8 @@
         durableBefore = DurableBefore.merge(durableBefore, reply.durableBeforeMap);
         if (tracker.recordSuccess(from) == RequestStatus.Success)
         {
-            if (durableBefore != null && !durableBefore.equals(TxnId.NONE))
-                node.send(tracker.nodes(), new SetGloballyDurable(durableBefore));
+            if (durableBefore != null && durableBefore.size() != 0)
+                node.send(tracker.nodes(), new SetGloballyDurable(txnId, durableBefore));
             trySuccess(null);
         }
     }
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateNoOp.java b/accord-core/src/main/java/accord/coordinate/CoordinateNoOp.java
deleted file mode 100644
index 40c009d..0000000
--- a/accord-core/src/main/java/accord/coordinate/CoordinateNoOp.java
+++ /dev/null
@@ -1,98 +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 accord.coordinate;
-
-import java.util.List;
-
-import accord.api.Result;
-import accord.local.Node;
-import accord.messages.Apply;
-import accord.messages.PreAccept.PreAcceptOk;
-import accord.primitives.Ballot;
-import accord.primitives.Deps;
-import accord.primitives.FullRoute;
-import accord.primitives.Seekables;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
-import accord.primitives.Writes;
-import accord.topology.Topologies;
-import accord.utils.Invariants;
-import accord.utils.async.AsyncResult;
-import accord.utils.async.AsyncResults;
-
-import static accord.coordinate.Propose.Invalidate.proposeAndCommitInvalidate;
-import static accord.primitives.Txn.Kind.NoOp;
-
-/**
- * Perform initial rounds of PreAccept and Accept until we have reached agreement about when we should execute.
- * If we are preempted by a recovery coordinator, we abort and let them complete (and notify us about the execution result)
- *
- * TODO (desired, testing): dedicated burn test to validate outcomes
- */
-public class CoordinateNoOp extends CoordinatePreAccept<Timestamp>
-{
-    private CoordinateNoOp(Node node, TxnId txnId, Txn txn, FullRoute<?> route)
-    {
-        super(node, txnId, txn, route);
-    }
-
-    public static AsyncResult<Timestamp> coordinate(Node node, Seekables<?, ?> keysOrRanges)
-    {
-        TxnId txnId = node.nextTxnId(NoOp, keysOrRanges.domain());
-        return coordinate(node, txnId, keysOrRanges);
-    }
-
-    public static AsyncResult<Timestamp> coordinate(Node node, TxnId txnId, Seekables<?, ?> keysOrRanges)
-    {
-        Invariants.checkArgument(txnId.kind() == NoOp);
-        FullRoute<?> route = node.computeRoute(txnId, keysOrRanges);
-        TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(txnId.epoch()), txnId, route.homeKey(), keysOrRanges);
-        if (mismatch != null)
-            return AsyncResults.failure(mismatch);
-        CoordinateNoOp coordinate = new CoordinateNoOp(node, txnId, node.agent().emptyTxn(NoOp, keysOrRanges), route);
-        coordinate.start();
-        return coordinate;
-    }
-
-    @Override
-    void onPreAccepted(Topologies topologies, Timestamp executeAt, List<PreAcceptOk> oks)
-    {
-        if (executeAt.isRejected())
-        {
-            proposeAndCommitInvalidate(node, Ballot.ZERO, txnId, route.homeKey(), route, executeAt, this);
-        }
-        else
-        {
-            Deps preacceptDeps = Deps.merge(oks, ok -> ok.deps);
-            new Propose<Timestamp>(node, topologies, Ballot.ZERO, txnId, txn, route, executeAt, preacceptDeps, this)
-            {
-                @Override
-                void onAccepted()
-                {
-                    Writes writes = txn.execute(txnId, txnId, null);
-                    Result result = txn.result(txnId, executeAt, null);
-                    Deps acceptDeps = Deps.merge(this.acceptOks, ok -> ok.deps);
-                    Apply.sendMaximal(node, txnId, route, txn, executeAt, acceptDeps, writes, result);
-                    accept(executeAt, null);
-                }
-            }.start();
-        }
-    }
-}
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java b/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java
index bfa0748..74979f4 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java
@@ -32,6 +32,7 @@
 import accord.messages.PreAccept.PreAcceptReply;
 import accord.primitives.Ballot;
 import accord.primitives.FullRoute;
+import accord.primitives.Seekables;
 import accord.primitives.Timestamp;
 import accord.primitives.Txn;
 import accord.primitives.TxnId;
@@ -51,6 +52,7 @@
 {
     final FastPathTracker tracker;
     private final List<PreAcceptOk> oks;
+    final Txn txn;
 
     CoordinatePreAccept(Node node, TxnId txnId, Txn txn, FullRoute<?> route)
     {
@@ -59,9 +61,10 @@
 
     CoordinatePreAccept(Node node, TxnId txnId, Txn txn, FullRoute<?> route, Topologies topologies)
     {
-        super(node, route, txnId, txn);
+        super(node, route, txnId);
         this.tracker = new FastPathTracker(topologies);
         this.oks = new ArrayList<>(topologies.estimateUniqueNodes());
+        this.txn = txn;
     }
 
     void contact(Set<Id> nodes, Topologies topologies, Callback<PreAcceptReply> callback)
@@ -80,6 +83,12 @@
     }
 
     @Override
+    Seekables<?, ?> keysOrRanges()
+    {
+        return txn.keys();
+    }
+
+    @Override
     public void onFailureInternal(Id from, Throwable failure)
     {
         switch (tracker.recordFailure(from))
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java b/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java
index 28487c9..61b2e17 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java
@@ -19,93 +19,39 @@
 package accord.coordinate;
 
 import accord.coordinate.tracking.AppliedTracker;
-import accord.coordinate.tracking.RequestStatus;
 import accord.local.Node;
 import accord.messages.Callback;
-import accord.messages.ReadData;
 import accord.messages.ReadData.ReadReply;
 import accord.messages.SetShardDurable;
 import accord.messages.WaitUntilApplied;
 import accord.primitives.Ranges;
 import accord.primitives.SyncPoint;
-import accord.topology.Topologies;
 import accord.utils.async.AsyncResult;
-import accord.utils.async.AsyncResults.SettableResult;
 
-/**
- *
- */
-public class CoordinateShardDurable extends SettableResult<Void> implements Callback<ReadReply>
+public class CoordinateShardDurable extends ExecuteSyncPoint<Ranges> implements Callback<ReadReply>
 {
-    final Node node;
-    final AppliedTracker tracker;
-    final SyncPoint<Ranges> exclusiveSyncPoint;
-
     private CoordinateShardDurable(Node node, SyncPoint<Ranges> exclusiveSyncPoint)
     {
-        // TODO (required): this isn't correct, we need to potentially perform a second round if a dependency executes in a future epoch and we have lost ownership of that epoch
-        Topologies topologies = node.topology().forEpoch(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.sourceEpoch());
-        this.node = node;
-        this.tracker = new AppliedTracker(topologies);
-        this.exclusiveSyncPoint = exclusiveSyncPoint;
+        super(node, new AppliedTracker(node.topology().forEpoch(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.sourceEpoch())), exclusiveSyncPoint);
     }
 
-    public static AsyncResult<Void> coordinate(Node node, SyncPoint<Ranges> exclusiveSyncPoint)
+    public static AsyncResult<SyncPoint<Ranges>> coordinate(Node node, SyncPoint<Ranges> exclusiveSyncPoint)
     {
         CoordinateShardDurable coordinate = new CoordinateShardDurable(node, exclusiveSyncPoint);
         coordinate.start();
         return coordinate;
     }
 
-    private void start()
+    protected void start()
     {
-        node.send(tracker.nodes(), to -> new WaitUntilApplied(to, tracker.topologies(), exclusiveSyncPoint.syncId, exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.syncId.epoch()), this);
+        node.send(tracker.nodes(), to -> new WaitUntilApplied(to, tracker.topologies(), syncPoint.syncId, syncPoint.keysOrRanges, syncPoint.syncId.epoch()), this);
     }
 
     @Override
-    public void onSuccess(Node.Id from, ReadReply reply)
+    protected void onSuccess()
     {
-        if (!reply.isOk())
-        {
-            switch ((ReadData.CommitOrReadNack)reply)
-            {
-                default: throw new AssertionError("Unhandled: " + reply);
-
-                case Insufficient:
-                    CoordinateSyncPoint.sendApply(node, from, exclusiveSyncPoint);
-                    return;
-
-                case Redundant:
-                    tryFailure(new RuntimeException("Unexpected reply"));
-                    return;
-
-                case Invalid:
-                    tryFailure(new Invalidated(exclusiveSyncPoint.syncId, exclusiveSyncPoint.homeKey));
-                    return;
-            }
-        }
-        else
-        {
-            // TODO (required): we also need to handle ranges not being safe to read
-            if (tracker.recordSuccess(from) == RequestStatus.Success)
-            {
-                node.configService().reportEpochRedundant(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.syncId.epoch());
-                node.send(tracker.nodes(), new SetShardDurable(exclusiveSyncPoint));
-                trySuccess(null);
-            }
-        }
-    }
-
-    @Override
-    public void onFailure(Node.Id from, Throwable failure)
-    {
-        if (tracker.recordFailure(from) == RequestStatus.Failed)
-            tryFailure(new Exhausted(exclusiveSyncPoint.syncId, exclusiveSyncPoint.homeKey));
-    }
-
-    @Override
-    public void onCallbackFailure(Node.Id from, Throwable failure)
-    {
-        tryFailure(failure);
+        node.configService().reportEpochRedundant(syncPoint.keysOrRanges, syncPoint.syncId.epoch());
+        node.send(tracker.nodes(), new SetShardDurable(syncPoint));
+        super.onSuccess();
     }
 }
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java b/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java
index a1987fd..49e170b 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java
@@ -19,11 +19,11 @@
 package accord.coordinate;
 
 import java.util.List;
-import java.util.function.BiConsumer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import accord.coordinate.CoordinationAdapter.Adapters;
 import accord.local.Node;
 import accord.messages.Apply;
 import accord.messages.PreAccept.PreAcceptOk;
@@ -40,6 +40,9 @@
 import accord.utils.async.AsyncResult;
 import accord.utils.async.AsyncResults;
 
+import static accord.coordinate.CoordinationAdapter.Invoke.execute;
+import static accord.coordinate.CoordinationAdapter.Invoke.propose;
+import static accord.coordinate.ExecutePath.FAST;
 import static accord.coordinate.Propose.Invalidate.proposeAndCommitInvalidate;
 import static accord.primitives.Timestamp.mergeMax;
 import static accord.primitives.Txn.Kind.ExclusiveSyncPoint;
@@ -57,76 +60,53 @@
     @SuppressWarnings("unused")
     private static final Logger logger = LoggerFactory.getLogger(CoordinateSyncPoint.class);
 
-    final S keysOrRanges;
-    // Whether to wait on the dependencies applying globally before returning a result
-    final boolean async;
+    final CoordinationAdapter<SyncPoint<S>> adapter;
 
-    private CoordinateSyncPoint(Node node, TxnId txnId, Txn txn, FullRoute<?> route, S keysOrRanges, boolean async)
+    private CoordinateSyncPoint(Node node, TxnId txnId, Txn txn, FullRoute<?> route, CoordinationAdapter<SyncPoint<S>> adapter)
     {
         super(node, txnId, txn, route, node.topology().withOpenEpochs(route, txnId, txnId));
-        checkArgument(txnId.kind() == Kind.SyncPoint || async, "Exclusive sync points only support async application");
-        this.keysOrRanges = keysOrRanges;
-        this.async = async;
+        this.adapter = adapter;
     }
 
     public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> exclusive(Node node, S keysOrRanges)
     {
-        return coordinate(node, ExclusiveSyncPoint, keysOrRanges, true);
+        return coordinate(node, ExclusiveSyncPoint, keysOrRanges, Adapters.exclusiveSyncPoint());
     }
 
     public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> exclusive(Node node, TxnId txnId, S keysOrRanges)
     {
-        return coordinate(node, txnId, keysOrRanges, true);
+        return coordinate(node, txnId, keysOrRanges, Adapters.exclusiveSyncPoint());
     }
 
-    public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> inclusive(Node node, S keysOrRanges, boolean async)
+    public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> inclusive(Node node, S keysOrRanges)
     {
-        return coordinate(node, Kind.SyncPoint, keysOrRanges, async);
+        return coordinate(node, Kind.SyncPoint, keysOrRanges, Adapters.inclusiveSyncPoint());
     }
 
-    private static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> coordinate(Node node, Kind kind, S keysOrRanges, boolean async)
+    public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> inclusiveAndAwaitQuorum(Node node, S keysOrRanges)
+    {
+        return coordinate(node, Kind.SyncPoint, keysOrRanges, Adapters.inclusiveSyncPointBlocking());
+    }
+
+    private static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> coordinate(Node node, Kind kind, S keysOrRanges, CoordinationAdapter<SyncPoint<S>> adapter)
     {
         checkArgument(kind == Kind.SyncPoint || kind == ExclusiveSyncPoint);
-        node.nextTxnId(Kind.SyncPoint, keysOrRanges.domain());
         TxnId txnId = node.nextTxnId(kind, keysOrRanges.domain());
-        return node.withEpoch(txnId.epoch(), () -> coordinate(node, txnId, keysOrRanges, async)).beginAsResult();
+        return node.withEpoch(txnId.epoch(), () -> coordinate(node, txnId, keysOrRanges, adapter)).beginAsResult();
     }
 
-    private static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> coordinate(Node node, TxnId txnId, S keysOrRanges, boolean async)
+    private static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> coordinate(Node node, TxnId txnId, S keysOrRanges, CoordinationAdapter<SyncPoint<S>> adapter)
     {
         checkArgument(txnId.kind() == Kind.SyncPoint || txnId.kind() == ExclusiveSyncPoint);
-        FullRoute route = node.computeRoute(txnId, keysOrRanges);
+        FullRoute<?> route = node.computeRoute(txnId, keysOrRanges);
         TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(txnId.epoch()), txnId, route.homeKey(), keysOrRanges);
         if (mismatch != null)
             return AsyncResults.failure(mismatch);
-        CoordinateSyncPoint<S> coordinate = new CoordinateSyncPoint(node, txnId, node.agent().emptyTxn(txnId.kind(), keysOrRanges), route, keysOrRanges, async);
+        CoordinateSyncPoint<S> coordinate = new CoordinateSyncPoint<>(node, txnId, node.agent().emptyTxn(txnId.kind(), keysOrRanges), route, adapter);
         coordinate.start();
         return coordinate;
     }
 
-    static <S extends Seekables<?, ?>> void blockOnDeps(Node node, Txn txn, TxnId txnId, FullRoute<?> route, S keysOrRanges, Deps deps, BiConsumer<SyncPoint<S>, Throwable> callback, boolean async)
-    {
-        // If deps are empty there is nothing to wait on application for so we can return immediately
-        boolean processAsyncCompletion = deps.isEmpty() || async;
-        BlockOnDeps.blockOnDeps(node, txnId, txn, route, deps, (result, throwable) -> {
-            // Don't want to process completion twice
-            if (processAsyncCompletion)
-            {
-                // Don't lose the error
-                if (throwable != null)
-                    node.agent().onUncaughtException(throwable);
-                return;
-            }
-            if (throwable != null)
-                callback.accept(null, throwable);
-            else
-                callback.accept(new SyncPoint<S>(txnId, deps, keysOrRanges, route, false), null);
-        });
-        // Notify immediately and the caller can add a listener to command completion to track local application
-        if (processAsyncCompletion)
-            callback.accept(new SyncPoint<S>(txnId, deps, keysOrRanges, route, true), null);
-    }
-
     @Override
     long executeAtEpoch()
     {
@@ -146,14 +126,15 @@
         {
             // we don't need to fetch deps from Accept replies, so we don't need to contact unsynced epochs
             topologies = node.topology().forEpoch(route, txnId.epoch());
+            // TODO (required): consider the required semantics of a SyncPoint
             if (tracker.hasFastPathAccepted() && txnId.kind() == Kind.SyncPoint)
-                blockOnDeps(node, txn, txnId, route, keysOrRanges, deps, this, async);
+                execute(adapter, node, topologies, route, FAST, txnId, txn, executeAt, deps, this);
             else
-                ProposeSyncPoint.proposeSyncPoint(node, topologies, Ballot.ZERO, txnId, txn, route, deps, this, async, tracker.nodes(), keysOrRanges);
+                propose(adapter, node, topologies, route, Ballot.ZERO, txnId, txn, executeAt, deps, this);
         }
     }
 
-    public static void sendApply(Node node, Node.Id to, SyncPoint syncPoint)
+    public static void sendApply(Node node, Node.Id to, SyncPoint<?> syncPoint)
     {
         TxnId txnId = syncPoint.syncId;
         Timestamp executeAt = txnId;
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java b/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java
index 39b9784..dc4395c 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java
@@ -21,6 +21,7 @@
 import java.util.List;
 
 import accord.api.Result;
+import accord.coordinate.CoordinationAdapter.Adapters;
 import accord.messages.PreAccept;
 import accord.topology.Topologies;
 import accord.local.Node;
@@ -34,9 +35,11 @@
 import accord.utils.async.AsyncResult;
 import accord.utils.async.AsyncResults;
 
-import static accord.coordinate.Execute.Path.FAST;
+import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue;
+import static accord.coordinate.CoordinationAdapter.Invoke.execute;
+import static accord.coordinate.CoordinationAdapter.Invoke.propose;
+import static accord.coordinate.ExecutePath.FAST;
 import static accord.coordinate.Propose.Invalidate.proposeAndCommitInvalidate;
-import static accord.coordinate.ProposeAndExecute.proposeAndExecute;
 
 /**
  * Perform initial rounds of PreAccept and Accept until we have reached agreement about when we should execute.
@@ -46,9 +49,12 @@
  */
 public class CoordinateTransaction extends CoordinatePreAccept<Result>
 {
+    final Txn txn;
+
     private CoordinateTransaction(Node node, TxnId txnId, Txn txn, FullRoute<?> route)
     {
         super(node, txnId, txn, route);
+        this.txn = txn;
     }
 
     public static AsyncResult<Result> coordinate(Node node, FullRoute<?> route, TxnId txnId, Txn txn)
@@ -67,7 +73,7 @@
         if (tracker.hasFastPathAccepted())
         {
             Deps deps = Deps.merge(oks, ok -> ok.witnessedAt.equals(txnId) ? ok.deps : null);
-            Execute.execute(node, topologies, route, FAST, txnId, txn, txnId, deps, this);
+            execute(executeAdapter(), node, topologies, route, FAST, txnId, txn, txnId, deps, settingCallback());
             node.agent().metricsEventsListener().onFastPathTaken(txnId, deps);
         }
         else
@@ -87,10 +93,21 @@
                 if (PreAccept.rejectExecuteAt(txnId, topologies))
                     proposeAndCommitInvalidate(node, Ballot.ZERO, txnId, route.homeKey(), route, executeAt, this);
                 else
-                    proposeAndExecute(node, topologies, Ballot.ZERO, txnId, txn, route, executeAt, deps, this);
+                    propose(proposeAdapter(), node, topologies, route, Ballot.ZERO, txnId, txn, executeAt, deps, this);
             }
 
             node.agent().metricsEventsListener().onSlowPathTaken(txnId, deps);
         }
     }
+
+    protected CoordinationAdapter<Result> proposeAdapter()
+    {
+        return Adapters.standard();
+    }
+
+    // TODO (expected): override in C* rather than default to configurability here
+    protected CoordinationAdapter<Result> executeAdapter()
+    {
+        return node.coordinationAdapter(txnId, Continue);
+    }
 }
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java b/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java
new file mode 100644
index 0000000..00d3460
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java
@@ -0,0 +1,284 @@
+/*
+ * 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 accord.coordinate;
+
+import java.util.function.BiConsumer;
+
+import javax.annotation.Nullable;
+
+import accord.api.Result;
+import accord.coordinate.ExecuteSyncPoint.ExecuteAtQuorum;
+import accord.local.Node;
+import accord.messages.Apply;
+import accord.primitives.Ballot;
+import accord.primitives.Deps;
+import accord.primitives.FullRoute;
+import accord.primitives.Participants;
+import accord.primitives.Ranges;
+import accord.primitives.Seekables;
+import accord.primitives.SyncPoint;
+import accord.primitives.Timestamp;
+import accord.primitives.Txn;
+import accord.primitives.TxnId;
+import accord.primitives.Writes;
+import accord.topology.Topologies;
+import accord.utils.Faults;
+
+import static accord.coordinate.ExecutePath.FAST;
+import static accord.coordinate.ExecutePath.SLOW;
+import static accord.messages.Apply.Kind.Maximal;
+import static accord.messages.Apply.Kind.Minimal;
+import static accord.primitives.Routable.Domain.Range;
+
+public interface CoordinationAdapter<R>
+{
+    interface Factory
+    {
+        enum Step { Continue, InitiateRecovery }
+        <R> CoordinationAdapter<R> get(TxnId txnId, Step step);
+    }
+
+    void propose(Node node, Topologies withUnsynced, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback);
+    void stabilise(Node node, Topologies coordinates, Topologies all, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback);
+    void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback);
+    void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super R, Throwable> callback);
+
+    class DefaultFactory implements Factory
+    {
+        @Override
+        public <R> CoordinationAdapter<R> get(TxnId txnId, Step step)
+        {
+            switch (step)
+            {
+                default: throw new AssertionError("Unhandled step: " + step);
+                case Continue: return (CoordinationAdapter<R>) Adapters.standard();
+                case InitiateRecovery: return (CoordinationAdapter<R>) Adapters.recovery();
+            }
+        }
+    }
+
+    /**
+     * Utility methods for correctly invoking the next phase of the state machine via a CoordinationAdapter.
+     * Simply ensures the topologies are correct before being passed to the instance method.
+     */
+    class Invoke
+    {
+        public static <R> void propose(CoordinationAdapter<R> adapter, Node node, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
+        {
+            propose(adapter, node, node.topology().withUnsyncedEpochs(route, txnId, executeAt), route, ballot, txnId, txn, executeAt, deps, callback);
+        }
+
+        public static <R> void propose(CoordinationAdapter<R> adapter, Node node, Topologies withUnsynced, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
+        {
+            adapter.propose(node, withUnsynced, route, ballot, txnId, txn, executeAt, deps, callback);
+        }
+
+        public static <R> void stabilise(CoordinationAdapter<R> adapter, Node node, Topologies any, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
+        {
+            Topologies coordinates = any.forEpochs(txnId.epoch(), txnId.epoch());
+            Topologies all;
+            if (txnId.epoch() == executeAt.epoch()) all = coordinates;
+            else if (any.currentEpoch() >= executeAt.epoch() && any.oldestEpoch() <= txnId.epoch()) all = any.forEpochs(txnId.epoch(), executeAt.epoch());
+            else all = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
+
+            adapter.stabilise(node, any, all, route, ballot, txnId, txn, executeAt, deps, callback);
+        }
+
+        public static <R> void execute(CoordinationAdapter<R> adapter, Node node, Topologies any, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
+        {
+            if (any.oldestEpoch() <= txnId.epoch() && any.currentEpoch() >= executeAt.epoch()) any = any.forEpochs(txnId.epoch(), executeAt.epoch());
+            else any = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
+            adapter.execute(node, any, route, path, txnId, txn, executeAt, deps, callback);
+        }
+
+        public static <R> void persist(CoordinationAdapter<R> adapter, Node node, Topologies any, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, @Nullable BiConsumer<? super R, Throwable> callback)
+        {
+            if (any.oldestEpoch() <= txnId.epoch() && any.currentEpoch() >= executeAt.epoch()) any = any.forEpochs(txnId.epoch(), executeAt.epoch());
+            else any = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
+            Topologies executes = any.forEpochs(executeAt.epoch(), executeAt.epoch());
+
+            adapter.persist(node, any, route, txnId, txn, executeAt, deps, writes, result, callback);
+        }
+
+        public static <R> void persist(CoordinationAdapter<R> adapter, Node node, FullRoute<?> route, Participants<?> sendTo, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, @Nullable BiConsumer<? super R, Throwable> callback)
+        {
+            Topologies all = node.topology().preciseEpochs(sendTo, txnId.epoch(), executeAt.epoch());
+            Topologies executes = all.forEpochs(executeAt.epoch(), executeAt.epoch());
+
+            adapter.persist(node, all, route, txnId, txn, executeAt, deps, writes, result, callback);
+        }
+    }
+
+    class Adapters
+    {
+        public static CoordinationAdapter<Result> standard()
+        {
+            return StandardTxnAdapter.INSTANCE;
+        }
+
+        // note that by default the recovery adapter is only used for the initial recovery decision - if e.g. propose is initiated
+        // then we revert back to standard adapter behaviour for later steps
+        public static CoordinationAdapter<Result> recovery()
+        {
+            return RecoveryTxnAdapter.INSTANCE;
+        }
+
+        public static <S extends Seekables<?, ?>> CoordinationAdapter<SyncPoint<S>> inclusiveSyncPoint()
+        {
+            return InclusiveSyncPointAdapter.INSTANCE;
+        }
+
+        public static <S extends Seekables<?, ?>> CoordinationAdapter<SyncPoint<S>> inclusiveSyncPointBlocking()
+        {
+            return InclusiveSyncPointBlockingAdapter.INSTANCE;
+        }
+
+        public static <S extends Seekables<?, ?>> CoordinationAdapter<SyncPoint<S>> exclusiveSyncPoint()
+        {
+            return ExclusiveSyncPointAdapter.INSTANCE;
+        }
+
+        public static abstract class AbstractTxnAdapter implements CoordinationAdapter<Result>
+        {
+            @Override
+            public void propose(Node node, Topologies withUnsynced, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
+            {
+                new ProposeTxn(node, withUnsynced, route, ballot, txnId, txn, executeAt, deps, callback).start();
+            }
+
+            @Override
+            public void stabilise(Node node, Topologies coordinates, Topologies all, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
+            {
+                if (Faults.TRANSACTION_INSTABILITY) execute(node, all, route, SLOW, txnId, txn, executeAt, deps, callback);
+                else new StabiliseTxn(node, coordinates, all, route, ballot, txnId, txn, executeAt, deps, callback).start();
+            }
+
+            @Override
+            public void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
+            {
+                if (txn.read().keys().isEmpty()) Invoke.persist(this, node, all, route, txnId, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null), callback);
+                else new ExecuteTxn(node, all, route, path, txnId, txn, txn.read().keys().toParticipants(), executeAt, deps, callback).start();
+            }
+        }
+
+        public static class StandardTxnAdapter extends AbstractTxnAdapter
+        {
+            public static final StandardTxnAdapter INSTANCE = new StandardTxnAdapter();
+            @Override
+            public void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super Result, Throwable> callback)
+            {
+                if (callback != null) callback.accept(result, null);
+                new PersistTxn(node, all, txnId, route, txn, executeAt, deps, writes, result)
+                    .start(Apply.FACTORY, Minimal, all, writes, result);
+            }
+        }
+
+        public static class RecoveryTxnAdapter extends AbstractTxnAdapter
+        {
+            public static final RecoveryTxnAdapter INSTANCE = new RecoveryTxnAdapter();
+            @Override
+            public void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super Result, Throwable> callback)
+            {
+                if (callback != null) callback.accept(result, null);
+                new PersistTxn(node, all, txnId, route, txn, executeAt, deps, writes, result)
+                    .start(Apply.FACTORY, Maximal, all, writes, result);
+            }
+        }
+
+        static abstract class AbstractSyncPointAdapter<S extends Seekables<?, ?>> implements CoordinationAdapter<SyncPoint<S>>
+        {
+            void invokeSuccess(Node node, FullRoute<?> route, TxnId txnId, Txn txn, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+            {
+                if (txn.keys().domain() == Range)
+                    node.configService().reportEpochClosed((Ranges)txn.keys(), txnId.epoch());
+                callback.accept(new SyncPoint<>(txnId, deps, (S)txn.keys(), route), null);
+            }
+
+            @Override
+            public void propose(Node node, Topologies withUnsynced, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+            {
+                new ProposeSyncPoint<>(this, node, withUnsynced, route, ballot, txnId, txn, executeAt, deps, callback).start();
+            }
+
+            @Override
+            public void stabilise(Node node, Topologies coordinates, Topologies all, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+            {
+                new StabiliseSyncPoint<>(this, node, coordinates, all, route, ballot, txnId, txn, executeAt, deps, callback).start();
+            }
+
+            @Override
+            public void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+            {
+                Invoke.persist(this, node, all, route, txnId, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null), callback);
+            }
+
+            @Override
+            public void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+            {
+                invokeSuccess(node, route, txnId, txn, deps, callback);
+                new PersistTxn(node, all, txnId, route, txn, executeAt, deps, writes, result)
+                    .start(Apply.FACTORY, Maximal, all, writes, result);
+            }
+        }
+
+        public static class ExclusiveSyncPointAdapter<S extends Seekables<?, ?>> extends AbstractSyncPointAdapter<S>
+        {
+            private static final ExclusiveSyncPointAdapter INSTANCE = new ExclusiveSyncPointAdapter();
+
+            @Override
+            public void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+            {
+                // TODO (required): remember and document why we don't use fast path for exclusive sync points
+                if (path == FAST)
+                {
+                    Invoke.stabilise(this, node, all, route, Ballot.ZERO, txnId, txn, executeAt, deps, callback);
+                }
+                else
+                {
+                    super.execute(node, all, route, path, txnId, txn, executeAt, deps, callback);
+                }
+            }
+        }
+
+        public static class InclusiveSyncPointAdapter<S extends Seekables<?, ?>> extends AbstractSyncPointAdapter<S>
+        {
+            private static final InclusiveSyncPointAdapter INSTANCE = new InclusiveSyncPointAdapter();
+        }
+
+        public static class InclusiveSyncPointBlockingAdapter<S extends Seekables<?, ?>> extends AbstractSyncPointAdapter<S>
+        {
+            private static final InclusiveSyncPointBlockingAdapter INSTANCE = new InclusiveSyncPointBlockingAdapter();
+
+            @Override
+            public void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+            {
+                ExecuteAtQuorum<S> execute = new ExecuteAtQuorum<S>(node, all, new SyncPoint<>(txnId, deps, (S)txn.keys(), route));
+                execute.addCallback(callback);
+                execute.start();
+            }
+
+            @Override
+            public void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+            {
+                throw new UnsupportedOperationException();
+            }
+        }
+    }
+
+}
diff --git a/accord-core/src/main/java/accord/coordinate/Execute.java b/accord-core/src/main/java/accord/coordinate/Execute.java
deleted file mode 100644
index dec581d..0000000
--- a/accord-core/src/main/java/accord/coordinate/Execute.java
+++ /dev/null
@@ -1,79 +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 accord.coordinate;
-
-import java.util.function.BiConsumer;
-
-import accord.api.Result;
-import accord.local.Node;
-import accord.primitives.Deps;
-import accord.primitives.FullRoute;
-import accord.primitives.Participants;
-import accord.primitives.Seekables;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
-import accord.primitives.Writes;
-import accord.primitives.Txn.Kind;
-import accord.topology.Topologies;
-
-import static accord.utils.Invariants.checkArgument;
-
-public interface Execute
-{
-    enum Path { FAST, SLOW, RECOVER }
-
-    interface Factory
-    {
-        Execute create(Node node, Topologies topologies, Path path, TxnId txnId, Txn txn, FullRoute<?> route, Participants<?> readScope, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback);
-    }
-
-    void start();
-
-    static void execute(Node node, Topologies anyTopologies, FullRoute<?> route, Path path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
-    {
-        Seekables<?, ?> readKeys = txn.read().keys();
-        Participants<?> readScope = readKeys.toParticipants();
-        // Recovery calls execute and we would like execute to run BlockOnDeps because that will notify the agent
-        // of the local barrier
-        if (txn.kind() == Kind.SyncPoint)
-        {
-            checkArgument(txnId.equals(executeAt));
-            BlockOnDeps.blockOnDeps(node, txnId, txn, route, deps, callback);
-        }
-        else
-        {
-            if (readKeys.isEmpty())
-            {
-                Result result = txn.result(txnId, executeAt, null);
-                Writes writes = txn.execute(txnId, executeAt, null);
-                anyTopologies = anyTopologies.forEpochs(executeAt.epoch(), executeAt.epoch());
-                Persist.persist(node, anyTopologies, route, txnId, txn, executeAt, deps, writes, result, callback);
-            }
-            else
-            {
-                if (anyTopologies.oldestEpoch() <= txnId.epoch() && anyTopologies.currentEpoch() >= executeAt.epoch()) anyTopologies = anyTopologies.forEpochs(txnId.epoch(), executeAt.epoch());
-                else anyTopologies = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
-
-                Execute execute = node.executionFactory().create(node, anyTopologies, path, txnId, txn, route, readScope, executeAt, deps, callback);
-                execute.start();
-            }
-        }
-    }
-}
diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java b/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java
index 07f22ef..81f7f6c 100644
--- a/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java
+++ b/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java
@@ -112,6 +112,7 @@
             default: throw new IllegalStateException();
             case Redundant:
             case Rejected:
+                // TODO (expected): shouldn't be preemptible (can be made redundant, but should be a special case)
                 callback.accept(null, new Preempted(txnId, route.homeKey()));
                 return Action.Aborted;
             case Insufficient:
diff --git a/accord-core/src/main/java/accord/impl/DomainTimestamps.java b/accord-core/src/main/java/accord/coordinate/ExecutePath.java
similarity index 87%
rename from accord-core/src/main/java/accord/impl/DomainTimestamps.java
rename to accord-core/src/main/java/accord/coordinate/ExecutePath.java
index 2d614c8..15bce6b 100644
--- a/accord-core/src/main/java/accord/impl/DomainTimestamps.java
+++ b/accord-core/src/main/java/accord/coordinate/ExecutePath.java
@@ -16,11 +16,9 @@
  * limitations under the License.
  */
 
-package accord.impl;
+package accord.coordinate;
 
-import accord.primitives.Timestamp;
-
-public interface DomainTimestamps
+public enum ExecutePath
 {
-    Timestamp max();
-}
\ No newline at end of file
+    FAST, SLOW, RECOVER
+}
diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java b/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java
new file mode 100644
index 0000000..713dd54
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java
@@ -0,0 +1,127 @@
+/*
+ * 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 accord.coordinate;
+
+import accord.api.Result;
+import accord.coordinate.tracking.AbstractSimpleTracker;
+import accord.coordinate.tracking.QuorumTracker;
+import accord.coordinate.tracking.RequestStatus;
+import accord.local.Node;
+import accord.messages.ApplyThenWaitUntilApplied;
+import accord.messages.Callback;
+import accord.messages.ReadData;
+import accord.messages.ReadData.ReadReply;
+import accord.primitives.Participants;
+import accord.primitives.Seekables;
+import accord.primitives.SyncPoint;
+import accord.primitives.Txn;
+import accord.primitives.Writes;
+import accord.topology.Topologies;
+import accord.utils.async.AsyncResults.SettableResult;
+
+public abstract class ExecuteSyncPoint<S extends Seekables<?, ?>> extends SettableResult<SyncPoint<S>> implements Callback<ReadReply>
+{
+    public static class SyncPointErased extends Throwable {}
+
+    public static class ExecuteAtQuorum<S extends Seekables<?, ?>> extends ExecuteSyncPoint<S>
+    {
+        ExecuteAtQuorum(Node node, Topologies topologies, SyncPoint<S> syncPoint)
+        {
+            super(node, new QuorumTracker(topologies), syncPoint);
+        }
+
+        @Override
+        protected void start()
+        {
+            Txn txn = node.agent().emptyTxn(syncPoint.syncId.kind(), syncPoint.keysOrRanges);
+            Writes writes = txn.execute(syncPoint.syncId, syncPoint.syncId, null);
+            Result result = txn.result(syncPoint.syncId, syncPoint.syncId, null);
+            node.send(tracker.topologies().nodes(), to -> {
+                Seekables<?, ?> notify = to.equals(node.id()) ? null : syncPoint.keysOrRanges;
+                Participants<?> participants = syncPoint.keysOrRanges.toParticipants();
+                return new ApplyThenWaitUntilApplied(to, tracker.topologies(), syncPoint.route(), syncPoint.syncId, txn, syncPoint.waitFor, participants, syncPoint.syncId.epoch(), writes, result, notify);
+            }, this);
+        }
+    }
+
+    final Node node;
+    final AbstractSimpleTracker<?> tracker;
+    final SyncPoint<S> syncPoint;
+
+    ExecuteSyncPoint(Node node, AbstractSimpleTracker<?> tracker, SyncPoint<S> syncPoint)
+    {
+        // TODO (required): this isn't correct, we need to potentially perform a second round if a dependency executes in a future epoch and we have lost ownership of that epoch
+        this.node = node;
+        this.tracker = tracker;
+        this.syncPoint = syncPoint;
+    }
+
+    protected abstract void start();
+
+    @Override
+    public synchronized void onSuccess(Node.Id from, ReadReply reply)
+    {
+        if (isDone()) return;
+
+        if (!reply.isOk())
+        {
+            switch ((ReadData.CommitOrReadNack)reply)
+            {
+                default: throw new AssertionError("Unhandled: " + reply);
+
+                case Insufficient:
+                    CoordinateSyncPoint.sendApply(node, from, syncPoint);
+                    return;
+
+                case Redundant:
+                    tryFailure(new SyncPointErased());
+                    return;
+
+                case Invalid:
+                    tryFailure(new Invalidated(syncPoint.syncId, syncPoint.homeKey));
+                    return;
+            }
+        }
+        else
+        {
+            // TODO (required): we also need to handle ranges not being safe to read
+            if (tracker.recordSuccess(from) == RequestStatus.Success)
+                onSuccess();
+        }
+    }
+
+    protected void onSuccess()
+    {
+        trySuccess(syncPoint);
+    }
+
+    @Override
+    public synchronized void onFailure(Node.Id from, Throwable failure)
+    {
+        if (isDone()) return;
+        if (tracker.recordFailure(from) == RequestStatus.Failed)
+            tryFailure(new Exhausted(syncPoint.syncId, syncPoint.homeKey));
+    }
+
+    @Override
+    public void onCallbackFailure(Node.Id from, Throwable failure)
+    {
+        tryFailure(failure);
+    }
+}
diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java b/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java
index e46a0ff..be2623d 100644
--- a/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java
+++ b/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java
@@ -42,6 +42,7 @@
 import accord.topology.Topologies;
 import org.agrona.collections.IntHashSet;
 
+import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue;
 import static accord.coordinate.ReadCoordinator.Action.Approve;
 import static accord.coordinate.ReadCoordinator.Action.ApprovePartial;
 import static accord.messages.Commit.Kind.StableFastPath;
@@ -49,14 +50,12 @@
 import static accord.messages.Commit.Kind.StableWithTxnAndDeps;
 import static accord.utils.Invariants.illegalState;
 
-public class ExecuteTxn extends ReadCoordinator<ReadReply> implements Execute
+public class ExecuteTxn extends ReadCoordinator<ReadReply>
 {
-    public static final Execute.Factory FACTORY = ExecuteTxn::new;
-
     @SuppressWarnings("unused")
     private static final Logger logger = LoggerFactory.getLogger(ExecuteTxn.class);
 
-    final Path path;
+    final ExecutePath path;
     final Txn txn;
     final Participants<?> readScope;
     final FullRoute<?> route;
@@ -66,7 +65,7 @@
     final BiConsumer<? super Result, Throwable> callback;
     private Data data;
 
-    private ExecuteTxn(Node node, Topologies topologies, Path path, TxnId txnId, Txn txn, FullRoute<?> route, Participants<?> readScope, Timestamp executeAt, Deps stableDeps, BiConsumer<? super Result, Throwable> callback)
+    ExecuteTxn(Node node, Topologies topologies, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Participants<?> readScope, Timestamp executeAt, Deps stableDeps, BiConsumer<? super Result, Throwable> callback)
     {
         // we need to send Stable to the origin epoch as well as the execution epoch
         // TODO (desired): permit slicing Topologies by key (though unnecessary if we eliminate the concept of non-participating home keys)
@@ -148,11 +147,16 @@
         if (failure == null)
         {
             Result result = txn.result(txnId, executeAt, data);
-            Persist.persist(node, allTopologies, route, txnId, txn, executeAt, stableDeps, txn.execute(txnId, executeAt, data), result, callback);
+            CoordinationAdapter.Invoke.persist(adapter(), node, allTopologies, route, txnId, txn, executeAt, stableDeps, txn.execute(txnId, executeAt, data), result, callback);
         }
         else
         {
             callback.accept(null, failure);
         }
     }
+
+    protected CoordinationAdapter<Result> adapter()
+    {
+        return node.coordinationAdapter(txnId, Continue);
+    }
 }
diff --git a/accord-core/src/main/java/accord/coordinate/FetchData.java b/accord-core/src/main/java/accord/coordinate/FetchData.java
index 1ece467..b7d9cae 100644
--- a/accord-core/src/main/java/accord/coordinate/FetchData.java
+++ b/accord-core/src/main/java/accord/coordinate/FetchData.java
@@ -216,7 +216,10 @@
         else
         {
             if (success == ReadCoordinator.Success.Success)
-                Invariants.checkState(isSufficient(merged), "Status %s is not sufficient", merged);
+            {
+                if (!isSufficient(merged))
+                    Invariants.checkState(isSufficient(merged), "Status %s is not sufficient", merged);
+            }
 
             // TODO (expected): should we automatically trigger a new fetch if we find executeAt but did not request enough information? would be more rob ust
             Propagate.propagate(node, txnId, sourceEpoch, success.withQuorum, route(), target, (CheckStatusOkFull) merged, callback);
diff --git a/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java b/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java
new file mode 100644
index 0000000..0793ae7
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java
@@ -0,0 +1,132 @@
+/*
+ * 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 accord.coordinate;
+
+import java.util.Set;
+
+import accord.coordinate.tracking.QuorumTracker;
+import accord.local.Node;
+import accord.messages.Callback;
+import accord.messages.GetMaxConflict;
+import accord.messages.GetMaxConflict.GetMaxConflictOk;
+import accord.primitives.FullRoute;
+import accord.primitives.Seekables;
+import accord.primitives.Timestamp;
+import accord.topology.Topologies;
+import accord.utils.async.AsyncResult;
+import accord.utils.async.AsyncResults;
+
+import static accord.coordinate.tracking.RequestStatus.Failed;
+import static accord.coordinate.tracking.RequestStatus.Success;
+
+/**
+ * Perform initial rounds of PreAccept and Accept until we have reached agreement about when we should execute.
+ * If we are preempted by a recovery coordinator, we abort and let them complete (and notify us about the execution result)
+ *
+ * TODO (desired, testing): dedicated burn test to validate outcomes
+ */
+public class FetchMaxConflict extends AbstractCoordinatePreAccept<Timestamp, GetMaxConflictOk>
+{
+    final QuorumTracker tracker;
+    final Seekables<?, ?> keysOrRanges;
+    Timestamp maxConflict;
+    long executionEpoch;
+
+    private FetchMaxConflict(Node node, FullRoute<?> route, Seekables<?, ?> keysOrRanges, long executionEpoch)
+    {
+        this(node, route, keysOrRanges, executionEpoch, node.topology().withUnsyncedEpochs(route, executionEpoch, executionEpoch));
+    }
+
+    private FetchMaxConflict(Node node, FullRoute<?> route, Seekables<?, ?> keysOrRanges, long executionEpoch, Topologies topologies)
+    {
+        super(node, route, null, topologies);
+        this.keysOrRanges = keysOrRanges;
+        this.maxConflict = Timestamp.NONE;
+        this.executionEpoch = executionEpoch;
+        this.tracker = new QuorumTracker(topologies);
+    }
+
+    public static AsyncResult<Timestamp> fetchMaxConflict(Node node, Seekables<?, ?> keysOrRanges)
+    {
+        long epoch = node.epoch();
+        FullRoute<?> route = node.computeRoute(epoch, keysOrRanges);
+        // TODO (required): need to ensure we permanently fail any bootstrap that is now impossible and mark as stale
+        TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(epoch), null, route.homeKey(), keysOrRanges);
+        if (mismatch != null)
+            return AsyncResults.failure(mismatch);
+        FetchMaxConflict coordinate = new FetchMaxConflict(node, route, keysOrRanges, epoch);
+        coordinate.start();
+        return coordinate;
+    }
+
+
+    @Override
+    Seekables<?, ?> keysOrRanges()
+    {
+        return keysOrRanges;
+    }
+
+    @Override
+    void contact(Set<Node.Id> nodes, Topologies topologies, Callback<GetMaxConflictOk> callback)
+    {
+        node.send(nodes, to -> new GetMaxConflict(to, topologies, route, keysOrRanges, executionEpoch));
+    }
+
+    @Override
+    void onSuccessInternal(Node.Id from, GetMaxConflictOk reply)
+    {
+        maxConflict = Timestamp.max(reply.maxConflict, maxConflict);
+        executionEpoch = Math.max(executionEpoch, reply.latestEpoch);
+
+        if (tracker.recordSuccess(from) == Success)
+            onPreAcceptedOrNewEpoch();
+    }
+
+    @Override
+    boolean onExtraSuccessInternal(Node.Id from, GetMaxConflictOk reply)
+    {
+        maxConflict = Timestamp.max(reply.maxConflict, maxConflict);
+        return true;
+    }
+
+    @Override
+    void onFailureInternal(Node.Id from, Throwable failure)
+    {
+        if (tracker.recordFailure(from) == Failed)
+            tryFailure(failure);
+    }
+
+    @Override
+    void onNewEpochTopologyMismatch(TopologyMismatch mismatch)
+    {
+        tryFailure(mismatch);
+    }
+
+    @Override
+    long executeAtEpoch()
+    {
+        return executionEpoch;
+    }
+
+    @Override
+    void onPreAccepted(Topologies topologies)
+    {
+        setSuccess(maxConflict);
+    }
+}
diff --git a/accord-core/src/main/java/accord/coordinate/Infer.java b/accord-core/src/main/java/accord/coordinate/Infer.java
index 7b1956a..f1b8016 100644
--- a/accord-core/src/main/java/accord/coordinate/Infer.java
+++ b/accord-core/src/main/java/accord/coordinate/Infer.java
@@ -295,7 +295,7 @@
         }
     }
 
-    public static InvalidIfNot invalidIfNotAtLeast(SafeCommandStore safeStore, TxnId txnId, Unseekables<?> query)
+    public static InvalidIfNot invalidIfNot(SafeCommandStore safeStore, TxnId txnId, Unseekables<?> query)
     {
         if (safeStore.commandStore().globalDurability(txnId).compareTo(Majority) >= 0)
         {
diff --git a/accord-core/src/main/java/accord/coordinate/Invalidate.java b/accord-core/src/main/java/accord/coordinate/Invalidate.java
index d71b8fc..1a409b8 100644
--- a/accord-core/src/main/java/accord/coordinate/Invalidate.java
+++ b/accord-core/src/main/java/accord/coordinate/Invalidate.java
@@ -260,7 +260,6 @@
     {
         // TODO (desired): merge with FetchData.InvalidateOnDone
         // TODO (desired): when sending to network, register a callback for when local application of commitInvalidate message ahs been performed, so no need to special-case
-        // TODO (required, consider): pick a reasonable upper bound, so we don't invalidate into an epoch/commandStore that no longer cares about this command
         node.forEachLocal(contextFor(txnId), commitTo, txnId.epoch(), txnId.epoch(), safeStore -> {
             Commands.commitInvalidate(safeStore, safeStore.get(txnId, txnId, commitTo), commitTo);
         }).begin((s, f) -> {
diff --git a/accord-core/src/main/java/accord/coordinate/Persist.java b/accord-core/src/main/java/accord/coordinate/Persist.java
index 47e370a..e0e2c92 100644
--- a/accord-core/src/main/java/accord/coordinate/Persist.java
+++ b/accord-core/src/main/java/accord/coordinate/Persist.java
@@ -20,7 +20,6 @@
 
 import java.util.HashSet;
 import java.util.Set;
-import java.util.function.BiConsumer;
 
 import accord.api.Result;
 import accord.coordinate.tracking.QuorumTracker;
@@ -32,26 +31,17 @@
 import accord.messages.InformDurable;
 import accord.primitives.Deps;
 import accord.primitives.FullRoute;
-import accord.primitives.PartialTxn;
 import accord.primitives.Timestamp;
 import accord.primitives.Txn;
 import accord.primitives.TxnId;
-import accord.primitives.Unseekables;
 import accord.primitives.Writes;
 import accord.topology.Topologies;
 
 import static accord.coordinate.tracking.RequestStatus.Success;
 import static accord.local.Status.Durability.Majority;
-import static accord.messages.Apply.executes;
-import static accord.messages.Apply.participates;
 
 public abstract class Persist implements Callback<ApplyReply>
 {
-    public interface Factory
-    {
-        Persist create(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result);
-    }
-
     protected final Node node;
     protected final TxnId txnId;
     protected final FullRoute<?> route;
@@ -65,36 +55,7 @@
     protected final Set<Id> persistedOn;
     boolean isDone;
 
-    public static void persist(Node node, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
-    {
-        Topologies executes = executes(node, route, executeAt);
-        persist(node, executes, route, txnId, txn, executeAt, stableDeps, writes, result, clientCallback);
-    }
-
-    public static void persist(Node node, Topologies executes, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
-    {
-        Topologies participates = participates(node, route, txnId, executeAt, executes);
-        node.persistFactory().create(node, executes, txnId, route, txn, executeAt, stableDeps, writes, result)
-                             .applyMinimal(participates, executes, writes, result, clientCallback);
-    }
-
-    public static void persistMaximal(Node node, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
-    {
-        Topologies executes = executes(node, route, executeAt);
-        Topologies participates = participates(node, route, txnId, executeAt, executes);
-        node.persistFactory().create(node, participates, txnId, route, txn, executeAt, deps, writes, result)
-                             .applyMaximal(participates, executes, writes, result, null);
-    }
-
-    public static void persistPartialMaximal(Node node, TxnId txnId, Unseekables<?> sendTo, FullRoute<?> route, PartialTxn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
-    {
-        Topologies executes = executes(node, sendTo, executeAt);
-        Topologies participates = participates(node, sendTo, txnId, executeAt, executes);
-        Persist persist = node.persistFactory().create(node, participates, txnId, route, txn, executeAt, deps, writes, result);
-        node.send(participates.nodes(), to -> Apply.applyMaximal(Apply.FACTORY, to, participates, executes, txnId, route, txn, executeAt, deps, writes, result), persist);
-    }
-
-    protected Persist(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
+    protected Persist(Node node, Topologies all, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
     {
         this.node = node;
         this.txnId = txnId;
@@ -104,8 +65,8 @@
         this.stableDeps = stableDeps;
         this.writes = writes;
         this.result = result;
-        this.topologies = topologies;
-        this.tracker = new QuorumTracker(topologies);
+        this.topologies = all;
+        this.tracker = new QuorumTracker(all);
         this.persistedOn = new HashSet<>();
     }
 
@@ -142,20 +103,13 @@
     @Override
     public void onCallbackFailure(Id from, Throwable failure)
     {
+        // TODO (required): handle exception
     }
 
-    public void applyMinimal(Topologies participates, Topologies executes, Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
+    public void start(Apply.Factory factory, Apply.Kind kind, Topologies all, Writes writes, Result result)
     {
-        registerClientCallback(writes, result, clientCallback);
         // applyMinimal is used for transaction execution by the original coordinator so it's important to use
         // Node's Apply factory in case the factory has to do synchronous Apply.
-        node.send(participates.nodes(), to -> Apply.applyMinimal(node.applyFactory(), to, participates, executes, txnId, route, txn, executeAt, stableDeps, writes, result), this);
+        node.send(all.nodes(), to -> factory.create(kind, to, all, txnId, route, txn, executeAt, stableDeps, writes, result), this);
     }
-    public void applyMaximal(Topologies participates, Topologies executes, Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
-    {
-        registerClientCallback(writes, result, clientCallback);
-        node.send(participates.nodes(), to -> Apply.applyMaximal(Apply.FACTORY, to, participates, executes, txnId, route, txn, executeAt, stableDeps, writes, result), this);
-    }
-
-    public abstract void registerClientCallback(Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback);
 }
diff --git a/accord-core/src/main/java/accord/coordinate/PersistTxn.java b/accord-core/src/main/java/accord/coordinate/PersistTxn.java
index c55777a..b2db464 100644
--- a/accord-core/src/main/java/accord/coordinate/PersistTxn.java
+++ b/accord-core/src/main/java/accord/coordinate/PersistTxn.java
@@ -18,8 +18,6 @@
 
 package accord.coordinate;
 
-import java.util.function.BiConsumer;
-
 import accord.api.Result;
 import accord.local.Node;
 import accord.primitives.Deps;
@@ -32,17 +30,8 @@
 
 public class PersistTxn extends Persist
 {
-    public static final Persist.Factory FACTORY = PersistTxn::new;
-
-    private PersistTxn(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+    public PersistTxn(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
     {
         super(node, topologies, txnId, route, txn, executeAt, deps, writes, result);
     }
-
-    @Override
-    public void registerClientCallback(Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
-    {
-        if (clientCallback != null)
-            clientCallback.accept(result, null);
-    }
 }
diff --git a/accord-core/src/main/java/accord/coordinate/Propose.java b/accord-core/src/main/java/accord/coordinate/Propose.java
index 0d27f75..b72408e 100644
--- a/accord-core/src/main/java/accord/coordinate/Propose.java
+++ b/accord-core/src/main/java/accord/coordinate/Propose.java
@@ -62,10 +62,10 @@
     private final Map<Id, AcceptReply> debug = debug() ? new HashMap<>() : null;
     final Timestamp executeAt;
     final QuorumTracker acceptTracker;
-    final BiConsumer<R, Throwable> callback;
+    final BiConsumer<? super R, Throwable> callback;
     private boolean isDone;
 
-    Propose(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Timestamp executeAt, Deps deps, BiConsumer<R, Throwable> callback)
+    Propose(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
     {
         this.node = node;
         this.ballot = ballot;
diff --git a/accord-core/src/main/java/accord/coordinate/ProposeAndExecute.java b/accord-core/src/main/java/accord/coordinate/ProposeAndExecute.java
deleted file mode 100644
index 2e7e464..0000000
--- a/accord-core/src/main/java/accord/coordinate/ProposeAndExecute.java
+++ /dev/null
@@ -1,64 +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 accord.coordinate;
-
-import java.util.function.BiConsumer;
-
-import accord.api.Result;
-import accord.local.Node;
-import accord.primitives.Ballot;
-import accord.primitives.Deps;
-import accord.primitives.FullRoute;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
-import accord.topology.Topologies;
-import accord.utils.Faults;
-
-class ProposeAndExecute extends Propose<Result>
-{
-    ProposeAndExecute(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, Timestamp executeAt, BiConsumer<Result, Throwable> callback)
-    {
-        super(node, topologies, ballot, txnId, txn, route, executeAt, deps, callback);
-    }
-
-    public static void proposeAndExecute(Node node, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route,
-                                         Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
-    {
-        Topologies topologies = node.topology().withUnsyncedEpochs(route, txnId, executeAt);
-        proposeAndExecute(node, topologies, ballot, txnId, txn, route, executeAt, deps, callback);
-    }
-
-    public static void proposeAndExecute(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route,
-                                         Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
-    {
-        ProposeAndExecute propose = new ProposeAndExecute(node, topologies, ballot, txnId, txn, route, deps, executeAt, callback);
-        propose.start();
-    }
-
-    @Override
-    void onAccepted()
-    {
-        Deps deps = this.deps;
-        if (!Faults.TRANSACTION_UNMERGED_DEPS)
-            deps = deps.with(Deps.merge(acceptOks, ok -> ok.deps));
-
-        Stabilise.stabilise(node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback);
-    }
-}
diff --git a/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java b/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java
index 1e8add5..6d3aab2 100644
--- a/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java
+++ b/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java
@@ -18,19 +18,15 @@
 
 package accord.coordinate;
 
-import java.util.Set;
 import java.util.function.BiConsumer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import accord.local.Node;
-import accord.local.Node.Id;
-import accord.messages.Apply;
 import accord.primitives.Ballot;
 import accord.primitives.Deps;
 import accord.primitives.FullRoute;
-import accord.primitives.Ranges;
 import accord.primitives.Seekables;
 import accord.primitives.SyncPoint;
 import accord.primitives.Timestamp;
@@ -39,50 +35,24 @@
 import accord.topology.Topologies;
 import accord.utils.Faults;
 
-import static accord.primitives.Txn.Kind.ExclusiveSyncPoint;
+import static accord.coordinate.CoordinationAdapter.Invoke.stabilise;
 
 public class ProposeSyncPoint<S extends Seekables<?, ?>> extends Propose<SyncPoint<S>>
 {
     @SuppressWarnings("unused")
     private static final Logger logger = LoggerFactory.getLogger(ProposeSyncPoint.class);
+    private final CoordinationAdapter<SyncPoint<S>> adapter;
 
-    // Whether to wait on the dependencies applying globally before returning a result
-    final boolean async;
-    final Set<Id> fastPathNodes;
-
-    final S keysOrRanges;
-
-    ProposeSyncPoint(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, Timestamp executeAt, BiConsumer<SyncPoint<S>, Throwable> callback, boolean async, Set<Id> fastPathNodes, S keysOrRanges)
+    ProposeSyncPoint(CoordinationAdapter<SyncPoint<S>> adapter, Node node, Topologies topologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
     {
         super(node, topologies, ballot, txnId, txn, route, executeAt, deps, callback);
-        this.async = async;
-        this.fastPathNodes = fastPathNodes;
-        this.keysOrRanges = keysOrRanges;
-    }
-
-    public static <S extends Seekables<?, ?>> Propose<SyncPoint<S>> proposeSyncPoint(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, BiConsumer<SyncPoint<S>, Throwable> callback, boolean async, Set<Id> fastPathNodes, S keysOrRanges)
-    {
-        ProposeSyncPoint proposeSyncPoint = new ProposeSyncPoint(node, topologies, ballot, txnId, txn, route, deps, txnId, callback, async, fastPathNodes, keysOrRanges);
-        proposeSyncPoint.start();
-        return proposeSyncPoint;
+        this.adapter = adapter;
     }
 
     @Override
     void onAccepted()
     {
-        Deps deps = this.deps;
-        if (!Faults.SYNCPOINT_UNMERGED_DEPS)
-            deps = deps.with(Deps.merge(acceptOks, ok -> ok.deps));
-
-        if (txnId.kind() == ExclusiveSyncPoint)
-        {
-            Apply.sendMaximal(node, txnId, route, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null));
-            node.configService().reportEpochClosed((Ranges)keysOrRanges, txnId.epoch());
-            callback.accept(new SyncPoint<S>(txnId, deps, keysOrRanges, route, true), null);
-        }
-        else
-        {
-            CoordinateSyncPoint.blockOnDeps(node, txn, txnId, route, keysOrRanges, deps, callback, async);
-        }
+        Deps deps = Faults.SYNCPOINT_UNMERGED_DEPS ? this.deps : this.deps.with(Deps.merge(acceptOks, ok -> ok.deps));
+        stabilise(adapter, node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback);
     }
 }
diff --git a/accord-core/src/main/java/accord/coordinate/ProposeTxn.java b/accord-core/src/main/java/accord/coordinate/ProposeTxn.java
new file mode 100644
index 0000000..075839d
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/ProposeTxn.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 accord.coordinate;
+
+import java.util.function.BiConsumer;
+
+import accord.api.Result;
+import accord.local.Node;
+import accord.primitives.Ballot;
+import accord.primitives.Deps;
+import accord.primitives.FullRoute;
+import accord.primitives.Timestamp;
+import accord.primitives.Txn;
+import accord.primitives.TxnId;
+import accord.topology.Topologies;
+import accord.utils.Faults;
+
+import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue;
+import static accord.coordinate.CoordinationAdapter.Invoke.stabilise;
+
+class ProposeTxn extends Propose<Result>
+{
+    ProposeTxn(Node node, Topologies topologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
+    {
+        super(node, topologies, ballot, txnId, txn, route, executeAt, deps, callback);
+    }
+
+    @Override
+    void onAccepted()
+    {
+        Deps deps = this.deps;
+        if (!Faults.TRANSACTION_UNMERGED_DEPS)
+            deps = deps.with(Deps.merge(acceptOks, ok -> ok.deps));
+
+        stabilise(adapter(), node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback);
+    }
+
+    protected CoordinationAdapter<Result> adapter()
+    {
+        return node.coordinationAdapter(txnId, Continue);
+    }
+}
diff --git a/accord-core/src/main/java/accord/coordinate/Recover.java b/accord-core/src/main/java/accord/coordinate/Recover.java
index e523169..0455dfa 100644
--- a/accord-core/src/main/java/accord/coordinate/Recover.java
+++ b/accord-core/src/main/java/accord/coordinate/Recover.java
@@ -28,6 +28,7 @@
 import java.util.function.Consumer;
 
 import accord.api.Result;
+import accord.coordinate.CoordinationAdapter.Invoke;
 import accord.coordinate.tracking.QuorumTracker;
 import accord.coordinate.tracking.RecoveryTracker;
 import accord.local.Node;
@@ -56,10 +57,13 @@
 import accord.utils.async.AsyncResult;
 import accord.utils.async.AsyncResults;
 
-import static accord.coordinate.Execute.Path.RECOVER;
+import static accord.coordinate.CoordinationAdapter.Factory.Step.InitiateRecovery;
+import static accord.coordinate.CoordinationAdapter.Invoke.execute;
+import static accord.coordinate.CoordinationAdapter.Invoke.persist;
+import static accord.coordinate.CoordinationAdapter.Invoke.stabilise;
+import static accord.coordinate.ExecutePath.RECOVER;
 import static accord.coordinate.Infer.InvalidateAndCallback.locallyInvalidateAndCallback;
 import static accord.coordinate.Propose.Invalidate.proposeInvalidate;
-import static accord.coordinate.ProposeAndExecute.proposeAndExecute;
 import static accord.coordinate.tracking.RequestStatus.Failed;
 import static accord.coordinate.tracking.RequestStatus.Success;
 import static accord.messages.BeginRecovery.RecoverOk.maxAcceptedOrLater;
@@ -68,6 +72,7 @@
 
 // TODO (low priority, cleanup): rename to Recover (verb); rename Recover message to not clash
 // TODO (expected): do not recover transactions that are known to be Stable and waiting to execute.
+// TODO (expected): separate out recovery of sync points from standard transactions
 public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throwable>
 {
     class AwaitCommit extends AsyncResults.SettableResult<Timestamp> implements Callback<WaitOnCommitOk>
@@ -128,6 +133,7 @@
         return result;
     }
 
+    private final CoordinationAdapter<Result> adapter;
     private final Node node;
     private final Ballot ballot;
     private final TxnId txnId;
@@ -147,6 +153,7 @@
         // TODO (required, correctness): we may have to contact all epochs to ensure we spot any future transaction that might not have taken us as dependency?
         //    or we need an exclusive sync point covering us and closing out the old epoch before recovering;
         //    or we need to manage dependencies for ranges we don't own in future epochs; this might be simplest
+        this.adapter = node.coordinationAdapter(txnId, InitiateRecovery);
         this.node = node;
         this.ballot = ballot;
         this.txnId = txnId;
@@ -257,7 +264,7 @@
                 {
                     withCommittedDeps(executeAt, stableDeps -> {
                         // TODO (future development correctness): when writes/result are partially replicated, need to confirm we have quorum of these
-                        Persist.persistMaximal(node, txnId, route, txn, executeAt, stableDeps, acceptOrCommit.writes, acceptOrCommit.result);
+                        persist(adapter, node, tracker.topologies(), route, txnId, txn, executeAt, stableDeps, acceptOrCommit.writes, acceptOrCommit.result, null);
                     });
                     accept(acceptOrCommit.result, null);
                     return;
@@ -267,7 +274,7 @@
                 case Stable:
                 {
                     withCommittedDeps(executeAt, stableDeps -> {
-                        Execute.execute(node, tracker.topologies(), route, RECOVER, txnId, txn, executeAt, stableDeps, this);
+                        execute(adapter, node, tracker.topologies(), route, RECOVER, txnId, txn, executeAt, stableDeps, this);
                     });
                     return;
                 }
@@ -276,7 +283,7 @@
                 case Committed:
                 {
                     withCommittedDeps(executeAt, committedDeps -> {
-                        Stabilise.stabilise(node, tracker.topologies(), route, ballot, txnId, txn, executeAt, committedDeps, this);
+                        stabilise(adapter, node, tracker.topologies(), route, ballot, txnId, txn, executeAt, committedDeps, this);
                     });
                     return;
                 }
@@ -320,7 +327,6 @@
             // we have to be certain these commands have not successfully committed without witnessing us (thereby
             // ruling out a fast path decision for us and changing our recovery decision).
             // So, we wait for these commands to finish committing before retrying recovery.
-            // TODO (required): check paper: do we assume that witnessing in PreAccept implies witnessing in Accept? Not guaranteed.
             // See whitepaper for more details
             awaitCommits(node, earlierAcceptedNoWitness).addCallback((success, failure) -> {
                 if (failure != null) accept(null, failure);
@@ -373,7 +379,7 @@
 
     private void propose(Timestamp executeAt, Deps deps)
     {
-        node.withEpoch(executeAt.epoch(), () -> proposeAndExecute(node, ballot, txnId, txn, route, executeAt, deps, this));
+        node.withEpoch(executeAt.epoch(), () -> Invoke.propose(adapter, node, route, ballot, txnId, txn, executeAt, deps, this));
     }
 
     private void retry()
diff --git a/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java b/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java
index 257893a..6987163 100644
--- a/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java
+++ b/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java
@@ -42,6 +42,8 @@
 import accord.utils.Invariants;
 import javax.annotation.Nullable;
 
+import static accord.coordinate.CoordinationAdapter.Factory.Step.InitiateRecovery;
+import static accord.coordinate.CoordinationAdapter.Invoke.persist;
 import static accord.local.Status.Durability.Majority;
 import static accord.local.Status.KnownDeps.DepsKnown;
 import static accord.local.Status.KnownExecuteAt.ExecuteAtKnown;
@@ -186,7 +188,7 @@
                             {
                                 Invariants.checkState(full.stableDeps.covering.containsAll(sendTo));
                                 Invariants.checkState(full.partialTxn.covering().containsAll(sendTo));
-                                Persist.persistPartialMaximal(node, txnId, sendTo, route, full.partialTxn, full.executeAt, full.stableDeps, full.writes, full.result);
+                                persist(node.coordinationAdapter(txnId, InitiateRecovery), node, route, sendTo, txnId, full.partialTxn, full.executeAt, full.stableDeps, full.writes, full.result, null);
                             }
                             propagate = full;
                         }
@@ -205,13 +207,12 @@
                     break;
                 }
 
-                // TODO (required): might not be able to fully recover transaction - may only have enough for local shard
                 Txn txn = full.partialTxn.reconstitute(route);
                 if (known.executeAt.isDecidedAndKnownToExecute() && known.deps.hasDecidedDeps() && known.outcome == Apply)
                 {
                     Deps deps = full.stableDeps.reconstitute(route());
                     node.withEpoch(full.executeAt.epoch(), () -> {
-                        Persist.persistMaximal(node, txnId, route(), txn, full.executeAt, deps, full.writes, full.result);
+                        persist(node.coordinationAdapter(txnId, InitiateRecovery), node, topologies, route(), txnId, txn, full.executeAt, deps, full.writes, full.result, null);
                     });
                     callback.accept(APPLIED, null);
                 }
diff --git a/accord-core/src/main/java/accord/coordinate/Stabilise.java b/accord-core/src/main/java/accord/coordinate/Stabilise.java
index e48ee7d..d204fab 100644
--- a/accord-core/src/main/java/accord/coordinate/Stabilise.java
+++ b/accord-core/src/main/java/accord/coordinate/Stabilise.java
@@ -22,7 +22,6 @@
 import java.util.Map;
 import java.util.function.BiConsumer;
 
-import accord.api.Result;
 import accord.coordinate.tracking.QuorumTracker;
 import accord.coordinate.tracking.RequestStatus;
 import accord.local.Node;
@@ -38,14 +37,14 @@
 import accord.primitives.Txn;
 import accord.primitives.TxnId;
 import accord.topology.Topologies;
-import accord.utils.Faults;
 
-import static accord.coordinate.Execute.Path.SLOW;
+import static accord.coordinate.CoordinationAdapter.Invoke.execute;
+import static accord.coordinate.ExecutePath.SLOW;
 import static accord.coordinate.tracking.RequestStatus.Failed;
 import static accord.messages.Commit.Kind.CommitWithTxn;
 import static accord.utils.Invariants.debug;
 
-public abstract class Stabilise implements Callback<ReadReply>
+public abstract class Stabilise<R> implements Callback<ReadReply>
 {
     final Node node;
     final Txn txn;
@@ -58,10 +57,10 @@
     private final Map<Node.Id, Object> debug = debug() ? new HashMap<>() : null;
     final QuorumTracker stableTracker;
     final Topologies allTopologies;
-    final BiConsumer<? super Result, Throwable> callback;
+    final BiConsumer<? super R, Throwable> callback;
     private boolean isDone;
 
-    public Stabilise(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, TxnId txnId, Ballot ballot, Txn txn, Timestamp executeAt, Deps stabiliseDeps, BiConsumer<? super Result, Throwable> callback)
+    public Stabilise(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, TxnId txnId, Ballot ballot, Txn txn, Timestamp executeAt, Deps stabiliseDeps, BiConsumer<? super R, Throwable> callback)
     {
         this.node = node;
         this.txn = txn;
@@ -76,40 +75,6 @@
         this.callback = callback;
     }
 
-    static void stabilise(Node node, Topologies anyTopologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
-    {
-        Topologies coordinates = anyTopologies.forEpochs(txnId.epoch(), txnId.epoch());
-        Topologies allTopologies;
-        if (txnId.epoch() == executeAt.epoch()) allTopologies = coordinates;
-        else if (anyTopologies.currentEpoch() >= executeAt.epoch() && anyTopologies.oldestEpoch() <= txnId.epoch()) allTopologies = anyTopologies.forEpochs(txnId.epoch(), executeAt.epoch());
-        else allTopologies = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
-
-        stabilise(node, coordinates, allTopologies, route, ballot, txnId, txn, executeAt, deps, callback);
-    }
-
-    public static void stabilise(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
-    {
-        switch (txnId.kind())
-        {
-            default:
-                throw new AssertionError("Unhandled Txn.Kind: " + txnId.kind());
-            case LocalOnly:
-                throw new AssertionError("Invalid Txn.Kind to stabilise: " + txnId.kind());
-            case SyncPoint:
-            case ExclusiveSyncPoint:
-                // TODO (expected): merge with branch below, as identical besides fault condition
-                if (Faults.SYNCPOINT_INSTABILITY) Execute.execute(node, allTopologies, route, SLOW, txnId, txn, executeAt, deps, callback);
-                else new StabiliseTxn(node, coordinates, allTopologies, route, txnId, ballot, txn, executeAt, deps, callback).start();
-                break;
-            case NoOp:
-            case Read:
-            case Write:
-                if (Faults.TRANSACTION_INSTABILITY) Execute.execute(node, allTopologies, route, SLOW, txnId, txn, executeAt, deps, callback);
-                else new StabiliseTxn(node, coordinates, allTopologies, route,
-                                      txnId, ballot, txn, executeAt, deps, callback).start();
-        }
-    }
-
     void start()
     {
         Commit.commitMinimal(node, stableTracker.topologies(), ballot, txnId, txn, route, executeAt, stabiliseDeps, this);
@@ -175,5 +140,10 @@
         callback.accept(null, failure);
     }
 
-    abstract void onStabilised();
+    protected void onStabilised()
+    {
+        execute(adapter(), node, allTopologies, route, SLOW, txnId, txn, executeAt, stabiliseDeps, callback);
+    }
+
+    protected abstract CoordinationAdapter<R> adapter();
 }
diff --git a/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java b/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java
new file mode 100644
index 0000000..e4936d2
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java
@@ -0,0 +1,48 @@
+/*
+ * 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 accord.coordinate;
+
+import java.util.function.BiConsumer;
+
+import accord.local.Node;
+import accord.primitives.Ballot;
+import accord.primitives.Deps;
+import accord.primitives.FullRoute;
+import accord.primitives.Seekables;
+import accord.primitives.SyncPoint;
+import accord.primitives.Timestamp;
+import accord.primitives.Txn;
+import accord.primitives.TxnId;
+import accord.topology.Topologies;
+
+public class StabiliseSyncPoint<S extends Seekables<?, ?>> extends Stabilise<SyncPoint<S>>
+{
+    final CoordinationAdapter<SyncPoint<S>> adapter;
+    StabiliseSyncPoint(CoordinationAdapter<SyncPoint<S>> adapter, Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+    {
+        super(node, coordinates, allTopologies, route, txnId, ballot, txn, executeAt, unstableDeps, callback);
+        this.adapter = adapter;
+    }
+
+    @Override
+    protected CoordinationAdapter<SyncPoint<S>> adapter()
+    {
+        return adapter;
+    }
+}
diff --git a/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java b/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java
index edbfd1b..da36f65 100644
--- a/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java
+++ b/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java
@@ -30,18 +30,18 @@
 import accord.primitives.TxnId;
 import accord.topology.Topologies;
 
-import static accord.coordinate.Execute.Path.SLOW;
+import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue;
 
-public class StabiliseTxn extends Stabilise
+public class StabiliseTxn extends Stabilise<Result>
 {
-    public StabiliseTxn(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, TxnId txnId, Ballot ballot, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer<? super Result, Throwable> callback)
+    StabiliseTxn(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer<? super Result, Throwable> callback)
     {
         super(node, coordinates, allTopologies, route, txnId, ballot, txn, executeAt, unstableDeps, callback);
     }
 
     @Override
-    void onStabilised()
+    protected CoordinationAdapter<Result> adapter()
     {
-        Execute.execute(node, allTopologies, route, SLOW, txnId, txn, executeAt, stabiliseDeps, callback);
+        return node.coordinationAdapter(txnId, Continue);
     }
 }
diff --git a/accord-core/src/main/java/accord/coordinate/TopologyMismatch.java b/accord-core/src/main/java/accord/coordinate/TopologyMismatch.java
index 11b9aa4..f8791f8 100644
--- a/accord-core/src/main/java/accord/coordinate/TopologyMismatch.java
+++ b/accord-core/src/main/java/accord/coordinate/TopologyMismatch.java
@@ -34,7 +34,7 @@
 
     private final EnumSet<Reason> reasons;
 
-    private TopologyMismatch(EnumSet<Reason> reasons, Topology topology, TxnId txnId, @Nullable RoutingKey homeKey, Routables<?> keysOrRanges)
+    private TopologyMismatch(EnumSet<Reason> reasons, Topology topology, @Nullable TxnId txnId, @Nullable RoutingKey homeKey, Routables<?> keysOrRanges)
     {
         super(txnId, homeKey, buildMessage(reasons, topology, homeKey, keysOrRanges));
         this.reasons = reasons;
@@ -72,7 +72,7 @@
     }
 
     @Nullable
-    public static TopologyMismatch checkForMismatch(Topology t, TxnId txnId, RoutingKey homeKey, Routables<?> keysOrRanges)
+    public static TopologyMismatch checkForMismatch(Topology t, @Nullable TxnId txnId, RoutingKey homeKey, Routables<?> keysOrRanges)
     {
         EnumSet<TopologyMismatch.Reason> reasons = null;
         if (!t.ranges().contains(homeKey))
diff --git a/accord-core/src/main/java/accord/coordinate/tracking/AbstractSimpleTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/AbstractSimpleTracker.java
new file mode 100644
index 0000000..845f334
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/tracking/AbstractSimpleTracker.java
@@ -0,0 +1,42 @@
+/*
+ * 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 accord.coordinate.tracking;
+
+import java.util.function.Function;
+import java.util.function.IntFunction;
+
+import accord.local.Node.Id;
+import accord.topology.Shard;
+import accord.topology.Topologies;
+
+public abstract class AbstractSimpleTracker<ST extends ShardTracker> extends AbstractTracker<ST>
+{
+    AbstractSimpleTracker(Topologies topologies, IntFunction<ST[]> arrayFactory, Function<Shard, ST> trackerFactory)
+    {
+        super(topologies, arrayFactory, trackerFactory);
+    }
+
+    AbstractSimpleTracker(Topologies topologies, IntFunction<ST[]> arrayFactory, ShardFactory<ST> trackerFactory)
+    {
+        super(topologies, arrayFactory, trackerFactory);
+    }
+
+    public abstract RequestStatus recordSuccess(Id from);
+    public abstract RequestStatus recordFailure(Id from);
+}
diff --git a/accord-core/src/main/java/accord/coordinate/tracking/AppliedTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/AppliedTracker.java
index da49dc0..155e4f8 100644
--- a/accord-core/src/main/java/accord/coordinate/tracking/AppliedTracker.java
+++ b/accord-core/src/main/java/accord/coordinate/tracking/AppliedTracker.java
@@ -26,7 +26,7 @@
 import static accord.coordinate.tracking.AbstractTracker.ShardOutcomes.NoChange;
 import static accord.coordinate.tracking.AbstractTracker.ShardOutcomes.Success;
 
-public class AppliedTracker extends AbstractTracker<AppliedTracker.AppliedShardTracker> implements ResponseTracker
+public class AppliedTracker extends AbstractSimpleTracker<AppliedTracker.AppliedShardTracker> implements ResponseTracker
 {
     public static class AppliedShardTracker extends ShardTracker
     {
diff --git a/accord-core/src/main/java/accord/coordinate/tracking/QuorumTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/QuorumTracker.java
index c771406..70bc352 100644
--- a/accord-core/src/main/java/accord/coordinate/tracking/QuorumTracker.java
+++ b/accord-core/src/main/java/accord/coordinate/tracking/QuorumTracker.java
@@ -24,7 +24,7 @@
 
 import static accord.coordinate.tracking.AbstractTracker.ShardOutcomes.*;
 
-public class QuorumTracker extends AbstractTracker<QuorumTracker.QuorumShardTracker> implements ResponseTracker
+public class QuorumTracker extends AbstractSimpleTracker<QuorumTracker.QuorumShardTracker> implements ResponseTracker
 {
     public static class QuorumShardTracker extends ShardTracker
     {
diff --git a/accord-core/src/main/java/accord/impl/AbstractConfigurationService.java b/accord-core/src/main/java/accord/impl/AbstractConfigurationService.java
index d33c123..2f93a99 100644
--- a/accord-core/src/main/java/accord/impl/AbstractConfigurationService.java
+++ b/accord-core/src/main/java/accord/impl/AbstractConfigurationService.java
@@ -269,8 +269,6 @@
         }
 
         long lastAcked = epochs.lastAcknowledged;
-        // TODO (now, review): lastAcked == 0, lastReceived = 2
-        // if we wait for epoch=1.acknowledge the test seems to wait forever... looks like burn test doesn't ack epoch=1
         if (lastAcked == 0 && lastReceived > 0)
         {
             epochs.acknowledgeFuture(epochs.minEpoch()).addCallback(() -> reportTopology(topology, startSync));
diff --git a/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java b/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java
index a3b3d5e..23de1e2 100644
--- a/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java
+++ b/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java
@@ -122,9 +122,9 @@
 
     protected abstract void onReadOk(Node.Id from, CommandStore commandStore, Data data, Ranges ranges);
 
-    protected boolean collectMaxApplied()
+    protected FetchRequest newFetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn)
     {
-        return false;
+        return new FetchRequest(sourceEpoch, syncId, ranges, partialDeps, rangeReadTxn(ranges));
     }
 
     @Override
@@ -135,7 +135,7 @@
         Ranges ownedRanges = ownedRangesForNode(to);
         Invariants.checkArgument(ownedRanges.containsAll(ranges), "Got a reply from %s for ranges %s, but owned ranges %s does not contain all the ranges", to, ranges, ownedRanges);
         PartialDeps partialDeps = syncPoint.waitFor.slice(ownedRanges, ranges);
-        node.send(to, new FetchRequest(syncPoint.sourceEpoch(), syncPoint.syncId, ranges, partialDeps, rangeReadTxn(ranges), collectMaxApplied()), new Callback<ReadReply>()
+        node.send(to, newFetchRequest(syncPoint.sourceEpoch(), syncPoint.syncId, ranges, partialDeps, rangeReadTxn(ranges)), new Callback<ReadReply>()
         {
             @Override
             public void onSuccess(Node.Id from, ReadReply reply)
@@ -228,50 +228,24 @@
 
     void abort(Ranges abort)
     {
-        // TODO (required, later): implement abort
+        // TODO (expected): implement abort
     }
 
     public static class FetchRequest extends WaitUntilAppliedAndReadData
     {
         public final PartialDeps partialDeps;
-        public final boolean collectMaxApplied;
-        private transient Timestamp maxApplied;
 
-        public FetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn, boolean collectMaxApplied)
+        public FetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn)
         {
             super(syncId, ranges, sourceEpoch, partialTxn);
             this.partialDeps = partialDeps;
-            this.collectMaxApplied = collectMaxApplied;
         }
 
         @Override
         protected void readComplete(CommandStore commandStore, Data result, Ranges unavailable)
         {
-            Ranges slice = commandStore.unsafeRangesForEpoch().allAt(txnId).subtract(unavailable);
-            if (collectMaxApplied)
-            {
-                commandStore.maxAppliedFor((Ranges)readScope, slice).begin((newMaxApplied, failure) -> {
-                    if (failure != null)
-                    {
-                        commandStore.agent().onUncaughtException(failure);
-                    }
-                    else
-                    {
-                        synchronized (this)
-                        {
-                            if (maxApplied == null) maxApplied = newMaxApplied;
-                            else maxApplied = Timestamp.max(maxApplied, newMaxApplied);
-                            Ranges reportUnavailable = unavailable.slice((Ranges)this.readScope, Minimal);
-                            super.readComplete(commandStore, result, reportUnavailable);
-                        }
-                    }
-                });
-            }
-            else
-            {
-                Ranges reportUnavailable = unavailable.slice((Ranges)this.readScope, Minimal);
-                super.readComplete(commandStore, result, reportUnavailable);
-            }
+            Ranges reportUnavailable = unavailable.slice((Ranges)this.readScope, Minimal);
+            super.readComplete(commandStore, result, reportUnavailable);
         }
 
         @Override
@@ -279,7 +253,12 @@
         {
             // TODO (review): If the fetch response actually does some streaming, but we send back the error
             // it is a lot of work and data that might move and be unaccounted for at the coordinator
-            node.reply(replyTo, replyContext, fail == null ? new FetchResponse(unavailable, data, maxApplied) : null, fail);
+            node.reply(replyTo, replyContext, fail == null ? new FetchResponse(unavailable, data, maxApplied()) : null, fail);
+        }
+
+        protected Timestamp maxApplied()
+        {
+            return null;
         }
 
         @Override
diff --git a/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java b/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java
index 89058bf..0de56e7 100644
--- a/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java
+++ b/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java
@@ -18,16 +18,14 @@
 
 package accord.impl;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 import java.util.function.Consumer;
 import java.util.function.Function;
 
-import accord.api.Key;
+import javax.annotation.Nullable;
+
+import com.google.common.annotations.VisibleForTesting;
+
 import accord.api.VisibleForImplementation;
-import accord.impl.CommandTimeseries.CommandLoader;
 import accord.local.*;
 import accord.primitives.*;
 import accord.utils.Invariants;
@@ -37,27 +35,10 @@
 
 public abstract class AbstractSafeCommandStore<CommandType extends SafeCommand,
                                                TimestampsForKeyType extends SafeTimestampsForKey,
-                                               CommandsForKeyType extends SafeCommandsForKey,
-                                               CommandsForKeyUpdateType extends SafeCommandsForKey.Update> extends SafeCommandStore
+                                               CommandsForKeyType extends SafeCommandsForKey> extends SafeCommandStore
 {
-    private static class PendingRegistration<T>
-    {
-        final T value;
-        final Ranges slice;
-        final TxnId txnId;
-
-        public PendingRegistration(T value, Ranges slice, TxnId txnId)
-        {
-            this.value = value;
-            this.slice = slice;
-            this.txnId = txnId;
-        }
-    }
     protected final PreLoadContext context;
 
-    private List<PendingRegistration<Seekable>> pendingSeekableRegistrations = null;
-    private List<PendingRegistration<Seekables<?, ?>>> pendingSeekablesRegistrations = null;
-
     public AbstractSafeCommandStore(PreLoadContext context)
     {
         this.context = context;
@@ -84,18 +65,9 @@
     protected abstract void addTimestampsForKeyInternal(TimestampsForKeyType cfk);
     protected abstract TimestampsForKeyType getTimestampsForKeyIfLoaded(RoutableKey key);
 
-    protected abstract CommandLoader<?> cfkLoader(RoutableKey key);
-    protected abstract CommandsForKeyType getDepsCommandsForKeyInternal(RoutableKey key);
-    protected abstract void addDepsCommandsForKeyInternal(CommandsForKeyType cfk);
-    protected abstract CommandsForKeyType getDepsCommandsForKeyIfLoaded(RoutableKey key);
-
-    protected abstract CommandsForKeyType getAllCommandsForKeyInternal(RoutableKey key);
-    protected abstract void addAllCommandsForKeyInternal(CommandsForKeyType cfk);
-    protected abstract CommandsForKeyType getAllCommandsForKeyIfLoaded(RoutableKey key);
-
-    protected abstract CommandsForKeyUpdateType getCommandsForKeyUpdateInternal(RoutableKey key);
-    protected abstract CommandsForKeyUpdateType createCommandsForKeyUpdateInternal(RoutableKey key);
-    protected abstract void addCommandsForKeyUpdateInternal(CommandsForKeyUpdateType update);
+    protected abstract CommandsForKeyType getCommandsForKeyInternal(RoutableKey key);
+    protected abstract void addCommandsForKeyInternal(CommandsForKeyType cfk);
+    protected abstract CommandsForKeyType getCommandsForKeyIfLoaded(RoutableKey key);
 
     @Override
     protected CommandType getInternalIfLoadedAndInitialised(TxnId txnId)
@@ -117,87 +89,44 @@
         return command;
     }
 
-    private CommandsForKeyType getCommandsIfLoaded(RoutableKey key, KeyHistory keyHistory)
+    private CommandsForKeyType getCommandsIfLoaded(RoutableKey key)
     {
-        switch (keyHistory)
-        {
-            case DEPS:
-                return getIfLoaded(key, this::getDepsCommandsForKeyInternal, this::addDepsCommandsForKeyInternal, this::getDepsCommandsForKeyIfLoaded);
-            case ALL:
-                return getIfLoaded(key, this::getAllCommandsForKeyInternal, this::addAllCommandsForKeyInternal, this::getAllCommandsForKeyIfLoaded);
-            default:
-                throw new IllegalArgumentException("CommandsForKey not available for " + keyHistory);
-        }
+        return getIfLoaded(key, this::getCommandsForKeyInternal, this::addCommandsForKeyInternal, this::getCommandsForKeyIfLoaded);
     }
 
-    private CommandsForKeyType commandsIfLoadedAndInitialised(RoutableKey key, KeyHistory keyHistory)
+    CommandsForKeyType commandsIfLoadedAndInitialised(RoutableKey key)
     {
-        CommandsForKeyType cfk = getCommandsIfLoaded(key, keyHistory);;
+        CommandsForKeyType cfk = getCommandsIfLoaded(key);
         if (cfk == null)
             return null;
         if (cfk.isEmpty())
-        {
-            cfk.initialize(cfkLoader(key));
-        }
-        else
-        {
-            RedundantBefore.Entry entry = commandStore().redundantBefore().get(key.toUnseekable());
-            if (entry != null && cfk.current().hasRedundant(entry.shardRedundantBefore()))
-                cfk.set(cfk.current().withoutRedundant(entry.shardRedundantBefore()));
-        }
+            cfk.initialize();
+
+        RedundantBefore.Entry entry = commandStore().redundantBefore().get(key.toUnseekable());
+        if (entry != null)
+            cfk.updateRedundantBefore(entry.shardRedundantBefore());
         return cfk;
     }
 
-    protected CommandsForKeyType commandsForKey(RoutableKey key, KeyHistory keyHistory)
+    @VisibleForTesting
+    public CommandsForKeyType commandsForKey(RoutableKey key)
     {
-        CommandsForKeyType cfk = getCommandsIfLoaded(key, keyHistory);
+        CommandsForKeyType cfk = getCommandsIfLoaded(key);
         Invariants.checkState(cfk != null, "%s was not specified in PreLoadContext", key);
         if (cfk.isEmpty())
-            cfk.initialize(cfkLoader(key));
+            cfk.initialize();
         return cfk;
     }
 
     @VisibleForImplementation
-    private CommandsForKeyType maybeCommandsForKey(RoutableKey key, KeyHistory keyHistory)
+    public CommandsForKeyType maybeCommandsForKey(RoutableKey key)
     {
-        CommandsForKeyType cfk = getCommandsIfLoaded(key, keyHistory);
+        CommandsForKeyType cfk = getCommandsIfLoaded(key);
         if (cfk == null || cfk.isEmpty())
             return null;
         return cfk;
     }
 
-    public CommandsForKeyType depsCommandsIfLoadedAndInitialised(RoutableKey key)
-    {
-        return commandsIfLoadedAndInitialised(key, KeyHistory.DEPS);
-    }
-
-    public CommandsForKeyType depsCommandsForKey(RoutableKey key)
-    {
-        return commandsForKey(key, KeyHistory.DEPS);
-    }
-
-    @VisibleForImplementation
-    public CommandsForKeyType maybeDepsCommandsForKey(RoutableKey key)
-    {
-        return maybeCommandsForKey(key, KeyHistory.DEPS);
-    }
-
-    public CommandsForKeyType allCommandsIfLoadedAndInitialised(RoutableKey key)
-    {
-        return commandsIfLoadedAndInitialised(key, KeyHistory.ALL);
-    }
-
-    public CommandsForKeyType allCommandsForKey(RoutableKey key)
-    {
-        return commandsForKey(key, KeyHistory.ALL);
-    }
-
-    @VisibleForImplementation
-    public CommandsForKeyType maybeAllCommandsForKey(RoutableKey key)
-    {
-        return maybeCommandsForKey(key, KeyHistory.ALL);
-    }
-
     public TimestampsForKeyType timestampsIfLoadedAndInitialised(RoutableKey key)
     {
         TimestampsForKeyType cfk = getIfLoaded(key, this::getTimestampsForKeyInternal, this::addTimestampsForKeyInternal, this::getTimestampsForKeyIfLoaded);
@@ -229,108 +158,59 @@
         return tfk;
     }
 
-    public CommandsForKeyUpdateType getOrCreateCommandsForKeyUpdate(RoutableKey key)
-    {
-        CommandsForKeyUpdateType update = getIfLoaded(key, this::getCommandsForKeyUpdateInternal, this::addCommandsForKeyUpdateInternal, this::createCommandsForKeyUpdateInternal);
-        if (update == null)
-        {
-            update = createCommandsForKeyUpdateInternal(key);
-            addCommandsForKeyUpdateInternal(update);
-        }
-
-        if (update.isEmpty())
-            update.initialize();
-
-        return update;
-    }
-
     @Override
-    public void removeCommandFromSeekableDeps(Seekable seekable, TxnId txnId, Timestamp executeAt, Status status)
+    protected void update(Command prev, Command updated, @Nullable Seekables<?, ?> keysOrRanges)
     {
-        // the cfk listener doesn't know if it can remove the given command from the deps set without loading
-        // the deps set, so we don't actually remove it until it becomes applied
-        if (!status.hasBeen(Status.Applied))
+        super.update(prev, updated, keysOrRanges);
+
+        if (!CommandsForKey.needsUpdate(prev, updated))
             return;
 
-        switch (seekable.domain())
-        {
-            case Key:
-                Key key = seekable.asKey();
-                CommandsForKeyUpdater.Mutable<?> updater = getOrCreateCommandsForKeyUpdate(key).deps();
-                updater.commands().remove(txnId);
-                break;
-            case Range:
-                break;
-            default:
-                throw new IllegalArgumentException();
-        }
+        TxnId txnId = updated.txnId();
+        if (!txnId.kind().isGloballyVisible() || !txnId.domain().isKey())
+            return;
+
+        // TODO (required): consider carefully epoch overlaps for dependencies;
+        //      here we're limiting our registration with CFK to the coordination epoch only
+        //      if we permit coordination+execution we have to do a very careful dance (or relax validation)
+        //      because for some keys we can expect e.g. PreAccept and Accept states to have been processed
+        //      and for other keys Committed onwards will appear suddenly (or, if we permit Accept to process
+        //      on its executeAt ranges, it could go either way).
+        Ranges ranges = ranges().allAt(txnId);
+        Keys keys;
+        if (keysOrRanges != null) keys = (Keys) keysOrRanges;
+        else if (updated.known().isDefinitionKnown()) keys = (Keys)updated.partialTxn().keys();
+        else if (prev.known().isDefinitionKnown()) keys = (Keys)prev.partialTxn().keys();
+        else if (updated.saveStatus().hasBeen(Status.Truncated)) return; // TODO (required): we may have transaction registered via Accept, and still want to expunge. we shouldn't special case: should ensure we have everything loaded, or permit asynchronous application
+        else if (updated.saveStatus().is(Status.AcceptedInvalidate)) return; // TODO (required): we may have transaction registered via Accept, and still want to expunge. we shouldn't special case: should ensure we have everything loaded, or permit asynchronous application
+        else throw illegalState("No keys to update CommandsForKey with");
+
+        Routables.foldl(keys, ranges, (self, p, key, u, i) -> {
+            SafeCommandsForKey cfk = self.commandsIfLoadedAndInitialised(key);
+            // TODO (required): we shouldn't special case invalidations, truncations or topology changes: should ensure we have everything loaded, or permit asynchronous application
+            Invariants.checkState(cfk != null || u.saveStatus().hasBeen(Status.Truncated) || u.saveStatus().is(Status.AcceptedInvalidate));
+            if (cfk != null)
+                cfk.update(p, u);
+            return u;
+        }, this, prev, updated, i->false);
     }
 
     @Override
     public boolean canExecuteWith(PreLoadContext context)
     {
+        // TODO (required): check if data is in cache, and if so simply add it to our context
         return context.isSubsetOf(this.context);
     }
 
-    @Override
-    public void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command)
-    {
-        if (pendingSeekablesRegistrations == null)
-            pendingSeekablesRegistrations = new ArrayList<>();
-        pendingSeekablesRegistrations.add(new PendingRegistration<>(keysOrRanges, slice, command.txnId()));
-    }
-
-    @Override
-    public void register(Seekable keyOrRange, Ranges slice, Command command)
-    {
-        if (pendingSeekableRegistrations == null)
-            pendingSeekableRegistrations = new ArrayList<>();
-        pendingSeekableRegistrations.add(new PendingRegistration<>(keyOrRange, slice, command.txnId()));
-    }
-
-    public abstract CommonAttributes completeRegistration(Seekables<?, ?> keysOrRanges, Ranges slice, CommandType command, CommonAttributes attrs);
-
-    public abstract CommonAttributes completeRegistration(Seekable keyOrRange, Ranges slice, CommandType command, CommonAttributes attrs);
-
-    private interface RegistrationCompleter<T, CommandType extends SafeCommand>
-    {
-        CommonAttributes complete(T value, Ranges ranges, CommandType command, CommonAttributes attrs);
-    }
-
-    private <T> void completeRegistrations(Map<TxnId, CommonAttributes> updates, List<PendingRegistration<T>> pendingRegistrations, RegistrationCompleter<T, CommandType> completer)
-    {
-        if (pendingRegistrations == null)
-            return;
-
-        for (PendingRegistration<T> pendingRegistration : pendingRegistrations)
-        {
-            TxnId txnId = pendingRegistration.txnId;
-            CommandType safeCommand = getInternal(pendingRegistration.txnId);
-            Command command = safeCommand.current();
-            CommonAttributes attrs = updates.getOrDefault(txnId, command);
-            attrs = completer.complete(pendingRegistration.value, pendingRegistration.slice, safeCommand, attrs);
-            if (attrs != command)
-                updates.put(txnId, attrs);
-        }
-    }
-
     protected abstract void invalidateSafeState();
-    protected abstract void applyCommandForKeyUpdates();
 
     public void postExecute()
     {
-        if (pendingSeekableRegistrations != null || pendingSeekablesRegistrations != null)
-        {
-            Map<TxnId, CommonAttributes> attributeUpdates = new HashMap<>();
-            completeRegistrations(attributeUpdates, pendingSeekablesRegistrations, this::completeRegistration);
-            completeRegistrations(attributeUpdates, pendingSeekableRegistrations, this::completeRegistration);
-            attributeUpdates.forEach(((txnId, attributes) -> get(txnId).updateAttributes(attributes)));
-        }
-        applyCommandForKeyUpdates();
     }
 
     public void complete()
     {
+        postExecute();
         invalidateSafeState();
     }
 }
diff --git a/accord-core/src/main/java/accord/impl/CommandTimeseries.java b/accord-core/src/main/java/accord/impl/CommandTimeseries.java
deleted file mode 100644
index 0c0c727..0000000
--- a/accord-core/src/main/java/accord/impl/CommandTimeseries.java
+++ /dev/null
@@ -1,537 +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 accord.impl;
-
-import java.util.*;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-import java.util.stream.Stream;
-import javax.annotation.Nullable;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.ImmutableSortedMap;
-
-import accord.api.Key;
-import accord.local.Command;
-import accord.local.SafeCommandStore;
-import accord.local.SaveStatus;
-import accord.local.Status;
-import accord.primitives.Seekable;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn.Kind.Kinds;
-import accord.primitives.TxnId;
-
-import static accord.local.SafeCommandStore.TestDep.ANY_DEPS;
-import static accord.local.SafeCommandStore.TestDep.WITH;
-import static accord.utils.Utils.ensureSortedImmutable;
-import static accord.utils.Utils.ensureSortedMutable;
-
-public class CommandTimeseries<D>
-{
-    public enum TestTimestamp { BEFORE, AFTER }
-    public enum TimestampType { TXN_ID, EXECUTE_AT }
-
-    private final Seekable keyOrRange;
-    protected final CommandLoader<D> loader;
-    public final ImmutableSortedMap<Timestamp, D> commands;
-
-    public CommandTimeseries(Builder<D> builder)
-    {
-        this.keyOrRange = builder.keyOrRange;
-        this.loader = builder.loader;
-        this.commands = ensureSortedImmutable(builder.commands);
-    }
-
-    CommandTimeseries(Seekable keyOrRange, CommandLoader<D> loader, ImmutableSortedMap<Timestamp, D> commands)
-    {
-        this.keyOrRange = keyOrRange;
-        this.loader = loader;
-        this.commands = commands;
-    }
-
-    public CommandTimeseries(Key keyOrRange, CommandLoader<D> loader)
-    {
-        this(keyOrRange, loader, ImmutableSortedMap.of());
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        CommandTimeseries<?> that = (CommandTimeseries<?>) o;
-        return keyOrRange.equals(that.keyOrRange) && loader.equals(that.loader) && commands.equals(that.commands);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        int hash = 1;
-        hash = 31 * hash + Objects.hashCode(keyOrRange);
-        hash = 31 * hash + Objects.hashCode(loader);
-        hash = 31 * hash + Objects.hashCode(commands);
-        return hash;
-    }
-
-    public D get(Timestamp key)
-    {
-        return commands.get(key);
-    }
-
-    public boolean isEmpty()
-    {
-        return commands.isEmpty();
-    }
-
-    public Timestamp maxTimestamp()
-    {
-        if (commands.isEmpty())
-            return Timestamp.NONE;
-        Timestamp result = null;
-        for (D data : commands.values())
-        {
-            if (result == null)
-            {
-                result = Timestamp.max(loader.txnId(data), loader.executeAt(data));
-            }
-            else
-            {
-                result = Timestamp.max(result, loader.txnId(data));
-                result = Timestamp.max(result, loader.executeAt(data));
-            }
-        }
-        return result;
-    }
-
-    public Timestamp minTimestamp()
-    {
-        if (commands.isEmpty())
-            return Timestamp.NONE;
-        Timestamp result = null;
-        for (D data : commands.values())
-        {
-            if (result == null)
-            {
-                result = Timestamp.nonNullOrMin(loader.txnId(data), loader.executeAt(data));
-            }
-            else
-            {
-                result = Timestamp.min(result, loader.txnId(data));
-                result = Timestamp.nonNullOrMin(result, loader.executeAt(data));
-            }
-        }
-        return result;
-    }
-
-    /**
-     * All commands before/after (exclusive of) the given timestamp
-     * <p>
-     * Note that {@code testDep} applies only to commands that know at least proposed deps; if specified any
-     * commands that do not know any deps will be ignored.
-     * <p>
-     * TODO (expected, efficiency): TestDep should be asynchronous; data should not be kept memory-resident as only used for recovery
-     */
-    public <P1, T> T mapReduce(Kinds testKind, TimestampType timestampType, TestTimestamp testTimestamp, Timestamp timestamp,
-                           SafeCommandStore.TestDep testDep, @Nullable TxnId depId,
-                           @Nullable Status minStatus, @Nullable Status maxStatus,
-                           SafeCommandStore.CommandFunction<P1, T, T> map, P1 p1, T initialValue, Predicate<? super T> terminatePredicate)
-    {
-        Iterable<D> dataIterable;
-        Predicate<Timestamp> executeAtPredicate;
-        if (timestampType == TimestampType.TXN_ID)
-        {
-            dataIterable = testTimestamp == TestTimestamp.BEFORE ? commands.headMap(timestamp, false).values() : commands.tailMap(timestamp, false).values();
-            executeAtPredicate = ts -> true;
-        }
-        else
-        {
-            dataIterable = commands.values();
-            executeAtPredicate = testTimestamp == TestTimestamp.BEFORE ? ts -> ts.compareTo(timestamp) < 0 : ts -> ts.compareTo(timestamp) > 0;
-        }
-
-
-        for (D data : dataIterable)
-        {
-            TxnId txnId = loader.txnId(data);
-            if (!testKind.test(txnId.kind())) continue;
-            SaveStatus status = loader.saveStatus(data);
-            if (minStatus != null && minStatus.compareTo(status.status) > 0)
-                continue;
-            if (maxStatus != null && maxStatus.compareTo(status.status) < 0)
-                continue;
-
-            Timestamp executeAt = loader.executeAt(data);
-            if (!executeAtPredicate.test(executeAt))
-                continue;
-
-            List<TxnId> deps = loader.depsIds(data);
-            // If we don't have any dependencies, we treat a dependency filter as a mismatch
-            if (testDep != ANY_DEPS && (!status.known.deps.hasProposedOrDecidedDeps() || (deps.contains(depId) != (testDep == WITH))))
-                continue;
-            initialValue = map.apply(p1, keyOrRange, txnId, executeAt, status.status, () -> deps, initialValue);
-            if (terminatePredicate.test(initialValue))
-                break;
-        }
-        return initialValue;
-    }
-
-    public Timestamp maxExecuteAtBefore(Timestamp before)
-    {
-        return commands.headMap(before).values().stream().map(loader::executeAt)
-                       .filter(Objects::nonNull).reduce(Timestamp::max).orElse(before);
-    }
-
-    Stream<TxnId> between(Timestamp min, Timestamp max, Predicate<Status> statusPredicate)
-    {
-        return commands.subMap(min, true, max, true).values().stream()
-                       .filter(d -> statusPredicate.test(loader.status(d))).map(loader::txnId);
-    }
-
-    public Stream<D> all()
-    {
-        return commands.values().stream();
-    }
-
-    Builder<D> unbuild()
-    {
-        return new Builder<>(this);
-    }
-
-    public CommandLoader<D> loader()
-    {
-        return loader;
-    }
-
-    public interface CommandLoader<D>
-    {
-        D saveForCFK(Command command);
-
-        TxnId txnId(D data);
-        Timestamp executeAt(D data);
-        SaveStatus saveStatus(D data);
-        List<TxnId> depsIds(D data);
-
-        default Status status(D data)
-        {
-            return saveStatus(data).status;
-        }
-
-        default Status.Known known(D data)
-        {
-            return saveStatus(data).known;
-        }
-    }
-
-    public static class Builder<D>
-    {
-        private final Seekable keyOrRange;
-        protected CommandLoader<D> loader;
-        protected NavigableMap<Timestamp, D> commands;
-
-        public Builder(Seekable keyOrRange, CommandLoader<D> loader)
-        {
-            this.keyOrRange = keyOrRange;
-            this.loader = loader;
-            this.commands = new TreeMap<>();
-        }
-
-        public Builder(CommandTimeseries<D> timeseries)
-        {
-            this.keyOrRange = timeseries.keyOrRange;
-            this.loader = timeseries.loader;
-            this.commands = timeseries.commands;
-        }
-
-        public Builder<D> add(Timestamp timestamp, Command command)
-        {
-            commands = ensureSortedMutable(commands);
-            commands.put(timestamp, loader.saveForCFK(command));
-            return this;
-        }
-
-        public Builder<D> add(Timestamp timestamp, D value)
-        {
-            commands = ensureSortedMutable(commands);
-            commands.put(timestamp, value);
-            return this;
-        }
-
-        public Builder<D> remove(Timestamp timestamp)
-        {
-            commands = ensureSortedMutable(commands);
-            commands.remove(timestamp);
-            return this;
-        }
-
-        public Builder<D> removeBefore(Timestamp timestamp)
-        {
-            commands = ensureSortedMutable(commands);
-            commands.headMap(timestamp, false).clear();
-            return this;
-        }
-
-        public CommandTimeseries<D> build()
-        {
-            return new CommandTimeseries<>(this);
-        }
-    }
-
-    public interface Update<T extends Timestamp, D>
-    {
-        interface Mutable<T extends Timestamp, D> extends Update<T, D>
-        {
-            void add(T ts, Command command);
-            void remove(T ts);
-        }
-
-        int numWrites();
-        int numDeletes();
-
-        boolean contains(T key);
-
-        default int numChanges()
-        {
-            return numWrites() + numDeletes();
-        }
-
-        default boolean isEmpty()
-        {
-            return numChanges() == 0;
-        }
-
-        void forEachWrite(BiConsumer<T, D> consumer);
-        void forEachDelete(Consumer<T> consumer);
-
-        default  CommandTimeseries<D> apply(CommandTimeseries<D> current)
-        {
-            if (isEmpty())
-                return current;
-
-            CommandTimeseries.Builder<D> builder = current.unbuild();
-            forEachWrite(builder::add);
-            forEachDelete(builder::remove);
-            return builder.build();
-        }
-
-    }
-
-    public static class ImmutableUpdate<T extends Timestamp, D> implements Update<T, D>
-    {
-        private static final ImmutableUpdate<?, ?> EMPTY = new ImmutableUpdate<>(ImmutableMap.of(), ImmutableSet.of());
-
-        public final ImmutableMap<T, D> writes;
-        public final ImmutableSet<T> deletes;
-
-        public ImmutableUpdate(ImmutableMap<T, D> writes, ImmutableSet<T> deletes)
-        {
-            this.writes = writes;
-            this.deletes = deletes;
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            ImmutableUpdate<?, ?> immutable = (ImmutableUpdate<?, ?>) o;
-            return Objects.equals(writes, immutable.writes) && Objects.equals(deletes, immutable.deletes);
-        }
-
-        public int hashCode()
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public String toString()
-        {
-            return "Update.Immutable{" +
-                   "writes=" + writes +
-                   ", deletes=" + deletes +
-                   '}';
-        }
-
-        public static <T extends Timestamp, D> ImmutableUpdate<T, D> empty()
-        {
-            return (ImmutableUpdate<T, D>) EMPTY;
-        }
-
-        @Override
-        public int numWrites()
-        {
-            return writes.size();
-        }
-
-        @Override
-        public void forEachWrite(BiConsumer<T, D> consumer)
-        {
-            writes.forEach(consumer);
-        }
-
-        @Override
-        public int numDeletes()
-        {
-            return deletes.size();
-        }
-
-        @Override
-        public void forEachDelete(Consumer<T> consumer)
-        {
-            deletes.forEach(consumer);
-        }
-
-        public boolean contains(T key)
-        {
-            return writes.containsKey(key) || deletes.contains(key);
-        }
-
-        public ImmutableUpdate<T, D> apply(Update<T, D> next)
-        {
-            if (next.isEmpty())
-                return this;
-
-            Map<T, D> writes = new HashMap<>(this.writes);
-            Set<T> deletes = new HashSet<>(this.deletes);
-
-            next.forEachDelete(k -> {
-                writes.remove(k);
-                deletes.add(k);
-            });
-
-            next.forEachWrite((k, v) -> {
-                writes.put(k , v);
-                deletes.remove(k);
-            });
-
-            return new ImmutableUpdate<>(ImmutableMap.copyOf(writes), ImmutableSet.copyOf(deletes));
-        }
-
-        public MutableUpdate<T, D> toMutable(CommandLoader<D> loader)
-        {
-            return new MutableUpdate<>(loader, new HashMap<>(writes), new HashSet<>(deletes));
-        }
-    }
-
-    public static class MutableUpdate<T extends Timestamp, D> implements Update.Mutable<T, D>
-    {
-        private final CommandLoader<D> loader;
-        private final Map<T, D> writes;
-        private final Set<T> deletes;
-
-        public MutableUpdate(CommandLoader<D> loader, Map<T, D> writes, Set<T> deletes)
-        {
-            this.loader = loader;
-            this.writes = writes;
-            this.deletes = deletes;
-        }
-
-        public MutableUpdate(CommandLoader<D> loader)
-        {
-            this(loader, new HashMap<>(), new HashSet<>());
-        }
-
-        @Override
-        public void add(T ts, Command command)
-        {
-            writes.put(ts, loader.saveForCFK(command));
-            deletes.remove(ts);
-        }
-
-        @Override
-        public void remove(T ts)
-        {
-            deletes.add(ts);
-            writes.remove(ts);
-        }
-
-        @Override
-        public int numWrites()
-        {
-            return writes.size();
-        }
-
-        @Override
-        public void forEachWrite(BiConsumer<T, D> consumer)
-        {
-            writes.forEach(consumer);
-        }
-
-        @Override
-        public int numDeletes()
-        {
-            return deletes.size();
-        }
-
-        @Override
-        public void forEachDelete(Consumer<T> consumer)
-        {
-            deletes.forEach(consumer);
-        }
-
-        public boolean contains(T key)
-        {
-            return writes.containsKey(key) || deletes.contains(key);
-        }
-
-        /**
-         * remove the given key from both the writes and deletes set
-         * @param key
-         */
-        void removeKeyUnsafe(T key)
-        {
-            writes.remove(key);
-            deletes.remove(key);
-        }
-
-        /**
-         * add the given write data directly, without using the command loader
-         * @param key
-         * @param value
-         */
-        void addWriteUnsafe(T key, D value)
-        {
-            writes.put(key, value);
-            deletes.remove(key);
-        }
-
-        /**
-         * if this update has a write or delete for the given key, remove
-         * it from this update and put it in the destination update
-         */
-        boolean transferKeyTo(T key, MutableUpdate<T, D> dst)
-        {
-            if (writes.containsKey(key))
-            {
-                dst.writes.put(key, this.writes.remove(key));
-                return true;
-            }
-            else if (deletes.remove(key))
-            {
-                dst.deletes.add(key);
-                return true;
-            }
-            return false;
-        }
-
-        public ImmutableUpdate<T, D> toImmutable()
-        {
-            return new ImmutableUpdate<>(ImmutableMap.copyOf(writes), ImmutableSet.copyOf(deletes));
-        }
-    }
-}
diff --git a/accord-core/src/main/java/accord/impl/CommandsForKey.java b/accord-core/src/main/java/accord/impl/CommandsForKey.java
index 59be3bd..185c143 100644
--- a/accord-core/src/main/java/accord/impl/CommandsForKey.java
+++ b/accord-core/src/main/java/accord/impl/CommandsForKey.java
@@ -18,44 +18,251 @@
 
 package accord.impl;
 
-import accord.api.Key;
-import accord.impl.CommandTimeseries.CommandLoader;
-import com.google.common.collect.ImmutableSortedMap;
-
+import java.util.Arrays;
+import java.util.EnumMap;
 import java.util.Objects;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import accord.api.Key;
 
 import accord.local.Command;
-import accord.local.PreLoadContext;
-import accord.local.SafeCommand;
-import accord.local.SafeCommandStore;
-import accord.primitives.Keys;
+import accord.local.SafeCommandStore.CommandFunction;
+import accord.local.SafeCommandStore.TestDep;
+import accord.local.SafeCommandStore.TestStartedAt;
+import accord.local.SafeCommandStore.TestStatus;
+import accord.local.SaveStatus;
+import accord.local.Status;
+import accord.primitives.Ballot;
 import accord.primitives.Timestamp;
+import accord.primitives.Txn.Kind.Kinds;
 import accord.primitives.TxnId;
+import accord.utils.Invariants;
+import accord.utils.SortedArrays;
+import accord.utils.SortedList;
 
-public class CommandsForKey implements DomainCommands
+import static accord.impl.CommandsForKey.InternalStatus.ACCEPTED;
+import static accord.impl.CommandsForKey.InternalStatus.COMMITTED;
+import static accord.impl.CommandsForKey.InternalStatus.STABLE;
+import static accord.impl.CommandsForKey.InternalStatus.HISTORICAL;
+import static accord.impl.CommandsForKey.InternalStatus.INVALID_OR_TRUNCATED;
+import static accord.impl.CommandsForKey.InternalStatus.TRANSITIVELY_KNOWN;
+import static accord.local.SafeCommandStore.TestDep.ANY_DEPS;
+import static accord.local.SafeCommandStore.TestDep.WITH;
+import static accord.primitives.Txn.Kind.Write;
+import static accord.utils.ArrayBuffers.cachedTxnIds;
+import static accord.utils.Invariants.illegalState;
+import static accord.utils.Invariants.isParanoid;
+import static accord.utils.SortedArrays.linearUnion;
+
+/**
+ * A specialised collection for efficiently representing and querying everything we need for making coordination
+ * and recovery decisions about a key's command conflicts.
+ *
+ * Every command we know about that is not shard-redundant is listed in the TxnId[] collection, which is sorted by TxnId.
+ * This list implies the contents of the deps of all commands in the collection - it is assumed that in the normal course
+ * of events every transaction will include the full set of TxnId we know. We only encode divergences from this, stored
+ * in each command's {@code missing} collection.
+ *
+ * We then go one step further, exploiting the fact that the missing collection exists solely to implement recovery,
+ * and so we elide from this missing collection any TxnId we have recorded as Committed or higher.
+ * Any recovery coordinator that contacts this replica will report that the command has been agreed to execute,
+ * and so will not need to decipher any fast-path decisions. So the missing collection is redundant, as no command's deps
+ * will not be queried for this TxnId's presence/absence.
+ * TODO (expected) this logic applies equally well to Accepted
+ *
+ * The goal with these behaviours is that this missing collection will ordinarily be empty, and represented by the exact
+ * same NO_TXNIDS array instance as every other command.
+ *
+ * We also exploit the property that most commands will also agree to execute at their proposed TxnId. If we have
+ * no missing collection to encode, and no modified executeAt, we store a global NoInfo object that takes up no
+ * space on heap. These NoInfo objects permit further efficiencies, as we may perform class-pointer comparisons
+ * before querying any contents to skip uninteresting contents, permitting fast iteration of the collection's contents.
+ *
+ * We also impose the condition that every TxnId is uniquely represented in the collection, so any executeAt and missing
+ * collection that represents the same value as the TxnId must be the same object present in the main TxnId[].
+ *
+ * This collection also implements transitive dependency elision.
+ * When evaluating mapReduceActive, we first establish the last-executing Stable write command (i.e. those whose deps
+ * are considered durably decided, and so must wait for all commands Committed with a lower executeAt).
+ * We then elide any Committed command that has a lower executeAt than this command.
+ *
+ * Both commands must be known at a majority, but neither might be Committed at any other replica.
+ * Either command may therefore be recovered.
+ * If the later command is recovered, this replica will report its Stable deps thereby recovering them.
+ * If this replica is not contacted, some other replica must participate that either has taken the same action as this replica,
+ * or else does not know the later command is Stable, and so will report the earlier command as a dependency again.
+ * If the earlier command is recovered, this replica will report that it is Committed, and so will not consult
+ * this replica's collection to decipher any fast path decision. Any other replica must either do the same, or else
+ * will correctly record this transaction as present in any relevant deps of later transactions.
+ *
+ * TODO (expected): optimisations:
+ *    1) we probably can rely only on COMMITTED status rather than STABLE for filtering active, but require STABLE still for filtering on status
+ *    2) do we need PREAPPLIED state?
+ *    3) consider storing a prefix of TxnId that are all NoInfo PreApplied encoded as a BitStream as only required for computing missing collection
+ *    4) consider storing (or caching) an int[] of records with an executeAt that occurs out of order, sorted by executeAt
+ *
+ *    TODO (required): randomised testing
+ */
+public class CommandsForKey implements CommandsSummary
 {
+    private static final boolean PRUNE_TRANSITIVE_DEPENDENCIES = true;
+    public static final TxnId[] NO_TXNIDS = new TxnId[0];
+    public static final Info[] NO_INFOS = new Info[0];
+
     public static class SerializerSupport
     {
-        public static Listener listener(Key key)
+        public static CommandsForKey create(Key key, TxnId redundantBefore, TxnId[] txnIds, Info[] infos)
         {
-            return new Listener(key);
-        }
-
-        public static  <D> CommandsForKey create(Key key,
-                                                 CommandLoader<D> loader,
-                                                 ImmutableSortedMap<Timestamp, D> commands)
-        {
-            return new CommandsForKey(key, loader, commands);
+            return new CommandsForKey(key, redundantBefore, txnIds, infos);
         }
     }
 
-    public static class Listener implements Command.DurableAndIdempotentListener
+    public enum InternalStatus
     {
-        protected final Key listenerKey;
+        TRANSITIVELY_KNOWN(false, false), // (unwitnessed; no need for mapReduce to witness)
+        HISTORICAL(false, false),
+        PREACCEPTED(false),
+        ACCEPTED(true),
+        COMMITTED(true),
+        STABLE(true),
+        // TODO (required): we can probably retire this status; STABLE is sufficient
+        PREAPPLIED(true),
+        INVALID_OR_TRUNCATED(false);
 
-        public Listener(Key listenerKey)
+        static final EnumMap<SaveStatus, InternalStatus> convert = new EnumMap<>(SaveStatus.class);
+        static final InternalStatus[] VALUES = values();
+        static
         {
-            this.listenerKey = listenerKey;
+            convert.put(SaveStatus.PreAccepted, PREACCEPTED);
+            convert.put(SaveStatus.AcceptedInvalidateWithDefinition, PREACCEPTED);
+            convert.put(SaveStatus.Accepted, ACCEPTED);
+            convert.put(SaveStatus.AcceptedWithDefinition, ACCEPTED);
+            convert.put(SaveStatus.PreCommittedWithDefinition, PREACCEPTED);
+            convert.put(SaveStatus.PreCommittedWithAcceptedDeps, ACCEPTED);
+            convert.put(SaveStatus.PreCommittedWithDefinitionAndAcceptedDeps, ACCEPTED);
+            convert.put(SaveStatus.Committed, COMMITTED);
+            convert.put(SaveStatus.Stable, STABLE);
+            convert.put(SaveStatus.ReadyToExecute, STABLE);
+            convert.put(SaveStatus.PreApplied, PREAPPLIED);
+            convert.put(SaveStatus.Applying, PREAPPLIED);
+            convert.put(SaveStatus.Applied, PREAPPLIED);
+            convert.put(SaveStatus.TruncatedApplyWithDeps, INVALID_OR_TRUNCATED);
+            convert.put(SaveStatus.TruncatedApplyWithOutcome, INVALID_OR_TRUNCATED);
+            convert.put(SaveStatus.TruncatedApply, INVALID_OR_TRUNCATED);
+            convert.put(SaveStatus.ErasedOrInvalidated, INVALID_OR_TRUNCATED);
+            convert.put(SaveStatus.Erased, INVALID_OR_TRUNCATED);
+            convert.put(SaveStatus.Invalidated, INVALID_OR_TRUNCATED);
+        }
+
+        public final boolean hasInfo;
+        public final NoInfo asNoInfo;
+        public final InfoWithAdditions asNoInfoOrAdditions;
+        final InternalStatus expectMatch;
+
+        InternalStatus(boolean hasInfo)
+        {
+            this(hasInfo, true);
+        }
+
+        InternalStatus(boolean hasInfo, boolean expectMatch)
+        {
+            this.hasInfo = hasInfo;
+            this.asNoInfo = new NoInfo(this);
+            this.asNoInfoOrAdditions = new InfoWithAdditions(asNoInfo, NO_TXNIDS, 0);
+            this.expectMatch = expectMatch ? this : null;
+        }
+
+        boolean hasExecuteAt()
+        {
+            return hasInfo;
+        }
+
+        boolean hasDeps()
+        {
+            return hasInfo;
+        }
+
+        boolean hasStableDeps()
+        {
+            return this == STABLE || this == PREAPPLIED;
+        }
+
+        public Timestamp depsKnownBefore(TxnId txnId, @Nullable Timestamp executeAt)
+        {
+            switch (this)
+            {
+                default: throw new AssertionError("Unhandled InternalStatus: " + this);
+                case TRANSITIVELY_KNOWN:
+                case INVALID_OR_TRUNCATED:
+                case HISTORICAL:
+                    throw new AssertionError("Invalid InternalStatus to know deps");
+
+                case PREACCEPTED:
+                case ACCEPTED:
+                    return txnId;
+
+                case PREAPPLIED:
+                case STABLE:
+                case COMMITTED:
+                    return executeAt == null ? txnId : executeAt;
+            }
+        }
+
+        @VisibleForTesting
+        public static InternalStatus from(SaveStatus status)
+        {
+            return convert.get(status);
+        }
+
+        public static InternalStatus get(int ordinal)
+        {
+            return VALUES[ordinal];
+        }
+    }
+
+    public static class Info
+    {
+        public final InternalStatus status;
+        // ACCESS DIRECTLY WITH CARE: if null, TxnId is implied; use accessor method to ensure correct value is returned
+        public final @Nullable Timestamp executeAt;
+        public final TxnId[] missing; // those TxnId we know of that would be expected to be found in the provided deps, but aren't
+
+        private Info(InternalStatus status, @Nullable Timestamp executeAt, TxnId[] missing)
+        {
+            this.status = status;
+            this.executeAt = executeAt;
+            this.missing = missing;
+        }
+
+        public static Info create(@Nonnull TxnId txnId, InternalStatus status, @Nonnull Timestamp executeAt, @Nonnull TxnId[] missing)
+        {
+            return new Info(status,
+                            Invariants.checkArgument(executeAt, executeAt == txnId || executeAt.compareTo(txnId) > 0),
+                            Invariants.checkArgument(missing, missing == NO_TXNIDS || missing.length > 0));
+        }
+
+        public static Info createMock(InternalStatus status, @Nullable Timestamp executeAt, TxnId[] missing)
+        {
+            return new Info(status, executeAt, Invariants.checkArgument(missing, missing == null || missing == NO_TXNIDS));
+        }
+
+        @VisibleForTesting
+        public Timestamp executeAt(TxnId txnId)
+        {
+            return executeAt;
+        }
+
+        Timestamp depsKnownBefore(TxnId txnId)
+        {
+            return status.depsKnownBefore(txnId, executeAt);
+        }
+
+        Info update(TxnId txnId, TxnId[] newMissing)
+        {
+            return newMissing == NO_TXNIDS && executeAt == txnId ? status.asNoInfo : Info.create(txnId, status, executeAt(txnId), newMissing);
         }
 
         @Override
@@ -63,59 +270,69 @@
         {
             if (this == o) return true;
             if (o == null || getClass() != o.getClass()) return false;
-            Listener that = (Listener) o;
-            return listenerKey.equals(that.listenerKey);
+            Info info = (Info) o;
+            return status == info.status && Objects.equals(executeAt, info.executeAt) && Arrays.equals(missing, info.missing);
         }
 
         @Override
         public int hashCode()
         {
-            return Objects.hash(listenerKey);
+            throw new UnsupportedOperationException();
         }
 
         @Override
         public String toString()
         {
-            return "ListenerProxy{" + listenerKey + '}';
+            return "Info{" +
+                   "status=" + status +
+                   ", executeAt=" + executeAt +
+                   ", missing=" + Arrays.toString(missing) +
+                   '}';
+        }
+    }
+
+    public static class NoInfo extends Info
+    {
+        NoInfo(InternalStatus status)
+        {
+            super(status, null, NO_TXNIDS);
         }
 
-        public Key key()
+        public Timestamp executeAt(TxnId txnId)
         {
-            return listenerKey;
-        }
-
-        @Override
-        public void onChange(SafeCommandStore safeStore, SafeCommand safeCommand)
-        {
-            CommandsForKeys.listenerUpdate((AbstractSafeCommandStore<?,?,?,?>) safeStore, listenerKey, safeCommand.current());
-        }
-
-        @Override
-        public PreLoadContext listenerPreLoadContext(TxnId caller)
-        {
-            return PreLoadContext.contextFor(caller, Keys.of(listenerKey));
+            return txnId;
         }
     }
 
     private final Key key;
-    private final CommandTimeseries<?> commands;
+    private final TxnId redundantBefore;
+    // any transactions that are durably decided (i.e. STABLE) and execute before this are durably dependent, and can be elided from mapReduceActive
+    private final @Nullable Timestamp maxStableWrite;
+    private final TxnId[] txnIds;
+    private final Info[] infos;
 
-    <D> CommandsForKey(Key key,
-                       CommandTimeseries<D> commands)
+    CommandsForKey(Key key, TxnId redundantBefore, TxnId[] txnIds, Info[] infos)
     {
-        this.key = key;
-        this.commands = commands;
+        this(key, redundantBefore, maxStableWriteBefore(Timestamp.MAX, txnIds, infos), txnIds, infos);
     }
 
-    <D> CommandsForKey(Key key, CommandLoader<D> loader, ImmutableSortedMap<Timestamp, D> commands)
-    {
-        this(key, new CommandTimeseries<>(key, loader, commands));
-    }
-
-    public <D> CommandsForKey(Key key, CommandLoader<D> loader)
+    CommandsForKey(Key key, TxnId redundantBefore, @Nullable Timestamp maxStableWrite, TxnId[] txnIds, Info[] infos)
     {
         this.key = key;
-        this.commands = new CommandTimeseries<>(key, loader);
+        this.redundantBefore = redundantBefore;
+        this.maxStableWrite = maxStableWrite;
+        this.txnIds = txnIds;
+        this.infos = infos;
+        if (isParanoid()) Invariants.checkArgument(SortedArrays.isSortedUnique(txnIds));
+    }
+
+    public CommandsForKey(Key key)
+    {
+        this.key = key;
+        this.redundantBefore = TxnId.NONE;
+        this.maxStableWrite = null;
+        this.txnIds = NO_TXNIDS;
+        this.infos = NO_INFOS;
     }
 
     @Override
@@ -124,50 +341,718 @@
         return "CommandsForKey@" + System.identityHashCode(this) + '{' + key + '}';
     }
 
+    public Key key()
+    {
+        return key;
+    }
+
+    public int size()
+    {
+        return txnIds.length;
+    }
+
+    public int indexOf(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId);
+    }
+
+    public TxnId txnId(int i)
+    {
+        return txnIds[i];
+    }
+
+    public Info info(int i)
+    {
+        return infos[i];
+    }
+
+    public TxnId redundantBefore()
+    {
+        return redundantBefore;
+    }
+
+    /**
+     * All commands before/after (exclusive of) the given timestamp
+     * <p>
+     * Note that {@code testDep} applies only to commands that MAY have the command in their deps; if specified any
+     * commands that do not know any deps will be ignored, as will any with an executeAt prior to the txnId.
+     * <p>
+     */
+    public <P1, T> T mapReduceFull(TxnId testTxnId,
+                                   Kinds testKind,
+                                   TestStartedAt testStartedAt,
+                                   TestDep testDep,
+                                   TestStatus testStatus,
+                                   CommandFunction<P1, T, T> map, P1 p1, T initialValue)
+    {
+        int start, end;
+        boolean isKnown;
+        {
+            int insertPos = Arrays.binarySearch(txnIds, testTxnId);
+            isKnown = insertPos >= 0;
+            if (!isKnown && testDep == WITH) return initialValue;
+            if (!isKnown) insertPos = -1 - insertPos;
+            switch (testStartedAt)
+            {
+                default: throw new AssertionError("Unhandled TestStartedAt: " + testTxnId);
+                case STARTED_BEFORE: start = 0; end = insertPos; break;
+                case STARTED_AFTER: start = insertPos; end = txnIds.length; break;
+                case ANY: start = 0; end = txnIds.length;
+            }
+        }
+
+        for (int i = start; i < end ; ++i)
+        {
+            TxnId txnId = txnIds[i];
+            if (!testKind.test(txnId.kind())) continue;
+
+            Info info = infos[i];
+            InternalStatus status = info.status;
+            switch (testStatus)
+            {
+                default: throw new AssertionError("Unhandled TestStatus: " + testStatus);
+                case IS_PROPOSED:
+                    if (status == ACCEPTED || status == COMMITTED) break;
+                    else continue;
+                case IS_STABLE:
+                    if (status.compareTo(STABLE) >= 0 && status.compareTo(INVALID_OR_TRUNCATED) < 0) break;
+                    else continue;
+                case ANY_STATUS:
+                    if (status == TRANSITIVELY_KNOWN)
+                        continue;
+            }
+
+            Timestamp executeAt = info.executeAt(txnId);
+            if (testDep != ANY_DEPS)
+            {
+                if (!status.hasInfo)
+                    continue;
+
+                if (executeAt.compareTo(testTxnId) <= 0)
+                    continue;
+
+                boolean hasAsDep = Arrays.binarySearch(info.missing, testTxnId) < 0;
+                if (hasAsDep != (testDep == WITH))
+                    continue;
+            }
+
+            initialValue = map.apply(p1, key, txnId, executeAt, initialValue);
+        }
+        return initialValue;
+    }
+
+    public <P1, T> T mapReduceActive(Timestamp startedBefore,
+                                     Kinds testKind,
+                                     CommandFunction<P1, T, T> map, P1 p1, T initialValue)
+    {
+        Timestamp maxStableWrite = maxStableWriteBefore(startedBefore);
+        int start = 0, end = insertPos(startedBefore);
+
+        for (int i = start; i < end ; ++i)
+        {
+            TxnId txnId = txnIds[i];
+            if (!testKind.test(txnId.kind()))
+                continue;
+
+            Info info = infos[i];
+            switch (info.status)
+            {
+                case COMMITTED:
+                case STABLE:
+                case PREAPPLIED:
+                    // TODO (expected): prove the correctness of this approach
+                    if (!PRUNE_TRANSITIVE_DEPENDENCIES || maxStableWrite == null || info.executeAt(txnId).compareTo(maxStableWrite) >= 0)
+                        break;
+                case TRANSITIVELY_KNOWN:
+                case INVALID_OR_TRUNCATED:
+                    continue;
+            }
+
+            initialValue = map.apply(p1, key, txnId, info.executeAt(txnId), initialValue);
+        }
+        return initialValue;
+    }
+
+    public CommandsForKey update(Command prev, Command next)
+    {
+        InternalStatus newStatus = InternalStatus.from(next.saveStatus());
+        if (newStatus == null)
+            return this;
+
+        TxnId txnId = next.txnId();
+        int pos = Arrays.binarySearch(txnIds, txnId);
+        if (pos < 0)
+        {
+            pos = -1 - pos;
+            if (!newStatus.hasInfo)
+                return insert(pos, txnId, newStatus.asNoInfo);
+
+            return insert(pos, txnId, newStatus, next);
+        }
+        else
+        {
+            // we do not permit multiple versions of the same TxnId, so use the existing one
+            txnId = txnIds[pos];
+
+            // update
+            InternalStatus prevStatus = prev == null ? null : InternalStatus.from(prev.saveStatus());
+            if (newStatus == prevStatus && (!newStatus.hasInfo || next.acceptedOrCommitted().equals(prev.acceptedOrCommitted())))
+                return this;
+
+            @Nonnull Info cur = Invariants.nonNull(infos[pos]);
+            // TODO (required): HACK to permit prev.saveStatus() == SaveStatus.AcceptedInvalidateWithDefinition as we don't always update as keys aren't guaranteed to be provided
+            //    fix as soon as we support async updates
+            Invariants.checkState(cur.status.expectMatch == prevStatus || (prev != null && prev.status() == Status.AcceptedInvalidate));
+
+            if (newStatus.hasInfo) return update(pos, txnId, cur, newStatus, next);
+            else return update(pos, cur, newStatus.asNoInfo);
+        }
+    }
+
+    public static boolean needsUpdate(Command prev, Command updated)
+    {
+        if (!updated.txnId().kind().isGloballyVisible())
+            return false;
+
+        SaveStatus prevStatus;
+        Ballot prevAcceptedOrCommitted;
+        if (prev == null)
+        {
+            prevStatus = SaveStatus.NotDefined;
+            prevAcceptedOrCommitted = Ballot.ZERO;
+        }
+        else
+        {
+            prevStatus = prev.saveStatus();
+            prevAcceptedOrCommitted = prev.acceptedOrCommitted();
+        }
+
+        return needsUpdate(prevStatus, prevAcceptedOrCommitted, updated.saveStatus(), updated.acceptedOrCommitted());
+    }
+
+    public static boolean needsUpdate(SaveStatus prevStatus, Ballot prevAcceptedOrCommitted, SaveStatus updatedStatus, Ballot updatedAcceptedOrCommitted)
+    {
+        InternalStatus prev = InternalStatus.from(prevStatus);
+        InternalStatus updated = InternalStatus.from(updatedStatus);
+        return updated != prev || (updated != null && updated.hasInfo && !prevAcceptedOrCommitted.equals(updatedAcceptedOrCommitted));
+    }
+
+    private Timestamp maxStableWriteBefore(Timestamp before)
+    {
+        if (maxStableWrite == null)
+            return null;
+
+        if (maxStableWrite.compareTo(before) < 0)
+            return maxStableWrite;
+
+        return maxStableWriteBefore(before, txnIds, infos);
+    }
+
+    private static Timestamp maxStableWriteBefore(Timestamp before, TxnId[] txnIds, Info[] infos)
+    {
+        int i = Arrays.binarySearch(txnIds, before);
+        if (i < 0) i = -1 -i;
+        Timestamp max = null;
+        while (--i >= 0)
+        {
+            TxnId txnId = txnIds[i];
+            if (txnId.kind() != Write) continue;
+
+            Info info = infos[i];
+            if (!info.status.hasStableDeps()) continue;
+
+            if (info.executeAt == null) max = txnId;
+            else if (info.executeAt.compareTo(before) < 0) max = info.executeAt;
+            else continue;
+
+            break;
+        }
+
+        while (--i >= 0)
+        {
+            Info info = infos[i];
+            TxnId txnId = txnIds[i];
+
+            if (info.getClass() == NoInfo.class) continue;
+            if (!info.status.hasStableDeps()) continue;
+            if (txnId.kind() != Write) continue;
+            if (info.executeAt.compareTo(max) <= 0 || info.executeAt.compareTo(before) >= 0) continue;
+
+            max = info.executeAt;
+        }
+        return max;
+    }
+
+    private CommandsForKey insert(int insertPos, TxnId insertTxnId, InternalStatus newStatus, Command command)
+    {
+        InfoWithAdditions newInfo = computeInsert(insertPos, insertTxnId, newStatus, command);
+        if (newInfo.additionCount == 0)
+            return insert(insertPos, insertTxnId, newInfo.info);
+
+        TxnId[] newTxnIds = new TxnId[txnIds.length + newInfo.additionCount + 1];
+        Info[] newInfos = new Info[newTxnIds.length];
+        insertWithAdditions(insertPos, insertTxnId, newInfo, newTxnIds, newInfos);
+        return update(newTxnIds, newInfos, insertTxnId, newInfo.info);
+    }
+
+    private CommandsForKey update(int updatePos, TxnId txnId, Info prevInfo, InternalStatus newStatus, Command command)
+    {
+        InfoWithAdditions newInfo = computeUpdate(updatePos, txnId, newStatus, command);
+        if (newInfo.additionCount == 0)
+            return update(updatePos, prevInfo, newInfo.info);
+
+        TxnId[] newTxnIds = new TxnId[txnIds.length + newInfo.additionCount];
+        Info[] newInfos = new Info[newTxnIds.length];
+        TxnId updateTxnId = txnIds[updatePos]; // want to reuse the existing TxnId for object identity
+        int newPos = updateWithAdditions(updatePos, txnIds[updatePos], newInfo, newTxnIds, newInfos);
+        if (prevInfo.status.compareTo(COMMITTED) < 0 && newStatus.compareTo(COMMITTED) >= 0)
+            removeMissing(newTxnIds, newInfos, newPos);
+        return update(newTxnIds, newInfos, updateTxnId, newInfo.info);
+    }
+
+    private int updateWithAdditions(int updatePos, TxnId updateTxnId, InfoWithAdditions withInfo, TxnId[] newTxnIds, Info[] newInfos)
+    {
+        return updateOrInsertWithAdditions(updatePos, updatePos, updateTxnId, withInfo, newTxnIds, newInfos);
+    }
+
+    private void insertWithAdditions(int pos, TxnId updateTxnId, InfoWithAdditions withInfo, TxnId[] newTxnIds, Info[] newInfos)
+    {
+        updateOrInsertWithAdditions(pos, -1, updateTxnId, withInfo, newTxnIds, newInfos);
+    }
+
+    private int updateOrInsertWithAdditions(int sourceInsertPos, int sourceUpdatePos, TxnId updateTxnId, InfoWithAdditions withInfo, TxnId[] newTxnIds, Info[] newInfos)
+    {
+        TxnId[] additions = withInfo.additions;
+        int additionCount = withInfo.additionCount;
+        int additionInsertPos = Arrays.binarySearch(additions, 0, additionCount, updateTxnId);
+        additionInsertPos = Invariants.checkArgument(-1 - additionInsertPos, additionInsertPos < 0);
+        int targetInsertPos = sourceInsertPos + additionInsertPos;
+
+        // additions plus the updateTxnId when necessary
+        TxnId[] missingSource = additions;
+        boolean insertSelfMissing = sourceUpdatePos < 0 && withInfo.info.status.compareTo(COMMITTED) < 0;
+
+        // the most recently constructed pure insert missing array, so that it may be reused if possible
+        TxnId[] cachedMissing = null;
+        int i = 0, j = 0, missingCount = 0, missingLimit = additionCount, count = 0;
+        while (i < infos.length)
+        {
+            if (count == targetInsertPos)
+            {
+                newTxnIds[count] = updateTxnId;
+                newInfos[count] = withInfo.info;
+                if (i == sourceUpdatePos) ++i;
+                else if (insertSelfMissing) ++missingCount;
+                ++count;
+                continue;
+            }
+
+            int c = j == additionCount ? -1 : txnIds[i].compareTo(additions[j]);
+            if (c < 0)
+            {
+                TxnId txnId = txnIds[i];
+                Info info = infos[i];
+                if (i == sourceUpdatePos)
+                {
+                    info = withInfo.info;
+                }
+                else if (info.status.hasDeps())
+                {
+                    Timestamp depsKnownBefore = info.status.depsKnownBefore(txnId, info.executeAt);
+                    if (insertSelfMissing && missingSource == additions && (missingCount != j || (depsKnownBefore != txnId && depsKnownBefore.compareTo(updateTxnId) > 0)))
+                    {
+                        missingSource = mergeMissing(additions, additionCount, updateTxnId, additionInsertPos);
+                        ++missingLimit;
+                    }
+
+                    int to = to(txnId, depsKnownBefore, missingSource, missingCount, missingLimit);
+                    if (to > 0)
+                    {
+                        TxnId[] missing = info.missing == NO_TXNIDS && to == missingCount
+                                          ? cachedMissing = ensureCachedMissing(missingSource, to, cachedMissing)
+                                          : linearUnion(info.missing, info.missing.length, missingSource, to, cachedTxnIds());
+                        info = info.update(txnId, missing);
+                    }
+                }
+                newTxnIds[count] = txnId;
+                newInfos[count] = info;
+                i++;
+            }
+            else if (c > 0)
+            {
+                newTxnIds[count] = additions[j++];
+                ++missingCount;
+                newInfos[count] = TRANSITIVELY_KNOWN.asNoInfo;
+            }
+            else
+            {
+                throw illegalState(txnIds[i] + " should be an insertion, but found match when merging with origin");
+            }
+            count++;
+        }
+
+        if (j < additionCount)
+        {
+            if (count <= targetInsertPos)
+            {
+                int length = targetInsertPos - count;
+                System.arraycopy(additions, j, newTxnIds, count, length);
+                Arrays.fill(newInfos, count, targetInsertPos, TRANSITIVELY_KNOWN.asNoInfo);
+                newTxnIds[targetInsertPos] = updateTxnId;
+                newInfos[targetInsertPos] = withInfo.info;
+                count = targetInsertPos + 1;
+                j = additionInsertPos;
+            }
+            System.arraycopy(additions, j, newTxnIds, count, additionCount - j);
+            Arrays.fill(newInfos, count, count + additionCount - j, TRANSITIVELY_KNOWN.asNoInfo);
+        }
+        else if (count == targetInsertPos)
+        {
+            newTxnIds[targetInsertPos] = updateTxnId;
+            newInfos[targetInsertPos] = withInfo.info;
+        }
+
+        cachedTxnIds().forceDiscard(additions, additionCount);
+        return targetInsertPos;
+    }
+
+    private CommandsForKey update(int pos, Info curInfo, Info newInfo)
+    {
+        if (curInfo == newInfo)
+            return this;
+
+        Info[] newInfos = infos.clone();
+        newInfos[pos] = newInfo;
+        if (curInfo.status.compareTo(COMMITTED) < 0 && newInfo.status.compareTo(COMMITTED) >= 0)
+            removeMissing(txnIds, newInfos, pos);
+        return update(txnIds, newInfos, txnIds[pos], newInfo);
+    }
+
+    /**
+     * Insert a new txnId and info
+     */
+    private CommandsForKey insert(int pos, TxnId newTxnId, Info newInfo)
+    {
+        TxnId[] newTxnIds = new TxnId[txnIds.length + 1];
+        System.arraycopy(txnIds, 0, newTxnIds, 0, pos);
+        newTxnIds[pos] = newTxnId;
+        System.arraycopy(txnIds, pos, newTxnIds, pos + 1, txnIds.length - pos);
+
+        Info[] newInfos = new Info[infos.length + 1];
+        if (newInfo.status.compareTo(COMMITTED) >= 0)
+        {
+            System.arraycopy(infos, 0, newInfos, 0, pos);
+            newInfos[pos] = newInfo;
+            System.arraycopy(infos, pos, newInfos, pos + 1, infos.length - pos);
+        }
+        else
+        {
+            insertInfoAndOneMissing(pos, newTxnId, newInfo, infos, newInfos, newTxnIds, 1);
+        }
+        return update(newTxnIds, newInfos, newTxnId, newInfo);
+    }
+
+    /**
+     * Insert a new txnId and info, then insert the txnId into the missing collection of any command that should have already caused us to witness it
+     */
+    private static void insertInfoAndOneMissing(int insertPos, TxnId txnId, Info newInfo, Info[] oldInfos, Info[] newInfos, TxnId[] newTxnIds, int offsetAfterInsertPos)
+    {
+        TxnId[] oneMissing = null;
+        for (int i = 0 ; i < insertPos ; ++i)
+        {
+            Info oldInfo = oldInfos[i];
+            if (oldInfo.getClass() != NoInfo.class)
+            {
+                Timestamp depsKnownBefore = oldInfo.depsKnownBefore(null);
+                if (depsKnownBefore != null && depsKnownBefore.compareTo(txnId) > 0)
+                {
+                    TxnId[] missing;
+                    if (oldInfo.missing == NO_TXNIDS)
+                        missing = oneMissing = ensureOneMissing(txnId, oneMissing);
+                    else
+                        missing = SortedArrays.insert(oldInfo.missing, txnId, TxnId[]::new);
+
+                    newInfos[i] = Info.create(txnId, oldInfo.status, oldInfo.executeAt, missing);
+                    continue;
+                }
+            }
+            newInfos[i] = oldInfo;
+        }
+
+        newInfos[insertPos] = newInfo;
+
+        for (int i = insertPos; i < oldInfos.length ; ++i)
+        {
+            Info oldInfo = oldInfos[i];
+            if (!oldInfo.status.hasDeps())
+            {
+                newInfos[i + offsetAfterInsertPos] = oldInfo;
+                continue;
+            }
+
+            TxnId[] missing;
+            if (oldInfo.missing == NO_TXNIDS)
+                missing = oneMissing = ensureOneMissing(txnId, oneMissing);
+            else
+                missing = SortedArrays.insert(oldInfo.missing, txnId, TxnId[]::new);
+
+            int newIndex = i + offsetAfterInsertPos;
+            newInfos[newIndex] = Info.create(newTxnIds[newIndex], oldInfo.status, oldInfo.executeAt(newTxnIds[newIndex]), missing);
+        }
+    }
+
+    private static void removeMissing(TxnId[] txnIds, Info[] infos, int pos)
+    {
+        TxnId removeTxnId = txnIds[pos];
+        for (int i = 0 ; i < infos.length ; ++i)
+        {
+            Info info = infos[i];
+            if (info.getClass() == NoInfo.class) continue;
+
+            TxnId[] missing = info.missing;
+            if (missing == NO_TXNIDS) continue;
+
+            // linear scan on object identity going to be faster in practice than binary search almost every time
+            int j = 0;
+            while (j < missing.length && missing[j] != removeTxnId) ++j;
+            if (j == missing.length) continue;
+
+            if (missing.length == 1)
+            {
+                missing = NO_TXNIDS;
+            }
+            else
+            {
+                int length = missing.length;
+                TxnId[] newMissing = new TxnId[length - 1];
+                System.arraycopy(missing, 0, newMissing, 0, j);
+                System.arraycopy(missing, j + 1, newMissing, j, length - (1 + j));
+                missing = newMissing;
+            }
+
+            infos[i] = info.update(txnIds[i], missing);
+        }
+    }
+
+    private static TxnId[] ensureOneMissing(TxnId txnId, TxnId[] oneMissing)
+    {
+        return oneMissing != null ? oneMissing : new TxnId[] { txnId };
+    }
+
+    private static TxnId[] ensureCachedMissing(TxnId[] missingIds, int missingIdCount, TxnId[] cachedMissing)
+    {
+        return cachedMissing != null && cachedMissing.length == missingIdCount ? cachedMissing : Arrays.copyOf(missingIds, missingIdCount);
+    }
+
+    private static TxnId[] mergeMissing(TxnId[] additions, int additionCount, TxnId updateTxnId, int additionInsertPos)
+    {
+        TxnId[] missingSource = new TxnId[additionCount + 1];
+        System.arraycopy(additions, 0, missingSource, 0, additionInsertPos);
+        System.arraycopy(additions, additionInsertPos, missingSource, additionInsertPos + 1, additionCount - additionInsertPos);
+        missingSource[additionInsertPos] = updateTxnId;
+        return missingSource;
+    }
+
+    private static int to(TxnId txnId, Timestamp depsKnownBefore, TxnId[] missingSource, int missingCount, int missingLimit)
+    {
+        if (depsKnownBefore == txnId) return missingCount;
+        int to = Arrays.binarySearch(missingSource, 0, missingLimit, depsKnownBefore);
+        if (to < 0) to = -1 - to;
+        return to;
+    }
+
+    static class InfoWithAdditions
+    {
+        final Info info;
+        final TxnId[] additions;
+        final int additionCount;
+
+        InfoWithAdditions(Info info, TxnId[] additions, int additionCount)
+        {
+            this.info = info;
+            this.additions = additions;
+            this.additionCount = additionCount;
+        }
+    }
+
+    private InfoWithAdditions computeInsert(int insertPos, TxnId txnId, InternalStatus newStatus, Command command)
+    {
+        return computeInfoAndAdditions(insertPos, -1, txnId, newStatus, command);
+    }
+
+    private InfoWithAdditions computeUpdate(int updatePos, TxnId txnId, InternalStatus newStatus, Command command)
+    {
+        return computeInfoAndAdditions(updatePos, updatePos, txnId, newStatus, command);
+    }
+
+    private InfoWithAdditions computeInfoAndAdditions(int insertPos, int updatePos, TxnId txnId, InternalStatus newStatus, Command command)
+    {
+        Timestamp executeAt = txnId;
+        if (newStatus.hasInfo)
+        {
+            executeAt = command.executeAt();
+            if (executeAt.equals(txnId)) executeAt = txnId;
+        }
+        Timestamp depsKnownBefore = newStatus.depsKnownBefore(txnId, executeAt);
+        return computeInfoAndAdditions(insertPos, updatePos, txnId, newStatus, executeAt, depsKnownBefore, command.partialDeps().keyDeps.txnIds(key));
+    }
+
+    private InfoWithAdditions computeInfoAndAdditions(int insertPos, int updatePos, TxnId txnId, InternalStatus newStatus, Timestamp executeAt, Timestamp depsKnownBefore, SortedList<TxnId> deps)
+    {
+        int depsKnownBeforePos;
+        if (depsKnownBefore == txnId)
+        {
+            depsKnownBeforePos = insertPos;
+        }
+        else
+        {
+            depsKnownBeforePos = Arrays.binarySearch(txnIds, insertPos, txnIds.length, depsKnownBefore);
+            Invariants.checkState(depsKnownBeforePos < 0);
+            depsKnownBeforePos = -1 - depsKnownBeforePos;
+        }
+
+        TxnId[] additions = NO_TXNIDS, missing = NO_TXNIDS;
+        int additionCount = 0, missingCount = 0;
+
+        int depsIndex = deps.find(redundantBefore);
+        if (depsIndex < 0) depsIndex = -1 - depsIndex;
+        int txnIdsIndex = 0;
+        while (txnIdsIndex < depsKnownBeforePos && depsIndex < deps.size())
+        {
+            TxnId t = txnIds[txnIdsIndex];
+            TxnId d = deps.get(depsIndex);
+            int c = t.compareTo(d);
+            if (c == 0)
+            {
+                ++txnIdsIndex;
+                ++depsIndex;
+            }
+            else if (c < 0)
+            {
+                // we expect to be missing ourselves
+                // we also permit any transaction we have recorded as COMMITTED or later to be missing, as recovery will not need to consult our information
+                if (txnIdsIndex != updatePos && infos[txnIdsIndex].status.compareTo(COMMITTED) < 0)
+                {
+                    if (missingCount == missing.length)
+                        missing = cachedTxnIds().resize(missing, missingCount, Math.max(8, missingCount * 2));
+                    missing[missingCount++] = t;
+                }
+                txnIdsIndex++;
+            }
+            else
+            {
+                if (additionCount >= additions.length)
+                    additions = cachedTxnIds().resize(additions, additionCount, Math.max(8, additionCount * 2));
+
+                additions[additionCount++] = d;
+                depsIndex++;
+            }
+        }
+
+        while (txnIdsIndex < depsKnownBeforePos)
+        {
+            if (txnIdsIndex != updatePos && infos[txnIdsIndex].status.compareTo(COMMITTED) < 0)
+            {
+                TxnId t = txnIds[txnIdsIndex];
+                if (missingCount == missing.length)
+                    missing = cachedTxnIds().resize(missing, missingCount, Math.max(8, missingCount * 2));
+                missing[missingCount++] = t;
+            }
+            txnIdsIndex++;
+        }
+
+        while (depsIndex < deps.size())
+        {
+            if (additionCount >= additions.length)
+                additions = cachedTxnIds().resize(additions, additionCount, Math.max(8, additionCount * 2));
+            additions[additionCount++] = deps.get(depsIndex++);
+        }
+
+        if (missingCount == 0 && executeAt == txnId)
+            return additionCount == 0 ? newStatus.asNoInfoOrAdditions : new InfoWithAdditions(newStatus.asNoInfo, additions, additionCount);
+
+        return new InfoWithAdditions(Info.create(txnId, newStatus, executeAt, cachedTxnIds().completeAndDiscard(missing, missingCount)), additions, additionCount);
+    }
+
+    private CommandsForKey update(TxnId[] newTxnIds, Info[] newInfos, TxnId updatedTxnId, Info updatedInfo)
+    {
+        Timestamp maxStableWrite = maybeUpdateMaxStableWrite(updatedTxnId, updatedInfo, this.maxStableWrite);
+        return new CommandsForKey(key, redundantBefore, maxStableWrite, newTxnIds, newInfos);
+    }
+
+    private static Timestamp maybeUpdateMaxStableWrite(TxnId txnId, Info info, Timestamp maxStableWrite)
+    {
+        if (!info.status.hasStableDeps() || txnId.kind() != Write)
+            return maxStableWrite;
+
+        Timestamp executeAt = info.executeAt(txnId);
+        if (maxStableWrite != null && maxStableWrite.compareTo(executeAt) >= 0)
+            return maxStableWrite;
+
+        return executeAt;
+    }
+
+    public CommandsForKey withoutRedundant(TxnId redundantBefore)
+    {
+        if (this.redundantBefore.compareTo(redundantBefore) >= 0)
+            return this;
+
+        int pos = insertPos(redundantBefore);
+        if (pos == 0)
+            return new CommandsForKey(key, redundantBefore, maxStableWrite, txnIds, infos);
+
+        TxnId[] newTxnIds = Arrays.copyOfRange(txnIds, pos, txnIds.length);
+        Info[] newInfos = Arrays.copyOfRange(infos, pos, infos.length);
+        for (int i = 0 ; i < newInfos.length ; ++i)
+        {
+            Info info = newInfos[i];
+            if (info.getClass() == NoInfo.class) continue;
+            if (info.missing == NO_TXNIDS) continue;
+            int j = Arrays.binarySearch(info.missing, redundantBefore);
+            if (j < 0) j = -1 - j;
+            if (j <= 0) continue;
+            TxnId[] newMissing = j == info.missing.length ? NO_TXNIDS : Arrays.copyOfRange(info.missing, j, info.missing.length);
+            newInfos[i] = info.update(txnIds[i], newMissing);
+        }
+        return new CommandsForKey(key, redundantBefore, maxStableWrite, newTxnIds, newInfos);
+    }
+
+    public CommandsForKey registerHistorical(TxnId txnId)
+    {
+        int i = Arrays.binarySearch(txnIds, txnId);
+        if (i >= 0)
+            return infos[i].status.compareTo(HISTORICAL) >= 0 ? this : update(i, infos[i], HISTORICAL.asNoInfo);
+
+        return insert(-1 - i, txnId, HISTORICAL.asNoInfo);
+    }
+
+    private int insertPos(Timestamp timestamp)
+    {
+        int i = Arrays.binarySearch(txnIds, timestamp);
+        if (i < 0) i = -1 -i;
+        return i;
+    }
+
+    public TxnId findFirst()
+    {
+        return txnIds.length > 0 ? txnIds[0] : null;
+    }
+
     @Override
     public boolean equals(Object o)
     {
         if (this == o) return true;
         if (o == null || getClass() != o.getClass()) return false;
         CommandsForKey that = (CommandsForKey) o;
-        return key.equals(that.key) && commands.equals(that.commands);
+        return Objects.equals(key, that.key)
+               && Objects.equals(redundantBefore, that.redundantBefore)
+               && Arrays.equals(txnIds, that.txnIds)
+               && Arrays.equals(infos, that.infos);
     }
 
     @Override
-    public final int hashCode()
+    public int hashCode()
     {
         throw new UnsupportedOperationException();
     }
-
-    public Key key()
-    {
-        return key;
-    }
-
-    @Override
-    public CommandTimeseries<?> commands()
-    {
-        return commands;
-    }
-
-    public <D> CommandsForKey update(CommandTimeseries.Update<TxnId, D> commands)
-    {
-        if (commands.isEmpty())
-            return this;
-
-        return new CommandsForKey(key, commands.apply((CommandTimeseries<D>) this.commands));
-    }
-
-    public boolean hasRedundant(TxnId redundantBefore)
-    {
-        return commands.minTimestamp().compareTo(redundantBefore) < 0;
-    }
-
-    public CommandsForKey withoutRedundant(TxnId redundantBefore)
-    {
-        return new CommandsForKey(key, (CommandTimeseries) commands.unbuild().removeBefore(redundantBefore).build());
-    }
-
-    public interface Update {}
-
 }
diff --git a/accord-core/src/main/java/accord/impl/CommandsForKeyGroupUpdater.java b/accord-core/src/main/java/accord/impl/CommandsForKeyGroupUpdater.java
deleted file mode 100644
index 63a3fe8..0000000
--- a/accord-core/src/main/java/accord/impl/CommandsForKeyGroupUpdater.java
+++ /dev/null
@@ -1,227 +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 accord.impl;
-
-import accord.impl.CommandTimeseries.CommandLoader;
-import accord.primitives.Timestamp;
-
-import java.util.Objects;
-
-/**
- * Contains updates for both commands for key sets. One for deps, one for all,
- * and a common set affecting both
- */
-public abstract class CommandsForKeyGroupUpdater<D>
-{
-    public static class Mutable<D> extends CommandsForKeyGroupUpdater<D>
-    {
-        private final CommandsForKeyUpdater.Mutable<D> deps;
-        private final CommandsForKeyUpdater.Mutable<D> all;
-        private final CommandsForKeyUpdater.Mutable<D> common;
-
-        public Mutable(CommandsForKeyUpdater.Mutable<D> deps, CommandsForKeyUpdater.Mutable<D> all, CommandsForKeyUpdater.Mutable<D> common)
-        {
-            this.deps = deps;
-            this.all = all;
-            this.common = common;
-        }
-
-        public Mutable(CommandLoader<D> loader)
-        {
-            this(new CommandsForKeyUpdater.Mutable<>(loader),
-                 new CommandsForKeyUpdater.Mutable<>(loader),
-                 new CommandsForKeyUpdater.Mutable<>(loader));
-        }
-
-        @Override
-        public CommandsForKeyUpdater.Mutable<D> deps()
-        {
-            return deps;
-        }
-
-        @Override
-        public CommandsForKeyUpdater.Mutable<D> all()
-        {
-            return all;
-        }
-
-        @Override
-        public CommandsForKeyUpdater.Mutable<D> common()
-        {
-            return common;
-        }
-
-        public Immutable<D> toImmutable()
-        {
-            return new Immutable<>(deps.toImmutable(), all.toImmutable(), common.toImmutable());
-        }
-    }
-
-    public static class Immutable<D> extends CommandsForKeyGroupUpdater<D>
-    {
-        private final CommandsForKeyUpdater.Immutable<D> deps;
-        private final CommandsForKeyUpdater.Immutable<D> all;
-        private final CommandsForKeyUpdater.Immutable<D> common;
-
-        public interface Factory<D, T extends Immutable<D>>
-        {
-            T create(CommandsForKeyUpdater.Immutable<D> deps, CommandsForKeyUpdater.Immutable<D> all, CommandsForKeyUpdater.Immutable<D> common);
-        }
-
-        public static <T> Factory<T, Immutable<T>> getFactory()
-        {
-            return Immutable::new;
-        }
-
-        public Immutable(CommandsForKeyUpdater.Immutable<D> deps, CommandsForKeyUpdater.Immutable<D> all, CommandsForKeyUpdater.Immutable<D> common)
-        {
-            this.deps = deps;
-            this.all = all;
-            this.common = common;
-        }
-
-        @Override
-        public String toString()
-        {
-            return "Immutable{" +
-                    "deps=" + deps +
-                    ", all=" + all +
-                    ", common=" + common +
-                    '}';
-        }
-
-        @Override
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            Immutable<?> immutable = (Immutable<?>) o;
-            return Objects.equals(deps, immutable.deps) && Objects.equals(all, immutable.all) && Objects.equals(common, immutable.common);
-        }
-
-        @Override
-        public int hashCode()
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public CommandsForKeyUpdater.Immutable<D> deps()
-        {
-            return deps;
-        }
-
-        @Override
-        public CommandsForKeyUpdater.Immutable<D> all()
-        {
-            return all;
-        }
-
-        @Override
-        public CommandsForKeyUpdater.Immutable<D> common()
-        {
-            return common;
-        }
-
-        public Mutable<D> toMutable(CommandLoader<D> loader)
-        {
-            return new Mutable<>(deps.toMutable(loader), all.toMutable(loader), common.toMutable(loader));
-        }
-
-        private static <T extends Timestamp, D> void mergeCommon(CommandTimeseries.ImmutableUpdate<T, D> newCommon,
-                                                                 CommandTimeseries.MutableUpdate<T, D> mergedCommon,
-                                                                 CommandTimeseries.MutableUpdate<T, D> mergedDeps,
-                                                                 CommandTimeseries.MutableUpdate<T, D> mergedAll)
-        {
-            newCommon.forEachWrite((k, v) -> {
-                mergedCommon.addWriteUnsafe(k, v);
-                mergedDeps.removeKeyUnsafe(k);
-                mergedAll.removeKeyUnsafe(k);
-            });
-            newCommon.forEachDelete(k -> {
-                mergedCommon.remove(k);
-                mergedDeps.removeKeyUnsafe(k);
-                mergedAll.removeKeyUnsafe(k);
-            });
-        }
-
-        private static <T extends Timestamp, D> void mergeSpecific(CommandTimeseries.ImmutableUpdate<T, D> newSpecific,
-                                                                   CommandTimeseries.MutableUpdate<T, D> mergedSpecific, CommandTimeseries.MutableUpdate<T, D> mergedOther, CommandTimeseries.MutableUpdate<T, D> mergedCommon)
-        {
-            newSpecific.forEachWrite((k, v) -> {
-                mergedSpecific.addWriteUnsafe(k, v);
-                mergedCommon.transferKeyTo(k, mergedOther);
-            });
-
-            newSpecific.forEachDelete(k -> {
-                mergedSpecific.remove(k);
-                mergedCommon.transferKeyTo(k, mergedOther);
-            });
-        }
-
-        public static  <D, T extends Immutable<D>> T merge(T current, T update, Factory<D, T> factory)
-        {
-            if (current == null || current.isEmpty())
-                return update;
-
-            if (update == null || update.isEmpty())
-                return current;
-
-            Mutable<D> merged = current.toMutable(null);
-
-            mergeCommon(update.common().commands(), merged.common.commands(), merged.deps.commands(), merged.all.commands());
-
-            mergeSpecific(update.deps().commands(), merged.deps.commands(), merged.all.commands(), merged.common.commands());
-            mergeSpecific(update.all().commands(), merged.all.commands(), merged.deps.commands(), merged.common.commands());
-
-            return factory.create(merged.deps.toImmutable(), merged.all.toImmutable(), merged.common.toImmutable());
-        }
-    }
-
-    public abstract CommandsForKeyUpdater<D> deps();
-
-    public abstract CommandsForKeyUpdater<D> all();
-
-    /**
-     * Update the deps and all CFK groups.
-     * In case conflicting updates have been applied to the common udpater and a specific one, the specific one should take precedence
-     */
-    public abstract CommandsForKeyUpdater<D> common();
-
-    public boolean isEmpty()
-    {
-        return deps().isEmpty() && all().isEmpty() && common().isEmpty();
-    }
-
-    public CommandsForKey applyToDeps(CommandsForKey current)
-    {
-        current = common().apply(current);
-        current = deps().apply(current);
-
-        return current;
-    }
-
-    public CommandsForKey applyToAll(CommandsForKey current)
-    {
-        current = common().apply(current);
-        current = all().apply(current);
-
-        return current;
-    }
-}
diff --git a/accord-core/src/main/java/accord/impl/CommandsForKeyUpdater.java b/accord-core/src/main/java/accord/impl/CommandsForKeyUpdater.java
deleted file mode 100644
index c46c327..0000000
--- a/accord-core/src/main/java/accord/impl/CommandsForKeyUpdater.java
+++ /dev/null
@@ -1,117 +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 accord.impl;
-
-import accord.impl.CommandTimeseries.CommandLoader;
-import accord.primitives.TxnId;
-
-import java.util.Objects;
-
-/**
- * Contains updates for a single commands for key dataset (ie: deps OR all)
- * @param <D>
- */
-public abstract class CommandsForKeyUpdater<D>
-{
-    public abstract CommandTimeseries.Update<TxnId, D> commands();
-
-    public static class Mutable<D> extends CommandsForKeyUpdater<D>
-    {
-        private final CommandTimeseries.MutableUpdate<TxnId, D> commands;
-
-        Mutable(CommandTimeseries.MutableUpdate<TxnId, D> commands)
-        {
-            this.commands = commands;
-        }
-
-        public Mutable(CommandLoader<D> loader)
-        {
-            this(new CommandTimeseries.MutableUpdate<>(loader));
-        }
-
-        @Override public CommandTimeseries.MutableUpdate<TxnId, D> commands() { return commands; }
-        public Immutable<D> toImmutable()
-        {
-            return new Immutable<>(commands.toImmutable());
-        }
-    }
-
-    public static class Immutable<D> extends CommandsForKeyUpdater<D>
-    {
-        private static final Immutable<Object> EMPTY = new Immutable<>(CommandTimeseries.ImmutableUpdate.empty());
-        private final CommandTimeseries.ImmutableUpdate<TxnId, D> commands;
-
-        public Immutable(CommandTimeseries.ImmutableUpdate<TxnId, D> commands)
-        {
-            this.commands = commands;
-        }
-
-        @Override
-        public String toString()
-        {
-            return "Immutable{" +
-                    "commands=" + commands +
-                    '}';
-        }
-
-        @Override
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            Immutable<?> immutable = (Immutable<?>) o;
-            return Objects.equals(commands, immutable.commands);
-        }
-
-        @Override
-        public int hashCode()
-        {
-            return Objects.hash(commands);
-        }
-
-        public static <D> Immutable<D> empty()
-        {
-            return (Immutable<D>) EMPTY;
-        }
-
-        @Override public CommandTimeseries.ImmutableUpdate<TxnId, D> commands() { return commands; }
-
-        public Mutable<D> toMutable(CommandLoader<D> loader)
-        {
-            return new Mutable<>(commands.toMutable(loader));
-        }
-    }
-
-    public int totalChanges()
-    {
-        return commands().numChanges();
-    }
-
-    public boolean isEmpty()
-    {
-        return commands().isEmpty();
-    }
-
-    public CommandsForKey apply(CommandsForKey current)
-    {
-        if (commands().isEmpty())
-            return current;
-        return current.update(commands());
-    }
-}
diff --git a/accord-core/src/main/java/accord/impl/CommandsForKeys.java b/accord-core/src/main/java/accord/impl/CommandsForKeys.java
index 56d5b9b..3ab2b7b 100644
--- a/accord-core/src/main/java/accord/impl/CommandsForKeys.java
+++ b/accord-core/src/main/java/accord/impl/CommandsForKeys.java
@@ -19,16 +19,13 @@
 package accord.impl;
 
 import accord.api.VisibleForImplementation;
-import accord.local.Command;
 import accord.local.CommandStore;
 import accord.primitives.RoutableKey;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
-import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static accord.local.Command.NotDefined.uninitialised;
 
 public class CommandsForKeys
 {
@@ -36,77 +33,10 @@
 
     private CommandsForKeys() {}
 
-    @VisibleForTesting
-    @VisibleForImplementation
-    public static TimestampsForKey updateMax(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey key, Timestamp timestamp)
+    public static void registerNotWitnessed(AbstractSafeCommandStore<?,?,?> safeStore, RoutableKey key, TxnId txnId)
     {
-        SafeTimestampsForKey tfk = safeStore.timestampsForKey(key);
-        return tfk.updateMax(timestamp);
-    }
-
-    public static Command.DurableAndIdempotentListener registerCommand(SafeTimestampsForKey timestamps, SafeCommandsForKey.Update<?,?> update, Command command)
-    {
-
-        update.common().commands().add(command.txnId(), command);
-
-        timestamps.updateMax(command.executeAt());
-
-        return new CommandsForKey.Listener(timestamps.key());
-    }
-
-    public static Command.DurableAndIdempotentListener registerCommand(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey key, Command command)
-    {
-        return registerCommand(safeStore.timestampsForKey(key), safeStore.getOrCreateCommandsForKeyUpdate(key), command);
-    }
-
-    public static void registerNotWitnessed(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey key, TxnId txnId)
-    {
-        SafeTimestampsForKey tfk = safeStore.timestampsForKey(key);
-        SafeCommandsForKey cfk = safeStore.depsCommandsForKey(key);
-
-        // FIXME: should be the recovery commands
-        if (cfk.current().commands().commands.containsKey(txnId))
-            return;
-
-        tfk.updateMax(txnId);
-        SafeCommandsForKey.Update<?,?> update = safeStore.getOrCreateCommandsForKeyUpdate(key);
-
-        update.common().commands().add(txnId, uninitialised(txnId));
-    }
-
-    public static void listenerUpdate(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey listenerKey, Command command)
-    {
-        if (logger.isTraceEnabled())
-            logger.trace("[{}]: updating as listener in response to change on {} with status {} ({})",
-                         listenerKey, command.txnId(), command.status(), command);
-
-        SafeTimestampsForKey tfk = safeStore.timestampsForKey(listenerKey);
-        SafeCommandsForKey.Update<?,?> update = safeStore.getOrCreateCommandsForKeyUpdate(listenerKey);
-
-        // add/remove the command on every listener update to avoid
-        // special denormalization handling in Cassandra
-        switch (command.status())
-        {
-            default: throw new AssertionError();
-            case PreAccepted:
-            case NotDefined:
-            case Accepted:
-            case AcceptedInvalidate:
-            case PreCommitted:
-            case Applied:
-            case PreApplied:
-            case Committed:
-            case Stable:
-            case ReadyToExecute:
-                update.common().commands().add(command.txnId(), command);
-                break;
-            case Invalidated:
-                update.common().commands().remove(command.txnId());
-            case Truncated:
-                break;
-        }
-
-        tfk.updateMax(command.executeAt());
+        SafeCommandsForKey cfk = safeStore.commandsForKey(key);
+        cfk.registerHistorical(txnId);
     }
 
     public static TimestampsForKey updateLastExecutionTimestamps(CommandStore commandStore, SafeTimestampsForKey tfk, Timestamp executeAt, boolean isForWriteTxn)
@@ -146,7 +76,7 @@
     }
 
     @VisibleForImplementation
-    public static <D> TimestampsForKey updateLastExecutionTimestamps(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey key, Timestamp executeAt, boolean isForWriteTxn)
+    public static <D> TimestampsForKey updateLastExecutionTimestamps(AbstractSafeCommandStore<?,?,?> safeStore, RoutableKey key, Timestamp executeAt, boolean isForWriteTxn)
     {
         return updateLastExecutionTimestamps(safeStore.commandStore(), safeStore.timestampsForKey(key), executeAt, isForWriteTxn);
     }
diff --git a/accord-core/src/main/java/accord/impl/CommandsSummary.java b/accord-core/src/main/java/accord/impl/CommandsSummary.java
new file mode 100644
index 0000000..7793c6e
--- /dev/null
+++ b/accord-core/src/main/java/accord/impl/CommandsSummary.java
@@ -0,0 +1,41 @@
+/*
+ * 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 accord.impl;
+
+import accord.local.SafeCommandStore.CommandFunction;
+import accord.local.SafeCommandStore.TestDep;
+import accord.local.SafeCommandStore.TestStartedAt;
+import accord.local.SafeCommandStore.TestStatus;
+import accord.primitives.Timestamp;
+import accord.primitives.Txn.Kind.Kinds;
+import accord.primitives.TxnId;
+
+public interface CommandsSummary
+{
+    <P1, T> T mapReduceFull(TxnId testTxnId,
+                            Kinds testKind,
+                            TestStartedAt testStartedAt,
+                            TestDep testDep,
+                            TestStatus testStatus,
+                            CommandFunction<P1, T, T> map, P1 p1, T initialValue);
+
+    <P1, T> T mapReduceActive(Timestamp startedBefore,
+                              Kinds testKind,
+                              CommandFunction<P1, T, T> map, P1 p1, T initialValue);
+}
diff --git a/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java b/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java
index e4db999..df39788 100644
--- a/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java
+++ b/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java
@@ -29,9 +29,11 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import accord.api.Scheduler;
 import accord.coordinate.CoordinateGloballyDurable;
 import accord.coordinate.CoordinateShardDurable;
 import accord.coordinate.CoordinateSyncPoint;
+import accord.coordinate.ExecuteSyncPoint.SyncPointErased;
 import accord.local.Node;
 import accord.local.ShardDistributor;
 import accord.primitives.Range;
@@ -69,7 +71,6 @@
  * The work for CoordinateShardDurable is further subdivided where each subrange a node operates on is divided a fixed
  * number of times and then processed one at a time with a fixed wait between them.
  *
- * // TODO review will the scheduler shut down on its own or do the individual tasks need to be canceled manually?
  * // TODO (expected): cap number of coordinations we can have in flight at once
  * Didn't go with recurring because it doesn't play well with async execution of these tasks
  */
@@ -78,6 +79,7 @@
     private static final Logger logger = LoggerFactory.getLogger(CoordinateDurabilityScheduling.class);
 
     private final Node node;
+    private Scheduler.Scheduled scheduled;
 
     /*
      * In each round at each node wait this amount of time between initiating new CoordinateShardDurable
@@ -156,26 +158,22 @@
     /**
      * Schedule regular invocations of CoordinateShardDurable and CoordinateGloballyDurable
      */
-    public void start()
+    public synchronized void start()
     {
         Invariants.checkState(!stop); // cannot currently restart safely
         long nowMicros = node.unix(MICROSECONDS);
         prevShardSyncTimeMicros = nowMicros;
         setNextGlobalSyncTime(nowMicros);
-        requeue();
+        scheduled = node.scheduler().recurring(this::run, frequencyMicros, MICROSECONDS);
     }
 
     public void stop()
     {
+        if (scheduled != null)
+            scheduled.cancel();
         stop = true;
     }
 
-    private void requeue()
-    {
-        if (!stop)
-            node.scheduler().once(this::run, frequencyMicros, MICROSECONDS);
-    }
-
     /**
      * Schedule the first CoordinateShardDurable execution for the current round. Sub-steps will be scheduled after
      * each sub-step completes, and once all are completed scheduleCoordinateShardDurable is called again.
@@ -185,35 +183,28 @@
         if (stop)
             return;
 
-        try
+        updateTopology();
+        if (currentGlobalTopology == null || currentGlobalTopology.size() == 0)
+            return;
+
+        long nowMicros = node.unix(MICROSECONDS);
+        if (nextGlobalSyncTimeMicros <= nowMicros)
         {
-            updateTopology();
-            if (currentGlobalTopology.size() == 0)
-                return;
-
-            long nowMicros = node.unix(MICROSECONDS);
-            if (nextGlobalSyncTimeMicros <= nowMicros)
-            {
-                startGlobalSync();
-                setNextGlobalSyncTime(nowMicros);
-            }
-
-            List<Ranges> coordinate = rangesToShardSync(nowMicros);
-            prevShardSyncTimeMicros = nowMicros;
-            if (coordinate.isEmpty())
-            {
-                logger.trace("Nothing pending in schedule for time slot at {}", nowMicros);
-                return;
-            }
-
-            logger.trace("Scheduling CoordinateShardDurable for {} at {}", coordinate, nowMicros);
-            for (Ranges ranges : coordinate)
-                startShardSync(ranges);
+            startGlobalSync();
+            setNextGlobalSyncTime(nowMicros);
         }
-        finally
+
+        List<Ranges> coordinate = rangesToShardSync(nowMicros);
+        prevShardSyncTimeMicros = nowMicros;
+        if (coordinate.isEmpty())
         {
-            requeue();
+            logger.trace("Nothing pending in schedule for time slot at {}", nowMicros);
+            return;
         }
+
+        logger.trace("Scheduling CoordinateShardDurable for {} at {}", coordinate, nowMicros);
+        for (Ranges ranges : coordinate)
+            startShardSync(ranges);
     }
 
     /**
@@ -238,7 +229,7 @@
             node.commandStores().any().execute(() -> {
                 CoordinateShardDurable.coordinate(node, exclusiveSyncPoint)
                                       .addCallback((success, fail) -> {
-                                          if (fail != null)
+                                          if (fail != null && fail.getClass() != SyncPointErased.class)
                                           {
                                               logger.trace("Exception coordinating local shard durability, will retry immediately", fail);
                                               coordinateShardDurableAfterExclusiveSyncPoint(node, exclusiveSyncPoint);
@@ -309,7 +300,6 @@
             return;
 
         currentGlobalTopology = latestGlobal;
-
         List<Node.Id> ids = new ArrayList<>(latestGlobal.nodes());
         Collections.sort(ids);
         globalIndex = ids.indexOf(node.id());
diff --git a/accord-core/src/main/java/accord/impl/DepsBuilder.java b/accord-core/src/main/java/accord/impl/DepsBuilder.java
deleted file mode 100644
index 75e17d1..0000000
--- a/accord-core/src/main/java/accord/impl/DepsBuilder.java
+++ /dev/null
@@ -1,176 +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 accord.impl;
-
-import accord.local.SafeCommandStore;
-import accord.local.Status;
-import accord.primitives.*;
-
-import java.util.*;
-import java.util.function.Consumer;
-
-public class DepsBuilder
-{
-    private static boolean CAN_PRUNE = Boolean.getBoolean("accord.can_prune_deps");
-    private final TxnId subject;
-    private final Map<Seekable, Builder> builders = new HashMap<>();
-
-    public static void setCanPruneUnsafe(boolean value)
-    {
-        CAN_PRUNE = value;
-    }
-
-    public static boolean getCanPrune()
-    {
-        return CAN_PRUNE;
-    }
-
-    public DepsBuilder(TxnId subject)
-    {
-        this.subject = subject;
-    }
-
-    private static class TxnInfo
-    {
-        private final TxnId txnId;
-        private final Status status;
-        private final Timestamp executeAt;
-        private final List<TxnId> depsIds;
-        private boolean required = false;
-
-        public TxnInfo(TxnId txnId, Status status, Timestamp executeAt, List<TxnId> depsIds)
-        {
-            this.txnId = txnId;
-            this.status = status;
-            this.executeAt = executeAt;
-            this.depsIds = depsIds;
-        }
-    }
-
-    /**
-     * To remove transitive dependencies, we filter out commands that exist in the deps list
-     * We need commands that are not superseded by other commands with later execution times
-     */
-    public static class Builder
-    {
-        private final Map<TxnId, TxnInfo> commands = new HashMap<>();
-        private final Map<TxnId, Set<TxnId>> supportingDeps = new HashMap<>();
-
-        public void add(TxnId txnId, Status status, Timestamp executeAt, List<TxnId> depsIds)
-        {
-            commands.put(txnId, new TxnInfo(txnId, status, executeAt, depsIds));
-            if (depsIds != null)
-                depsIds.forEach(id -> supportingDeps.computeIfAbsent(id, i -> new HashSet<>()).add(txnId));
-        }
-
-        private void markRequired(TxnInfo info)
-        {
-            if (info.required)
-                return;
-
-            info.required = true;
-
-            Set<TxnId> supportingIds = supportingDeps.get(info.txnId);
-            if (supportingIds == null || supportingIds.isEmpty())
-                return;
-
-            for (TxnId supportingId : supportingIds)
-            {
-                TxnInfo supporting = commands.get(supportingId);
-                if (supporting == null)
-                    continue;
-
-                markRequired(supporting);
-            }
-        }
-
-        private boolean isRequired(TxnInfo info)
-        {
-            if (!CAN_PRUNE || info.required)
-                return true;
-
-            if (!info.status.hasBeen(Status.Applied))
-                return true;
-
-            Set<TxnId> supportingIds = supportingDeps.get(info.txnId);
-            if (supportingIds == null || supportingIds.isEmpty())
-                return true;
-
-            for (TxnId supportingId : supportingIds)
-            {
-                TxnInfo supporting = commands.get(supportingId);
-
-                // TODO: check for eviction of txnId and skip if it's evicted
-
-                if (!supporting.status.hasBeen(Status.Applied))
-                    return true;
-
-                if (supporting.executeAt.compareTo(info.executeAt) < 0)
-                    return true;
-
-                // FIXME: is this actually needed?
-                // let execution order strongly connected components
-                if (info.depsIds.contains(supportingId))
-                    return true;
-            }
-
-            return false;
-        }
-
-        private void preprocess(TxnInfo info)
-        {
-            if (isRequired(info))
-                markRequired(info);
-        }
-
-        private void preprocess()
-        {
-            commands.values().forEach(this::preprocess);
-        }
-
-        public <T> void build(SafeCommandStore safeStore, Seekable seekable, Consumer<TxnId> consumer)
-        {
-            preprocess();
-            commands.forEach(((txnId, info) -> {
-                if (info.required)
-                    consumer.accept(txnId);
-                else
-                    safeStore.removeCommandFromSeekableDeps(seekable, txnId, info.executeAt, info.status);
-            }));
-        }
-    }
-
-    public void add(TxnId txnId, Seekable seekable, Status status, Timestamp executeAt, List<TxnId> depsIds)
-    {
-        // don't add self to deps
-        if (txnId.equals(subject))
-            return;
-
-        builders.computeIfAbsent(seekable, s -> new Builder()).add(txnId, status, executeAt, depsIds);
-    }
-
-    public PartialDeps buildPartialDeps(SafeCommandStore safeStore, Ranges ranges)
-    {
-        try (PartialDeps.Builder result = new PartialDeps.Builder(ranges);)
-        {
-            builders.forEach((seekable, builder) -> builder.build(safeStore, seekable, txnId -> result.add(seekable, txnId)));
-            return result.build();
-        }
-    }
-}
diff --git a/accord-core/src/main/java/accord/impl/DomainCommands.java b/accord-core/src/main/java/accord/impl/DomainCommands.java
deleted file mode 100644
index ee1423f..0000000
--- a/accord-core/src/main/java/accord/impl/DomainCommands.java
+++ /dev/null
@@ -1,26 +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 accord.impl;
-
-public interface DomainCommands
-{
-    enum Kind { COORDINATION, RECOVERY }
-
-    CommandTimeseries<?> commands();
-}
diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
index 42c043c..9b49cdf 100644
--- a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
+++ b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
@@ -26,7 +26,6 @@
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
-import java.util.Objects;
 import java.util.Queue;
 import java.util.Set;
 import java.util.TreeMap;
@@ -35,15 +34,16 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
 import java.util.function.Consumer;
 import java.util.function.Function;
-import java.util.function.Predicate;
+import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
 import com.google.common.annotations.VisibleForTesting;
 
 import accord.local.*;
-import accord.utils.TriFunction;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,7 +51,6 @@
 import accord.api.DataStore;
 import accord.api.Key;
 import accord.api.ProgressLog;
-import accord.impl.CommandTimeseries.CommandLoader;
 import accord.local.CommandStores.RangesForEpoch;
 import accord.primitives.AbstractKeys;
 import accord.primitives.Deps;
@@ -60,6 +59,7 @@
 import accord.primitives.Range;
 import accord.primitives.Ranges;
 import accord.primitives.Routable;
+import accord.primitives.Routable.Domain;
 import accord.primitives.RoutableKey;
 import accord.primitives.Routables;
 import accord.primitives.Seekable;
@@ -71,12 +71,13 @@
 import accord.utils.async.AsyncChain;
 import accord.utils.async.AsyncChains;
 
-import static accord.local.Command.NotDefined.uninitialised;
-
 import static accord.local.SafeCommandStore.TestDep.ANY_DEPS;
 import static accord.local.SafeCommandStore.TestDep.WITH;
+import static accord.local.SafeCommandStore.TestStatus.ANY_STATUS;
+import static accord.local.SafeCommandStore.TestStartedAt.STARTED_BEFORE;
 import static accord.local.SaveStatus.Applying;
 import static accord.local.SaveStatus.ReadyToExecute;
+import static accord.local.Status.Applied;
 import static accord.local.Status.Stable;
 import static accord.local.Status.Truncated;
 import static accord.local.Status.NotDefined;
@@ -92,8 +93,7 @@
     final NavigableMap<TxnId, GlobalCommand> commands = new TreeMap<>();
     final NavigableMap<Timestamp, GlobalCommand> commandsByExecuteAt = new TreeMap<>();
     private final NavigableMap<RoutableKey, GlobalTimestampsForKey> timestampsForKey = new TreeMap<>();
-    private final NavigableMap<RoutableKey, GlobalCommandsForKey> depsCommandsForKey = new TreeMap<>();
-    private final NavigableMap<RoutableKey, GlobalCommandsForKey> allCommandsForKey = new TreeMap<>();
+    private final NavigableMap<RoutableKey, GlobalCommandsForKey> commandsForKey = new TreeMap<>();
     // TODO (find library, efficiency): this is obviously super inefficient, need some range map
 
     private final TreeMap<TxnId, RangeCommand> rangeCommands = new TreeMap<>();
@@ -148,7 +148,6 @@
     {
         if (CHECK_DEPENDENCY_INVARIANTS)
         {
-
             newGlobalCommand.addListener(new Command.TransientListener()
             {
                 @Override
@@ -197,7 +196,7 @@
 
                             if (prev.txnId().isWrite())
                                 participants = participants.subtract(intersectingParticipants);
-                        };
+                        }
                     }
                 }
 
@@ -229,34 +228,19 @@
         return commands.containsKey(txnId);
     }
 
-    public GlobalCommandsForKey depsCommandsForKeyIfPresent(Key key)
+    public GlobalCommandsForKey commandsForKeyIfPresent(Key key)
     {
-        return depsCommandsForKey.get(key);
+        return commandsForKey.get(key);
     }
 
-    public GlobalCommandsForKey depsCommandsForKey(Key key)
+    public GlobalCommandsForKey commandsForKey(Key key)
     {
-        return depsCommandsForKey.computeIfAbsent(key, GlobalCommandsForKey::new);
+        return commandsForKey.computeIfAbsent(key, GlobalCommandsForKey::new);
     }
 
-    public boolean hasDepsCommandsForKey(Key key)
+    public boolean hasCommandsForKey(Key key)
     {
-        return depsCommandsForKey.containsKey(key);
-    }
-
-    public GlobalCommandsForKey allCommandsForKeyIfPresent(Key key)
-    {
-        return allCommandsForKey.get(key);
-    }
-
-    public GlobalCommandsForKey allCommandsForKey(Key key)
-    {
-        return allCommandsForKey.computeIfAbsent(key, GlobalCommandsForKey::new);
-    }
-
-    public boolean hasAllCommandsForKey(Key key)
-    {
-        return allCommandsForKey.containsKey(key);
+        return commandsForKey.containsKey(key);
     }
 
     public GlobalTimestampsForKey timestampsForKey(Key key)
@@ -269,60 +253,7 @@
         return timestampsForKey.get(key);
     }
 
-    public CommonAttributes register(InMemorySafeStore safeStore, Seekables<?, ?> keysOrRanges, Ranges slice, SafeCommand command, CommonAttributes attrs)
-    {
-        switch (keysOrRanges.domain())
-        {
-            default: throw new AssertionError();
-            case Key:
-                CommonAttributes.Mutable mutable = attrs.mutable();
-                forEach(keysOrRanges, slice, key -> {
-                    Command.DurableAndIdempotentListener listener = CommandsForKeys.registerCommand(safeStore, key, command.current());
-                    mutable.addListener(listener);
-                });
-                return mutable;
-            case Range:
-                // TODO (required): confirm we slice to owned ranges on C*
-                rangeCommands.computeIfAbsent(command.txnId(), ignore -> new RangeCommand(commands.get(command.txnId())))
-                        .update(((Ranges)keysOrRanges).slice(slice, Minimal));
-        }
-        return attrs;
-    }
-
-    public CommonAttributes register(InMemorySafeStore safeStore, Seekable keyOrRange, Ranges slice, SafeCommand command, CommonAttributes attrs)
-    {
-        switch (keyOrRange.domain())
-        {
-            default: throw new AssertionError();
-            case Key:
-                CommonAttributes.Mutable mutable = attrs.mutable();
-                forEach(keyOrRange, slice, key -> {
-                    Command.DurableAndIdempotentListener listener = CommandsForKeys.registerCommand(safeStore, key, command.current());
-                    mutable.addListener(listener);
-                });
-                return mutable;
-            case Range:
-                // TODO (required): confirm we slice to owned ranges on C*
-                rangeCommands.computeIfAbsent(command.txnId(), ignore -> new RangeCommand(commands.get(command.txnId())))
-                        .update(Ranges.of((Range)keyOrRange).slice(slice, Minimal));
-        }
-        return attrs;
-    }
-
-    private NavigableMap<RoutableKey, GlobalCommandsForKey> globalCommandsMapFor(KeyHistory keyHistory)
-    {
-        switch (keyHistory)
-        {
-            case DEPS:
-                return depsCommandsForKey;
-            case ALL:
-                return allCommandsForKey;
-            default:
-                throw new IllegalStateException(keyHistory + " does not have a commands map");
-        }
-    }
-
-    private <O> O mapReduceForKey(InMemorySafeStore safeStore, Routables<?> keysOrRanges, Ranges slice, KeyHistory keyHistory, TriFunction<TimestampsForKey, CommandsForKey, O, O> map, O accumulate, Predicate<? super O> terminate)
+    private <O> O mapReduceForKey(InMemorySafeStore safeStore, Routables<?> keysOrRanges, Ranges slice, BiFunction<CommandsForKey, O, O> map, O accumulate)
     {
         switch (keysOrRanges.domain()) {
             default:
@@ -332,27 +263,11 @@
                 for (Key key : keys)
                 {
                     if (!slice.contains(key)) continue;
-                    SafeTimestampsForKey timestamps = safeStore.timestampsIfLoadedAndInitialised(key);
-                    if (timestamps.current() == null)
+                    CommandsForKey commands = safeStore.commandsIfLoadedAndInitialised(key).current();
+                    if (commands == null)
                         continue;
 
-                    CommandsForKey commands = null;
-                    switch (keyHistory)
-                    {
-                        case DEPS:
-                            commands = safeStore.depsCommandsIfLoadedAndInitialised(key).current();
-                            break;
-                        case ALL:
-                            commands = safeStore.allCommandsIfLoadedAndInitialised(key).current();
-                            break;
-                    }
-
-                    if (commands == null && keyHistory != KeyHistory.NONE)
-                        continue;
-
-                    accumulate = map.apply(timestamps.current(), commands, accumulate);
-                    if (terminate.test(accumulate))
-                        return accumulate;
+                    accumulate = map.apply(commands, accumulate);
                 }
                 break;
             case Range:
@@ -362,28 +277,13 @@
                 {
                     // TODO (required): this method should fail if it requires more info than available
                     // TODO (required): I don't think this can possibly work in C*, as we don't know which timestampsForKey we need
-                    NavigableMap<RoutableKey, GlobalCommandsForKey> commandsMap = keyHistory.isNone() ? null : globalCommandsMapFor(keyHistory);
-                    for (Map.Entry<RoutableKey, GlobalTimestampsForKey> entry : timestampsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive()).entrySet())
+                    for (Map.Entry<RoutableKey, GlobalCommandsForKey> entry : commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive()).entrySet())
                     {
-                        RoutableKey key = entry.getKey();
-                        TimestampsForKey timestamps = entry.getValue().value();
-
-                        GlobalCommandsForKey globalCommands = keyHistory.isNone() ? null : commandsMap.get(key);
-                        if (timestamps == null)
-                        {
-                            Invariants.checkState(globalCommands == null || globalCommands.value() == null);
+                        GlobalCommandsForKey globalCommands = entry.getValue();
+                        CommandsForKey commands = globalCommands.value();
+                        if (commands == null)
                             continue;
-                        }
-
-                        // TODO (required): what if consumer wants timestamps independently? not safe, should declare separately
-                        if (globalCommands == null && !keyHistory.isNone())
-                            continue;
-
-                        CommandsForKey commands = keyHistory.isNone() ? null : Invariants.nonNull(globalCommands.value());
-
-                        accumulate = map.apply(timestamps, commands, accumulate);
-                        if (terminate.test(accumulate))
-                            return accumulate;
+                        accumulate = map.apply(commands, accumulate);
                     }
                 }
         }
@@ -402,10 +302,8 @@
             case Range:
                 Ranges ranges = (Ranges) keysOrRanges;
                 ranges.slice(slice).forEach(range -> {
-                    depsCommandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive())
-                            .keySet().forEach(forEach);
-                    allCommandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive())
-                            .keySet().forEach(forEach);
+                    commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive())
+                                  .keySet().forEach(forEach);
                 });
         }
     }
@@ -423,10 +321,8 @@
             case Range:
                 Range range = (Range) keyOrRange;
                 Ranges.of(range).slice(slice).forEach(r -> {
-                    depsCommandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive())
-                            .keySet().forEach(forEach);
-                    allCommandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive())
-                            .keySet().forEach(forEach);
+                    commandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive())
+                                  .keySet().forEach(forEach);
                 });
         }
     }
@@ -457,11 +353,7 @@
             }
         });
         ranges.forEach(r -> {
-            depsCommandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive()).values().forEach(forKey -> {
-                if (!forKey.isEmpty())
-                    forKey.value(forKey.value().withoutRedundant(syncId));
-            });
-            allCommandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive()).values().forEach(forKey -> {
+            commandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive()).values().forEach(forKey -> {
                 if (!forKey.isEmpty())
                     forKey.value(forKey.value().withoutRedundant(syncId));
             });
@@ -471,35 +363,24 @@
     protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges,
                                                 Map<TxnId, InMemorySafeCommand> commands,
                                                 Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey,
-                                                Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommandsForKeys,
-                                                Map<RoutableKey, InMemorySafeCommandsForKey> completeCommandsForKeys)
+                                                Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKeys)
     {
-        return new InMemorySafeStore(this, ranges, context, commands, timestampsForKey, minimalCommandsForKeys, completeCommandsForKeys);
+        return new InMemorySafeStore(this, ranges, context, commands, timestampsForKey, commandsForKeys);
     }
 
     private void loadCommandsForKey(RoutableKey key,
                                     KeyHistory keyHistory,
                                     Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey,
-                                    Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommands,
-                                    Map<RoutableKey, InMemorySafeCommandsForKey> completeCommands)
+                                    Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey)
     {
-        switch (keyHistory)
-        {
-            case ALL:
-                // loading recovery commands implicitly loads deps
-                completeCommands.put(key, allCommandsForKey((Key) key).createSafeReference());
-            case DEPS:
-                minimalCommands.put(key, depsCommandsForKey((Key) key).createSafeReference());
-                break;
-        }
+        commandsForKey.put(key, commandsForKey((Key) key).createSafeReference());
         timestampsForKey.put(key, timestampsForKey((Key) key).createSafeReference());
     }
 
     protected final InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges)
     {
         Map<TxnId, InMemorySafeCommand> commands = new HashMap<>();
-        Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommands = new HashMap<>();
-        Map<RoutableKey, InMemorySafeCommandsForKey> completeCommands = new HashMap<>();
+        Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey = new HashMap<>();
         Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey = new HashMap<>();
 
         context.forEachId(txnId -> commands.put(txnId, lazyReference(txnId)));
@@ -510,22 +391,14 @@
             {
                 case Key:
                     RoutableKey key = (RoutableKey) seekable;
-                    switch (context.keyHistory())
-                    {
-                        case ALL:
-                            // loading recovery commands implicitly loads deps
-                            completeCommands.put(key, allCommandsForKey((Key) key).createSafeReference());
-                        case DEPS:
-                            minimalCommands.put(key, depsCommandsForKey((Key) key).createSafeReference());
-                            break;
-                    }
+                    commandsForKey.put(key, commandsForKey((Key) key).createSafeReference());
                     timestampsForKey.put(key, timestampsForKey((Key) key).createSafeReference());
                     break;
                 case Range:
                     // load range cfks here
             }
         }
-        return createSafeStore(context, ranges, commands, timestampsForKey, minimalCommands, completeCommands);
+        return createSafeStore(context, ranges, commands, timestampsForKey, commandsForKey);
     }
 
     public SafeCommandStore beginOperation(PreLoadContext context)
@@ -593,6 +466,22 @@
         }
     }
 
+    private static Timestamp maxApplied(SafeCommandStore safeStore, Seekables<?, ?> keysOrRanges, Ranges slice)
+    {
+        Timestamp max = ((InMemoryCommandStore)safeStore.commandStore()).maxRedundant;
+        for (GlobalCommand command : ((InMemoryCommandStore) safeStore.commandStore()).commands.values())
+        {
+            if (command.value().hasBeen(Applied))
+                max = Timestamp.max(command.value().executeAt(), max);
+        }
+        return max;
+    }
+
+    public AsyncChain<Timestamp> maxAppliedFor(Seekables<?, ?> keysOrRanges, Ranges slice)
+    {
+        return submit(PreLoadContext.contextFor(keysOrRanges), safeStore -> maxApplied(safeStore, keysOrRanges, slice));
+    }
+
     @Override
     public String toString()
     {
@@ -618,66 +507,6 @@
         }
     }
 
-    static class CFKEntry extends TxnId
-    {
-        final boolean uninitialised;
-        public CFKEntry(TxnId copy, boolean uninitialised)
-        {
-            super(copy);
-            this.uninitialised = uninitialised;
-        }
-    }
-
-    class CFKLoader implements CommandLoader<CFKEntry>
-    {
-        final RoutableKey key;
-        CFKLoader(RoutableKey key)
-        {
-            this.key = key;
-        }
-
-        private Command loadForCFK(CFKEntry entry)
-        {
-            GlobalCommand globalCommand = commandIfPresent(entry);
-            if (globalCommand != null && globalCommand.value() != null)
-                return globalCommand.value();
-            if (entry.uninitialised)
-                return uninitialised(entry);
-            throw illegalState("Could not find command for CFK for " + entry);
-        }
-
-        @Override
-        public TxnId txnId(CFKEntry txnId)
-        {
-            return loadForCFK(txnId).txnId();
-        }
-
-        @Override
-        public Timestamp executeAt(CFKEntry txnId)
-        {
-            return loadForCFK(txnId).executeAt();
-        }
-
-        @Override
-        public SaveStatus saveStatus(CFKEntry txnId)
-        {
-            return loadForCFK(txnId).saveStatus();
-        }
-
-        @Override
-        public List<TxnId> depsIds(CFKEntry data)
-        {
-            PartialDeps deps = loadForCFK(data).partialDeps();
-            return deps != null ? deps.txnIds() : Collections.emptyList();
-        }
-
-        @Override
-        public CFKEntry saveForCFK(Command command)
-        {
-            return new CFKEntry(command.txnId(), command.saveStatus().isUninitialised());
-        }
-    }
-
     public static abstract class GlobalState<V>
     {
         private V value;
@@ -790,14 +619,12 @@
         }
     }
 
-    public static class InMemorySafeStore extends AbstractSafeCommandStore<InMemorySafeCommand, InMemorySafeTimestampsForKey, InMemorySafeCommandsForKey, InMemorySafeCommandsForKeyUpdate>
+    public static class InMemorySafeStore extends AbstractSafeCommandStore<InMemorySafeCommand, InMemorySafeTimestampsForKey, InMemorySafeCommandsForKey>
     {
         private final InMemoryCommandStore commandStore;
         private final Map<TxnId, InMemorySafeCommand> commands;
         private final Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey;
-        private final Map<RoutableKey, InMemorySafeCommandsForKey> depsCommandsForKey;
-        private final Map<RoutableKey, InMemorySafeCommandsForKey> allCommandsForKey;
-        private final Map<RoutableKey, InMemorySafeCommandsForKeyUpdate> updatesForKey = new HashMap<>();
+        private final Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey;
         private final RangesForEpoch ranges;
 
         public InMemorySafeStore(InMemoryCommandStore commandStore,
@@ -805,14 +632,12 @@
                                  PreLoadContext context,
                                  Map<TxnId, InMemorySafeCommand> commands,
                                  Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey,
-                                 Map<RoutableKey, InMemorySafeCommandsForKey> depsCommandsForKey,
-                                 Map<RoutableKey, InMemorySafeCommandsForKey> allCommandsForKey)
+                                 Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey)
         {
             super(context);
             this.commandStore = commandStore;
             this.commands = commands;
-            this.depsCommandsForKey = depsCommandsForKey;
-            this.allCommandsForKey = allCommandsForKey;
+            this.commandsForKey = commandsForKey;
             this.timestampsForKey = timestampsForKey;
             this.ranges = Invariants.nonNull(ranges);
         }
@@ -860,107 +685,46 @@
         }
 
         @Override
-        protected InMemorySafeCommandsForKey getDepsCommandsForKeyInternal(RoutableKey key)
+        protected InMemorySafeCommandsForKey getCommandsForKeyInternal(RoutableKey key)
         {
-            return depsCommandsForKey.get(key);
+            return commandsForKey.get(key);
         }
 
         @Override
-        protected void addDepsCommandsForKeyInternal(InMemorySafeCommandsForKey cfk)
+        protected void addCommandsForKeyInternal(InMemorySafeCommandsForKey cfk)
         {
-            depsCommandsForKey.put(cfk.key(), cfk);
+            commandsForKey.put(cfk.key(), cfk);
         }
 
         @Override
-        protected InMemorySafeCommandsForKey getDepsCommandsForKeyIfLoaded(RoutableKey key)
+        protected InMemorySafeCommandsForKey getCommandsForKeyIfLoaded(RoutableKey key)
         {
             if (!commandStore.canExposeUnloaded())
                 return null;
-            GlobalCommandsForKey global = commandStore.depsCommandsForKeyIfPresent((Key) key);
+            GlobalCommandsForKey global = commandStore.commandsForKeyIfPresent((Key) key);
             return global != null ? global.createSafeReference() : null;
         }
 
         @Override
-        protected InMemorySafeCommandsForKey getAllCommandsForKeyInternal(RoutableKey key)
+        protected void update(Command prev, Command updated, @Nullable Seekables<?, ?> keysOrRanges)
         {
-            return allCommandsForKey.get(key);
-        }
+            super.update(prev, updated, keysOrRanges);
 
-        @Override
-        protected void addAllCommandsForKeyInternal(InMemorySafeCommandsForKey cfk)
-        {
-            allCommandsForKey.put(cfk.key(), cfk);
-        }
+            TxnId txnId = updated.txnId();
+            if (txnId.domain() != Domain.Range)
+                return;
+            if (keysOrRanges == null && !updated.known().isDefinitionKnown())
+                return;
+            if (prev != null && prev.known().isDefinitionKnown())
+                return;
 
-        @Override
-        protected InMemorySafeCommandsForKey getAllCommandsForKeyIfLoaded(RoutableKey key)
-        {
-            if (!commandStore.canExposeUnloaded())
-                return null;
-            GlobalCommandsForKey global = commandStore.allCommandsForKeyIfPresent((Key) key);
-            return global != null ? global.createSafeReference() : null;
-        }
+            if (keysOrRanges == null) keysOrRanges = updated.partialTxn().keys();
 
-        @Override
-        protected InMemorySafeCommandsForKeyUpdate getCommandsForKeyUpdateInternal(RoutableKey key)
-        {
-            return updatesForKey.get(key);
-        }
+            Ranges slice = ranges().allBetween(txnId, updated.executeAtOrTxnId());
+            slice = commandStore.redundantBefore().removeShardRedundant(txnId, updated.executeAt(), slice);
+            commandStore.rangeCommands.computeIfAbsent(txnId, ignore -> new RangeCommand(commandStore.commands.get(txnId)))
+                         .update(((Ranges)keysOrRanges).slice(slice, Minimal));
 
-        @Override
-        protected InMemorySafeCommandsForKeyUpdate createCommandsForKeyUpdateInternal(RoutableKey key)
-        {
-            return new InMemorySafeCommandsForKeyUpdate((Key) key, (CommandLoader<CFKEntry>) cfkLoader(key));
-        }
-
-        @Override
-        protected void addCommandsForKeyUpdateInternal(InMemorySafeCommandsForKeyUpdate update)
-        {
-            updatesForKey.put(update.key(), update);
-        }
-
-        private CommandsForKey applyUpdate(CommandsForKey cfk, InMemorySafeCommandsForKeyUpdate update, KeyHistory keyHistory)
-        {
-            switch (keyHistory)
-            {
-                case DEPS:
-                    return update.applyToDeps(cfk);
-                case ALL:
-                    return update.applyToAll(cfk);
-                default:
-                    throw new IllegalArgumentException("Unhandled KeyHistory " + keyHistory);
-            }
-        }
-
-        private void applyCommandForKeyUpdate(InMemorySafeCommandsForKeyUpdate update, KeyHistory kind, Map<RoutableKey, InMemorySafeCommandsForKey> commandsMap, Function<Key, GlobalCommandsForKey> getter)
-        {
-            Key key = update.key();
-            InMemorySafeCommandsForKey safeCFK = commandsMap.get(key);
-            if (safeCFK != null)
-            {
-                if (safeCFK.isEmpty()) safeCFK.initialize(cfkLoader(key));
-                safeCFK.update(applyUpdate(safeCFK.current(), update, kind));
-            }
-            else
-            {
-                GlobalCommandsForKey globalCFK = getter.apply(key);
-                CommandsForKey cfk = globalCFK.value();
-                if (cfk == null)
-                    cfk = new CommandsForKey(key, cfkLoader(key));
-                globalCFK.value(applyUpdate(cfk, update, kind));
-            }
-        }
-
-        private void applyCommandForKeyUpdate(InMemorySafeCommandsForKeyUpdate update)
-        {
-            applyCommandForKeyUpdate(update, KeyHistory.DEPS, depsCommandsForKey, commandStore::depsCommandsForKey);
-            applyCommandForKeyUpdate(update, KeyHistory.ALL, allCommandsForKey, commandStore::allCommandsForKey);
-        }
-
-        @Override
-        protected void applyCommandForKeyUpdates()
-        {
-            updatesForKey.values().forEach(this::applyCommandForKeyUpdate);
         }
 
         @Override
@@ -993,26 +757,6 @@
             return ranges;
         }
 
-        @Override
-        public Timestamp maxConflict(Seekables<?, ?> keysOrRanges, Ranges slice)
-        {
-            Timestamp timestamp = commandStore.mapReduceForKey(this, keysOrRanges, slice, KeyHistory.NONE, (timestamps, commands, prev) -> Timestamp.nonNullOrMax(timestamps.max(), prev), Timestamp.NONE, Objects::isNull);
-            Seekables<?, ?> sliced = keysOrRanges.slice(slice, Minimal);
-            for (RangeCommand command : commandStore.rangeCommands.values())
-            {
-                if (command.ranges.intersects(sliced))
-                    timestamp = Timestamp.nonNullOrMax(timestamp, command.command.value().executeAt());
-            }
-            // TODO (required): check we have this on C*-side; accept TxnId instead of Ranges when updating API, to avoid slicing
-            return Timestamp.nonNullOrMax(timestamp, commandStore.redundantBefore().minConflict(keysOrRanges.slice(slice)));
-        }
-
-        @Override
-        public void erase(SafeCommand command)
-        {
-            commands.remove(command.txnId());
-        }
-
         // TODO (preferable): this can have protected visibility if under CommandStore, and this is perhaps a better place to put it also
         @Override
         public void registerHistoricalTransactions(Deps deps)
@@ -1080,41 +824,30 @@
                 this.deps = deps != null ? deps.txnIds() : Collections.emptyList();
             }
         }
+
+        @Override
+        public <P1, T> T mapReduceActive(Seekables<?, ?> keysOrRanges, Ranges slice, Timestamp startedBefore, Kinds testKind, CommandFunction<P1, T, T> map, P1 p1, T accumulate)
+        {
+            accumulate = commandStore.mapReduceForKey(this, keysOrRanges, slice, (commands, prev) -> {
+                return commands.mapReduceActive(startedBefore, testKind, map, p1, prev);
+            }, accumulate);
+
+            return mapReduceRangesInternal(keysOrRanges, slice, startedBefore, null, testKind, STARTED_BEFORE, ANY_DEPS, ANY_STATUS, map, p1, accumulate);
+        }
+
         // TODO (expected): instead of accepting a slice, accept the min/max epoch and let implementation handle it
         @Override
-        public <P1, T> T mapReduce(Seekables<?, ?> keysOrRanges, Ranges slice, KeyHistory keyHistory, Kinds testKind, TestTimestamp testTimestamp, Timestamp timestamp, TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus, CommandFunction<P1, T, T> map, P1 p1, T accumulate, Predicate<? super T> terminate)
+        public <P1, T> T mapReduceFull(Seekables<?, ?> keysOrRanges, Ranges slice, TxnId testTxnId, Kinds testKind, TestStartedAt testStartedAt, TestDep testDep, TestStatus testStatus, CommandFunction<P1, T, T> map, P1 p1, T accumulate)
         {
-            accumulate = commandStore.mapReduceForKey(this, keysOrRanges, slice, keyHistory, (timestamps, commands, prev) -> {
-                CommandTimeseries.TimestampType timestampType;
-                switch (testTimestamp)
-                {
-                    default: throw new AssertionError();
-                    case STARTED_AFTER:
-                    case STARTED_BEFORE:
-                        timestampType = CommandTimeseries.TimestampType.TXN_ID;
-                        break;
-                    case EXECUTES_AFTER:
-                    case MAY_EXECUTE_BEFORE:
-                        timestampType = CommandTimeseries.TimestampType.EXECUTE_AT;
-                }
-                CommandTimeseries.TestTimestamp remapTestTimestamp;
-                switch (testTimestamp)
-                {
-                    default: throw new AssertionError();
-                    case STARTED_AFTER:
-                    case EXECUTES_AFTER:
-                        remapTestTimestamp = CommandTimeseries.TestTimestamp.AFTER;
-                        break;
-                    case STARTED_BEFORE:
-                    case MAY_EXECUTE_BEFORE:
-                        remapTestTimestamp = CommandTimeseries.TestTimestamp.BEFORE;
-                }
-                return commands.commands().mapReduce(testKind, timestampType, remapTestTimestamp, timestamp, testDep, depId, minStatus, maxStatus, map, p1, prev, terminate);
-            }, accumulate, terminate);
+            accumulate = commandStore.mapReduceForKey(this, keysOrRanges, slice, (commands, prev) -> {
+                return commands.mapReduceFull(testTxnId, testKind, testStartedAt, testDep, testStatus, map, p1, prev);
+            }, accumulate);
 
-            if (terminate.test(accumulate))
-                return accumulate;
+            return mapReduceRangesInternal(keysOrRanges, slice, testTxnId, testTxnId, testKind, testStartedAt, testDep, testStatus, map, p1, accumulate);
+        }
 
+        private <P1, T> T mapReduceRangesInternal(Seekables<?, ?> keysOrRanges, Ranges slice, @Nonnull Timestamp testTimestamp, @Nullable TxnId testTxnId, Kinds testKind, TestStartedAt testStartedAt, TestDep testDep, TestStatus testStatus, CommandFunction<P1, T, T> map, P1 p1, T accumulate)
+        {
             // TODO (find lib, efficiency): this is super inefficient, need to store Command in something queryable
             Seekables<?, ?> sliced = keysOrRanges.slice(slice, Minimal);
             Map<Range, List<TxnInfo>> collect = new TreeMap<>(Range::compare);
@@ -1125,29 +858,37 @@
                     return;
 
                 Invariants.nonNull(command);
-                switch (testTimestamp)
+                switch (testStartedAt)
                 {
                     default: throw new AssertionError();
                     case STARTED_AFTER:
-                        if (command.txnId().compareTo(timestamp) < 0) return;
+                        if (command.txnId().compareTo(testTimestamp) <= 0) return;
                         else break;
                     case STARTED_BEFORE:
-                        if (command.txnId().compareTo(timestamp) > 0) return;
-                        else break;
-                    case EXECUTES_AFTER:
-                        if (command.executeAt().compareTo(timestamp) < 0) return;
-                        else break;
-                    case MAY_EXECUTE_BEFORE:
-                        Timestamp compareTo = command.executeAtIfKnownElseTxnId();
-                        if (compareTo.compareTo(timestamp) > 0) return;
-                        else break;
+                        if (command.txnId().compareTo(testTimestamp) >= 0) return;
+                    case ANY:
+                        if (testDep != ANY_DEPS && command.executeAtOrTxnId().compareTo(testTxnId) < 0)
+                            return;
                 }
 
-                if (minStatus != null && command.status().compareTo(minStatus) < 0)
-                    return;
-
-                if (maxStatus != null && command.status().compareTo(maxStatus) > 0)
-                    return;
+                switch (testStatus)
+                {
+                    default: throw new AssertionError("Unhandled TestStatus: " + testStatus);
+                    case ANY_STATUS:
+                        break;
+                    case IS_PROPOSED:
+                        switch (command.status())
+                        {
+                            default: return;
+                            case PreCommitted:
+                            case Committed:
+                            case Accepted:
+                        }
+                        break;
+                    case IS_STABLE:
+                        if (command.status().compareTo(Stable) < 0 || command.status().compareTo(Truncated) >= 0)
+                            return;
+                }
 
                 if (!testKind.test(command.txnId().kind()))
                     return;
@@ -1168,7 +909,7 @@
                     // and so it is safe to execute, when in fact it is only a dependency on a different shard
                     // (and that other shard, perhaps, does not know that it is a dependency - and so it is not durably known)
                     // TODO (required): consider this some more
-                    if ((testDep == WITH) == !command.partialDeps().intersects(depId, rangeCommand.ranges))
+                    if ((testDep == WITH) == !command.partialDeps().intersects(testTxnId, rangeCommand.ranges))
                         return;
                 }
 
@@ -1184,20 +925,19 @@
                 }, collect);
             }));
 
-            if (minStatus == null && testDep == ANY_DEPS)
+            if (testStatus == ANY_STATUS && testDep == ANY_DEPS)
             {
                 commandStore.historicalRangeCommands.forEach(((txnId, ranges) -> {
-                    switch (testTimestamp)
+                    switch (testStartedAt)
                     {
                         default: throw new AssertionError();
                         case STARTED_AFTER:
-                        case EXECUTES_AFTER:
-                            if (txnId.compareTo(timestamp) < 0) return;
+                            if (txnId.compareTo(testTimestamp) <= 0) return;
                             else break;
                         case STARTED_BEFORE:
-                        case MAY_EXECUTE_BEFORE:
-                            if (txnId.compareTo(timestamp) > 0) return;
+                            if (txnId.compareTo(testTimestamp) >= 0) return;
                             else break;
+                        case ANY:
                     }
 
                     if (!testKind.test(txnId.kind()))
@@ -1234,9 +974,7 @@
                 for (TxnInfo command : e.getValue())
                 {
                     T initial = accumulate;
-                    accumulate = map.apply(p1, e.getKey(), command.txnId, command.executeAt, command.status, () -> command.deps, initial);
-                    if (terminate.test(accumulate))
-                        return accumulate;
+                    accumulate = map.apply(p1, e.getKey(), command.txnId, command.executeAt, initial);
                 }
             }
 
@@ -1244,24 +982,6 @@
         }
 
         @Override
-        public CommonAttributes completeRegistration(Seekables<?, ?> keysOrRanges, Ranges slice, InMemorySafeCommand command, CommonAttributes attrs)
-        {
-            return commandStore.register(this, keysOrRanges, slice, command, attrs);
-        }
-
-        @Override
-        public CommonAttributes completeRegistration(Seekable keyOrRange, Ranges slice, InMemorySafeCommand command, CommonAttributes attrs)
-        {
-            return commandStore.register(this, keyOrRange, slice, command, attrs);
-        }
-
-        @Override
-        public CommandLoader<?> cfkLoader(RoutableKey key)
-        {
-            return commandStore.new CFKLoader(key);
-        }
-
-        @Override
         protected void invalidateSafeState()
         {
             commands.values().forEach(c -> {
@@ -1277,16 +997,7 @@
             });
             commands.values().forEach(SafeState::invalidate);
             timestampsForKey.values().forEach(SafeState::invalidate);
-            depsCommandsForKey.values().forEach(SafeState::invalidate);
-            allCommandsForKey.values().forEach(SafeState::invalidate);
-            updatesForKey.values().forEach(SafeState::invalidate);
-        }
-
-        @Override
-        public void complete()
-        {
-            postExecute();
-            super.complete();
+            commandsForKey.values().forEach(SafeState::invalidate);
         }
     }
 
@@ -1451,10 +1162,9 @@
                                   PreLoadContext context,
                                   Map<TxnId, InMemorySafeCommand> commands,
                                   Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey,
-                                  Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommandsForKey,
-                                  Map<RoutableKey, InMemorySafeCommandsForKey> completeCommandsForKey)
+                                  Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey)
             {
-                super(commandStore, ranges, context, commands, timestampsForKey, minimalCommandsForKey, completeCommandsForKey);
+                super(commandStore, ranges, context, commands, timestampsForKey, commandsForKey);
             }
 
             @Override
@@ -1470,20 +1180,6 @@
                 assertThread();
                 return super.getInternal(txnId);
             }
-
-            @Override
-            public void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command)
-            {
-                assertThread();
-                super.register(keysOrRanges, slice, command);
-            }
-
-            @Override
-            public void register(Seekable keyOrRange, Ranges slice, Command command)
-            {
-                assertThread();
-                super.register(keyOrRange, slice, command);
-            }
         }
 
         public Debug(int id, NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, EpochUpdateHolder epochUpdateHolder)
@@ -1492,9 +1188,9 @@
         }
 
         @Override
-        protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges, Map<TxnId, InMemorySafeCommand> commands, Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKeyMap, Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommandsForKeys, Map<RoutableKey, InMemorySafeCommandsForKey> completeCommandsForKeys)
+        protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges, Map<TxnId, InMemorySafeCommand> commands, Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKeyMap, Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKeys)
         {
-            return new DebugSafeStore(this, ranges, context, commands, timestampsForKeyMap, minimalCommandsForKeys, completeCommandsForKeys);
+            return new DebugSafeStore(this, ranges, context, commands, timestampsForKeyMap, commandsForKeys);
         }
     }
 
diff --git a/accord-core/src/main/java/accord/impl/InMemorySafeCommandsForKeyUpdate.java b/accord-core/src/main/java/accord/impl/InMemorySafeCommandsForKeyUpdate.java
deleted file mode 100644
index 2c7f955..0000000
--- a/accord-core/src/main/java/accord/impl/InMemorySafeCommandsForKeyUpdate.java
+++ /dev/null
@@ -1,60 +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 accord.impl;
-
-import accord.api.Key;
-import accord.impl.CommandTimeseries.CommandLoader;
-import accord.impl.InMemoryCommandStore.CFKEntry;
-import accord.utils.Invariants;
-
-public class InMemorySafeCommandsForKeyUpdate extends SafeCommandsForKey.Update<CommandsForKey.Update, CFKEntry>
-{
-    private static final CommandsForKey.Update VALUE = new CommandsForKey.Update() {};
-
-    private boolean invalidated = false;
-
-    public InMemorySafeCommandsForKeyUpdate(Key key, CommandLoader<CFKEntry> loader)
-    {
-        super(key, loader);
-    }
-
-    @Override
-    public void initialize()
-    {
-        Invariants.checkState(isEmpty());
-    }
-
-    @Override
-    public CommandsForKey.Update current()
-    {
-        return VALUE;
-    }
-
-    @Override
-    public void invalidate()
-    {
-        invalidated = true;
-    }
-
-    @Override
-    public boolean invalidated()
-    {
-        return invalidated;
-    }
-}
diff --git a/accord-core/src/main/java/accord/impl/SafeCommandsForKey.java b/accord-core/src/main/java/accord/impl/SafeCommandsForKey.java
index df1a749..aea0dbc 100644
--- a/accord-core/src/main/java/accord/impl/SafeCommandsForKey.java
+++ b/accord-core/src/main/java/accord/impl/SafeCommandsForKey.java
@@ -19,7 +19,8 @@
 package accord.impl;
 
 import accord.api.Key;
-import accord.impl.CommandTimeseries.CommandLoader;
+import accord.local.Command;
+import accord.primitives.TxnId;
 
 public abstract class SafeCommandsForKey implements SafeState<CommandsForKey>
 {
@@ -43,25 +44,35 @@
         return update;
     }
 
-    public CommandsForKey initialize(CommandLoader<?> loader)
+    CommandsForKey update(Command prev, Command update)
     {
-        return update(new CommandsForKey(key, loader));
+        CommandsForKey current = current();
+        CommandsForKey next = current.update(prev, update);
+        if (next != current)
+            set(next);
+        return next;
     }
 
-    public static abstract class Update<U extends CommandsForKey.Update, D> extends CommandsForKeyGroupUpdater.Mutable<D> implements SafeState<U>
+    CommandsForKey registerHistorical(TxnId txnId)
     {
-        private final Key key;
+        CommandsForKey current = current();
+        CommandsForKey next = current.registerHistorical(txnId);
+        if (next != current)
+            set(next);
+        return next;
+    }
 
-        public Update(Key key, CommandLoader<D> loader)
-        {
-            super(loader);
-            this.key = key;
-        }
+    CommandsForKey updateRedundantBefore(TxnId redundantBefore)
+    {
+        CommandsForKey current = current();
+        CommandsForKey next = current.withoutRedundant(redundantBefore);
+        if (next != current)
+            set(next);
+        return next;
+    }
 
-        public Key key()
-        {
-            return key;
-        }
-        public abstract void initialize();
+    public void initialize()
+    {
+        set(new CommandsForKey(key));
     }
 }
diff --git a/accord-core/src/main/java/accord/impl/SafeTimestampsForKey.java b/accord-core/src/main/java/accord/impl/SafeTimestampsForKey.java
index ee5d58c..42eb3c5 100644
--- a/accord-core/src/main/java/accord/impl/SafeTimestampsForKey.java
+++ b/accord-core/src/main/java/accord/impl/SafeTimestampsForKey.java
@@ -19,10 +19,7 @@
 package accord.impl;
 
 import accord.api.Key;
-import accord.api.VisibleForImplementation;
 import accord.primitives.Timestamp;
-import accord.utils.Invariants;
-import com.google.common.annotations.VisibleForTesting;
 
 public abstract class SafeTimestampsForKey implements SafeState<TimestampsForKey>
 {
@@ -51,35 +48,10 @@
         return update(new TimestampsForKey(key));
     }
 
-    @VisibleForTesting
-    @VisibleForImplementation
-    public static Timestamp updateMax(TimestampsForKey tfk, Timestamp timestamp)
-    {
-        Invariants.checkArgument(tfk != null || timestamp != null);
-        if (tfk == null)
-            return timestamp;
-        if (timestamp == null)
-            return tfk.max();
-        return Timestamp.max(tfk.max(), timestamp);
-    }
-
-    @VisibleForTesting
-    @VisibleForImplementation
-    public <D> TimestampsForKey updateMax(Timestamp timestamp)
+    TimestampsForKey updateLastExecutionTimestamps(Timestamp lastExecutedTimestamp, long lastExecutedHlc, Timestamp lastWriteTimestamp)
     {
         TimestampsForKey current = current();
         return update(new TimestampsForKey(current.key(),
-                                           updateMax(current, timestamp),
-                                           current.lastExecutedTimestamp(),
-                                           current.rawLastExecutedHlc(),
-                                           current.lastWriteTimestamp()));
-    }
-
-    <D> TimestampsForKey updateLastExecutionTimestamps(Timestamp lastExecutedTimestamp, long lastExecutedHlc, Timestamp lastWriteTimestamp)
-    {
-        TimestampsForKey current = current();
-        return update(new TimestampsForKey(current.key(),
-                                           current.max(),
                                            lastExecutedTimestamp,
                                            lastExecutedHlc,
                                            lastWriteTimestamp));
diff --git a/accord-core/src/main/java/accord/impl/TimestampsForKey.java b/accord-core/src/main/java/accord/impl/TimestampsForKey.java
index 65fc094..32494b6 100644
--- a/accord-core/src/main/java/accord/impl/TimestampsForKey.java
+++ b/accord-core/src/main/java/accord/impl/TimestampsForKey.java
@@ -24,33 +24,30 @@
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 
-public class TimestampsForKey implements DomainTimestamps
+public class TimestampsForKey
 {
     public static final long NO_LAST_EXECUTED_HLC = Long.MIN_VALUE;
 
     public static class SerializerSupport
     {
-        public static <D> TimestampsForKey create(Key key,
-                                                  Timestamp max,
+        public static TimestampsForKey create(Key key,
                                                   Timestamp lastExecutedTimestamp,
                                                   long lastExecutedHlc,
                                                   Timestamp lastWriteTimestamp)
         {
-            return new TimestampsForKey(key, max, lastExecutedTimestamp, lastExecutedHlc, lastWriteTimestamp);
+            return new TimestampsForKey(key, lastExecutedTimestamp, lastExecutedHlc, lastWriteTimestamp);
         }
     }
 
     private final Key key;
-    private final Timestamp max;
     private final Timestamp lastExecutedTimestamp;
     // TODO (desired): we have leaked C* implementation details here
     private final long rawLastExecutedHlc;
     private final Timestamp lastWriteTimestamp;
 
-    public TimestampsForKey(Key key, Timestamp max, Timestamp lastExecutedTimestamp, long rawLastExecutedHlc, Timestamp lastWriteTimestamp)
+    public TimestampsForKey(Key key, Timestamp lastExecutedTimestamp, long rawLastExecutedHlc, Timestamp lastWriteTimestamp)
     {
         this.key = key;
-        this.max = max;
         this.lastExecutedTimestamp = lastExecutedTimestamp;
         this.rawLastExecutedHlc = rawLastExecutedHlc;
         this.lastWriteTimestamp = lastWriteTimestamp;
@@ -59,7 +56,6 @@
     public TimestampsForKey(Key key)
     {
         this.key = key;
-        this.max = Timestamp.NONE;
         this.lastExecutedTimestamp = Timestamp.NONE;
         this.rawLastExecutedHlc = 0;
         this.lastWriteTimestamp = Timestamp.NONE;
@@ -70,7 +66,7 @@
         if (this == o) return true;
         if (o == null || getClass() != o.getClass()) return false;
         TimestampsForKey that = (TimestampsForKey) o;
-        return rawLastExecutedHlc == that.rawLastExecutedHlc && Objects.equals(key, that.key) && Objects.equals(max, that.max) && Objects.equals(lastExecutedTimestamp, that.lastExecutedTimestamp) && Objects.equals(lastWriteTimestamp, that.lastWriteTimestamp);
+        return rawLastExecutedHlc == that.rawLastExecutedHlc && Objects.equals(key, that.key) && Objects.equals(lastExecutedTimestamp, that.lastExecutedTimestamp) && Objects.equals(lastWriteTimestamp, that.lastWriteTimestamp);
     }
 
     public int hashCode()
@@ -83,11 +79,6 @@
         return key;
     }
 
-    public Timestamp max()
-    {
-        return max;
-    }
-
     public Timestamp lastExecutedTimestamp()
     {
         return lastExecutedTimestamp;
@@ -110,7 +101,7 @@
 
     public TimestampsForKey withoutRedundant(TxnId redundantBefore)
     {
-        return new TimestampsForKey(key, max.compareTo(redundantBefore) < 0 ? Timestamp.NONE : max,
+        return new TimestampsForKey(key,
                                     lastExecutedTimestamp.compareTo(redundantBefore) < 0 ? Timestamp.NONE : lastExecutedTimestamp,
                                     rawLastExecutedHlc < redundantBefore.hlc() ? NO_LAST_EXECUTED_HLC : rawLastExecutedHlc,
                                     lastWriteTimestamp.compareTo(redundantBefore) < 0 ? Timestamp.NONE : lastWriteTimestamp);
@@ -141,7 +132,6 @@
     {
         return "TimestampsForKey@" + System.identityHashCode(this) + '{' +
                "key=" + key +
-               ", max=" + max +
                ", lastExecutedTimestamp=" + lastExecutedTimestamp +
                ", rawLastExecutedHlc=" + rawLastExecutedHlc +
                ", lastWriteTimestamp=" + lastWriteTimestamp +
diff --git a/accord-core/src/main/java/accord/local/Bootstrap.java b/accord-core/src/main/java/accord/local/Bootstrap.java
index 3d685a0..d268812 100644
--- a/accord-core/src/main/java/accord/local/Bootstrap.java
+++ b/accord-core/src/main/java/accord/local/Bootstrap.java
@@ -28,7 +28,7 @@
 import accord.api.DataStore.FetchRanges;
 import accord.api.DataStore.FetchResult;
 import accord.api.DataStore.StartingRangeFetch;
-import accord.coordinate.CoordinateNoOp;
+import accord.coordinate.FetchMaxConflict;
 import accord.coordinate.CoordinateSyncPoint;
 import accord.primitives.Ranges;
 import accord.primitives.Routable;
@@ -45,7 +45,6 @@
 import static accord.primitives.Routables.Slice.Minimal;
 import static accord.primitives.Txn.Kind.ExclusiveSyncPoint;
 import static accord.primitives.Txn.Kind.LocalOnly;
-import static accord.primitives.Txn.Kind.NoOp;
 import static accord.utils.Invariants.illegalState;
 
 /**
@@ -137,7 +136,7 @@
                // TODO (required, correcness) : PreLoadContext only works with Seekables, which doesn't allow mixing Keys and Ranges... But Deps has both Keys AND Ranges!
                // TODO (required): is localSyncId even being used anymore
                // ATM all known implementations store ranges in-memory, but this will not be true soon, so this will need to be addressed
-               .flatMap(syncPoint -> node.withEpoch(epoch, () -> store.submit(contextFor(localSyncId, syncPoint.waitFor.keyDeps.keys(), KeyHistory.DEPS), safeStore1 -> {
+               .flatMap(syncPoint -> node.withEpoch(epoch, () -> store.submit(contextFor(localSyncId, syncPoint.waitFor.keyDeps.keys(), KeyHistory.COMMANDS), safeStore1 -> {
                    if (valid.isEmpty()) // we've lost ownership of the range
                        return AsyncResults.success(Ranges.EMPTY);
 
@@ -226,18 +225,14 @@
         {
             if (maxApplied == null)
             {
-                TxnId txnId;
                 synchronized (this)
                 {
                     if (state.startedAt == Integer.MAX_VALUE)
                         state.startedAt = logicalClock++;
-                    txnId = node.nextTxnId(NoOp, Routable.Domain.Range);
                 }
                 // TODO (expected): associate callbacks with this CommandStore, to remove synchronization
-                node.withEpoch(txnId.epoch(), () -> {
-                    CoordinateNoOp.coordinate(node, txnId, state.ranges)
-                                  .begin((executeAt, failure) -> safeToReadCallback(state, executeAt, failure));
-                });
+                FetchMaxConflict.fetchMaxConflict(node, state.ranges)
+                                .begin((executeAt, failure) -> safeToReadCallback(state, executeAt, failure));
             }
             else
             {
diff --git a/accord-core/src/main/java/accord/local/Cleanup.java b/accord-core/src/main/java/accord/local/Cleanup.java
index 97191dc..8c2dd18 100644
--- a/accord-core/src/main/java/accord/local/Cleanup.java
+++ b/accord-core/src/main/java/accord/local/Cleanup.java
@@ -36,6 +36,7 @@
 import static accord.local.Status.Durability.Universal;
 import static accord.local.Status.Durability.UniversalOrInvalidated;
 import static accord.local.Status.PreCommitted;
+import static accord.primitives.Txn.Kind.EphemeralRead;
 import static accord.utils.Invariants.illegalState;
 
 /**
@@ -100,6 +101,9 @@
 
     public static Cleanup shouldCleanup(TxnId txnId, Status status, Durability durability, EpochSupplier toEpoch, Route<?> route, RedundantBefore redundantBefore, DurableBefore durableBefore, boolean enforceInvariants)
     {
+        if (txnId.kind() == EphemeralRead)
+            return Cleanup.NO; // TODO (required): clean-up based on timeout
+
         if (durableBefore.min(txnId) == Universal)
         {
             if (status.hasBeen(PreCommitted) && !status.hasBeen(Applied)) // TODO (expected): may be stale
diff --git a/accord-core/src/main/java/accord/local/Command.java b/accord-core/src/main/java/accord/local/Command.java
index fd18291..edb3310 100644
--- a/accord-core/src/main/java/accord/local/Command.java
+++ b/accord-core/src/main/java/accord/local/Command.java
@@ -128,12 +128,6 @@
         }
     }
 
-    static PreLoadContext contextForCommand(Command command)
-    {
-        Invariants.checkState(command.hasBeen(Status.PreAccepted) && command.partialTxn() != null);
-        return command instanceof PreLoadContext ? (PreLoadContext) command : PreLoadContext.contextFor(command.txnId(), command.partialTxn().keys());
-    }
-
     private static Durability durability(Durability durability, SaveStatus status)
     {
         if (status.compareTo(SaveStatus.PreApplied) >= 0 && !status.hasBeen(Invalidated) && durability == NotDurable)
@@ -174,10 +168,15 @@
             return Truncated.invalidated(txnId, durableListeners);
         }
 
-        public static Truncated truncatedApply(CommonAttributes common,SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result)
+        public static Truncated truncatedApply(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result)
         {
             return Truncated.truncatedApply(common, saveStatus, executeAt, writes, result);
         }
+
+        public static Truncated truncatedApply(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result, Timestamp executesAtLeast)
+        {
+            return Truncated.truncatedApply(common, saveStatus, executeAt, writes, result, executesAtLeast);
+        }
     }
 
     private static SaveStatus validateCommandClass(SaveStatus status, Class<?> expected, Class<?> actual)
@@ -190,27 +189,40 @@
         return status;
     }
 
-    private static SaveStatus validateCommandClass(SaveStatus status, Class<?> klass)
+    private static SaveStatus validateCommandClass(TxnId txnId, SaveStatus status, Class<?> klass)
     {
-        switch (status.status)
+        switch (status)
         {
+            case Uninitialised:
             case NotDefined:
                 return validateCommandClass(status, NotDefined.class, klass);
             case PreAccepted:
                 return validateCommandClass(status, PreAccepted.class, klass);
-            case AcceptedInvalidate:
             case Accepted:
+            case AcceptedWithDefinition:
+            case AcceptedInvalidate:
+            case AcceptedInvalidateWithDefinition:
             case PreCommitted:
+            case PreCommittedWithAcceptedDeps:
+            case PreCommittedWithDefinition:
+            case PreCommittedWithDefinitionAndAcceptedDeps:
                 return validateCommandClass(status, Accepted.class, klass);
             case Committed:
             case ReadyToExecute:
             case Stable:
                 return validateCommandClass(status, Committed.class, klass);
             case PreApplied:
+            case Applying:
             case Applied:
                 return validateCommandClass(status, Executed.class, klass);
+            case TruncatedApply:
+            case TruncatedApplyWithDeps:
+            case TruncatedApplyWithOutcome:
+                if (txnId.kind().awaitsOnlyDeps())
+                    return validateCommandClass(status, TruncatedAwaitsOnlyDeps.class, klass);
+            case Erased:
+            case ErasedOrInvalidated:
             case Invalidated:
-            case Truncated:
                 return validateCommandClass(status, Truncated.class, klass);
             default:
                 throw illegalState("Unhandled status " + status);
@@ -228,15 +240,14 @@
         private final TxnId txnId;
         private final SaveStatus status;
         private final Durability durability;
-        @Nullable
-        private final Route<?> route;
+        private final @Nullable Route<?> route;
         private final Ballot promised;
         private final Listeners.Immutable listeners;
 
         private AbstractCommand(TxnId txnId, SaveStatus status, Durability durability, @Nullable Route<?> route, Ballot promised, Listeners.Immutable listeners)
         {
             this.txnId = txnId;
-            this.status = validateCommandClass(status, getClass());
+            this.status = validateCommandClass(txnId, status, getClass());
             this.durability = durability;
             this.route = route;
             this.promised = promised;
@@ -246,7 +257,7 @@
         private AbstractCommand(CommonAttributes common, SaveStatus status, Ballot promised)
         {
             this.txnId = common.txnId();
-            this.status = validateCommandClass(status, getClass());
+            this.status = validateCommandClass(txnId, status, getClass());
             this.durability = common.durability();
             this.route = common.route();
             this.promised = promised;
@@ -352,8 +363,6 @@
                 switch (known.executeAt)
                 {
                     default: throw new AssertionError("Unhandled KnownExecuteAt: " + known.executeAt);
-                    case ExecuteAtNotWitnessed:
-                        Invariants.checkState(executeAt == null);
                     case ExecuteAtErased:
                     case ExecuteAtUnknown:
                         break;
@@ -410,7 +419,7 @@
                 case CleaningUp:
                     break;
                 case ReadyToExclude:
-                    Invariants.checkState(!validate.saveStatus().hasBeen(Status.Committed) || validate.asCommitted().waitingOn == null);
+                    Invariants.checkState(validate.saveStatus() != SaveStatus.Committed || validate.asCommitted().waitingOn == null);
                     break;
                 case WaitingToExecute:
                 case ReadyToExecute:
@@ -475,6 +484,11 @@
     public abstract Listeners.Immutable<DurableAndIdempotentListener> durableListeners();
     public abstract SaveStatus saveStatus();
 
+    /**
+     * Only meaningful when txnId.kind().awaitsOnlyDeps()
+     */
+    public Timestamp executesAtLeast() { return executeAt(); }
+
     static boolean isSameClass(Command command, Class<? extends Command> klass)
     {
         return command.getClass() == klass;
@@ -502,11 +516,6 @@
             throw new IllegalArgumentException(errorMsg + format(" expected %s got %s", klass.getSimpleName(), command.getClass().getSimpleName()));
     }
 
-    public PreLoadContext contextForSelf()
-    {
-        return contextForCommand(this);
-    }
-
     public abstract Timestamp executeAt();
     public abstract Ballot acceptedOrCommitted();
 
@@ -715,7 +724,7 @@
         }
     }
 
-    public static final class Truncated extends AbstractCommand
+    public static class Truncated extends AbstractCommand
     {
         @Nullable final Timestamp executeAt;
         @Nullable final Writes writes;
@@ -787,22 +796,41 @@
             Invariants.checkArgument(command.known().executeAt.isDecidedAndKnownToExecute());
             if (route == null) route = Route.castToNonNullFullRoute(command.route());
             Durability durability = Durability.mergeAtLeast(command.durability(), ShardUniversal);
+            if (command.txnId().kind().awaitsOnlyDeps())
+            {
+                Timestamp executesAtLeast = command.hasBeen(Stable) ? command.executesAtLeast() : null;
+                return validate(new TruncatedAwaitsOnlyDeps(command.txnId(), SaveStatus.TruncatedApply, durability, route, command.executeAt(), EMPTY, null, null, executesAtLeast));
+            }
             return validate(new Truncated(command.txnId(), SaveStatus.TruncatedApply, durability, route, command.executeAt(), EMPTY, null, null));
         }
 
         public static Truncated truncatedApplyWithOutcome(Executed command)
         {
             Durability durability = Durability.mergeAtLeast(command.durability(), ShardUniversal);
+            if (command.txnId().kind().awaitsOnlyDeps())
+                return validate(new TruncatedAwaitsOnlyDeps(command.txnId(), SaveStatus.TruncatedApplyWithOutcome, durability, command.route(), command.executeAt(), EMPTY, command.writes, command.result, command.executesAtLeast()));
             return validate(new Truncated(command.txnId(), SaveStatus.TruncatedApplyWithOutcome, durability, command.route(), command.executeAt(), EMPTY, command.writes, command.result));
         }
 
         public static Truncated truncatedApply(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result)
         {
+            Invariants.checkArgument(!common.txnId().kind().awaitsOnlyDeps());
+            Durability durability = checkTruncatedApplyInvariants(common, saveStatus, executeAt);
+            return validate(new Truncated(common.txnId(), saveStatus, durability, common.route(), executeAt, EMPTY, writes, result));
+        }
+
+        public static Truncated truncatedApply(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result, Timestamp dependencyExecutesAt)
+        {
+            Invariants.checkArgument(common.txnId().kind().awaitsOnlyDeps());
+            Durability durability = checkTruncatedApplyInvariants(common, saveStatus, executeAt);
+            return validate(new TruncatedAwaitsOnlyDeps(common.txnId(), saveStatus, durability, common.route(), executeAt, EMPTY, writes, result, dependencyExecutesAt));
+        }
+
+        private static Durability checkTruncatedApplyInvariants(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt)
+        {
             Invariants.checkArgument(executeAt != null);
             Invariants.checkArgument(saveStatus == SaveStatus.TruncatedApply || saveStatus == SaveStatus.TruncatedApplyWithDeps || saveStatus == SaveStatus.TruncatedApplyWithOutcome);
-            // TODO review Is this correctly handling all three versions of truncatedApplyStatus? Should writes be null some of the time?
-            Durability durability = Durability.mergeAtLeast(common.durability(), ShardUniversal);
-            return validate(new Truncated(common.txnId(), saveStatus, durability, common.route(), executeAt, EMPTY, writes, result));
+            return Durability.mergeAtLeast(common.durability(), ShardUniversal);
         }
 
         public static Truncated invalidated(Command command)
@@ -863,6 +891,43 @@
         }
     }
 
+    public static class TruncatedAwaitsOnlyDeps extends Truncated
+    {
+        @Nullable final Timestamp executesAtLeast;
+
+        public TruncatedAwaitsOnlyDeps(CommonAttributes commonAttributes, SaveStatus saveStatus, @Nullable Timestamp executeAt, @Nullable Writes writes, @Nullable Result result, @Nullable Timestamp executesAtLeast)
+        {
+            super(commonAttributes, saveStatus, executeAt, writes, result);
+            this.executesAtLeast = executesAtLeast;
+        }
+
+        public TruncatedAwaitsOnlyDeps(TxnId txnId, SaveStatus saveStatus, Durability durability, @Nullable Route<?> route, @Nullable Timestamp executeAt, Listeners.Immutable listeners, @Nullable Writes writes, @Nullable Result result, @Nullable Timestamp executesAtLeast)
+        {
+            super(txnId, saveStatus, durability, route, executeAt, listeners, writes, result);
+            this.executesAtLeast = executesAtLeast;
+        }
+
+        public Timestamp executesAtLeast()
+        {
+            return executesAtLeast;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (!super.equals(o)) return false;
+            return Objects.equals(executesAtLeast, ((TruncatedAwaitsOnlyDeps)o).executesAtLeast);
+        }
+
+        @Override
+        public boolean isEqualOrFuller(Command command)
+        {
+            if (!super.isEqualOrFuller(command)) return false;
+            return Objects.equals(executesAtLeast, ((TruncatedAwaitsOnlyDeps)command).executesAtLeast);
+        }
+
+    }
+
     public static class PreAccepted extends AbstractCommand
     {
         private final Timestamp executeAt;
@@ -1030,6 +1095,14 @@
         }
 
         @Override
+        public Timestamp executesAtLeast()
+        {
+            if (!txnId().kind().awaitsOnlyDeps()) return executeAt();
+            if (status().hasBeen(Stable)) return waitingOn.executeAtLeast(executeAt());
+            return null;
+        }
+
+        @Override
         public Command updateAttributes(CommonAttributes attrs, Ballot promised)
         {
             return validate(new Committed(attrs, saveStatus(), executeAt(), promised, acceptedOrCommitted(), waitingOn()));
@@ -1208,6 +1281,11 @@
             return null;
         }
 
+        public Timestamp executeAtLeast(Timestamp ifNull)
+        {
+            return ifNull;
+        }
+
         public static WaitingOn none(Deps deps)
         {
             ImmutableBitSet empty = new ImmutableBitSet(deps.txnIdCount());
@@ -1585,6 +1663,12 @@
         }
 
         @Override
+        public Timestamp executeAtLeast(Timestamp ifNull)
+        {
+            return executeAtLeast != null ? executeAtLeast : ifNull;
+        }
+
+        @Override
         public boolean equals(Object other)
         {
             return other.getClass() == WaitingOnWithExecuteAt.class && this.equals((WaitingOnWithExecuteAt) other);
diff --git a/accord-core/src/main/java/accord/local/CommandStore.java b/accord-core/src/main/java/accord/local/CommandStore.java
index 2ed1481..cf774b8 100644
--- a/accord-core/src/main/java/accord/local/CommandStore.java
+++ b/accord-core/src/main/java/accord/local/CommandStore.java
@@ -30,7 +30,7 @@
 import accord.local.CommandStores.RangesForEpoch;
 import accord.primitives.Keys;
 import accord.primitives.Range;
-import accord.primitives.Txn.Kind.Kinds;
+import accord.primitives.Routables;
 import accord.utils.async.AsyncChain;
 
 import accord.api.ConfigurationService.EpochReady;
@@ -67,7 +67,7 @@
 import accord.utils.async.AsyncResults;
 
 import static accord.api.ConfigurationService.EpochReady.DONE;
-import static accord.local.KeyHistory.DEPS;
+import static accord.local.KeyHistory.COMMANDS;
 import static accord.local.PreLoadContext.contextFor;
 import static accord.local.PreLoadContext.empty;
 import static accord.primitives.AbstractRanges.UnionMode.MERGE_ADJACENT;
@@ -149,9 +149,11 @@
     // TODO (expected): schedule regular pruning of these collections
     // bootstrapBeganAt and shardDurableAt are both canonical data sets mostly used for debugging / constructing
     private NavigableMap<TxnId, Ranges> bootstrapBeganAt = ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY); // additive (i.e. once inserted, rolled-over until invalidated, and the floor entry contains additions)
+    // TODO (required): updates to these variables should be buffered like with Command updates in SafeCommandStore, to be rolled-back in event of a failure processing the command.
     private RedundantBefore redundantBefore = RedundantBefore.EMPTY;
     // TODO (expected): store this only once per node
     private DurableBefore durableBefore = DurableBefore.EMPTY;
+    private MaxConflicts maxConflicts = MaxConflicts.EMPTY;
     protected RangesForEpoch rangesForEpoch;
 
     // TODO (desired): merge with redundantBefore?
@@ -223,21 +225,6 @@
     public abstract <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> apply);
     public abstract void shutdown();
 
-    private static Timestamp maxApplied(SafeCommandStore safeStore, Seekables<?, ?> keysOrRanges, Ranges slice)
-    {
-        return safeStore.mapReduce(keysOrRanges, slice, KeyHistory.NONE, Kinds.Ws,
-                                   SafeCommandStore.TestTimestamp.STARTED_AFTER, Timestamp.NONE,
-                                   SafeCommandStore.TestDep.ANY_DEPS, null,
-                                   Status.PreApplied, Status.Truncated,
-                                   (p1, key, txnId, executeAt, status, deps, max) -> Timestamp.max(max, executeAt),
-                                   null, Timestamp.NONE, Timestamp.MAX::equals);
-    }
-
-    public AsyncChain<Timestamp> maxAppliedFor(Seekables<?, ?> keysOrRanges, Ranges slice)
-    {
-        return submit(PreLoadContext.contextFor(keysOrRanges), safeStore -> maxApplied(safeStore, keysOrRanges, slice));
-    }
-
     // implementations are expected to override this for persistence
     protected void setRejectBefore(ReducingRangeMap<Timestamp> newRejectBefore)
     {
@@ -269,7 +256,6 @@
         this.durableBefore = durableBefore;
     }
 
-
     /**
      * To be overridden by implementations, to ensure the new state is persisted.
      */
@@ -279,6 +265,20 @@
     }
 
     /**
+     * To be overridden by implementations, to ensure the new state is persisted.
+     */
+    protected void setMaxConflicts(MaxConflicts maxConflicts)
+    {
+        this.maxConflicts = maxConflicts;
+    }
+
+    protected void updateMaxConflicts(Command prev, Command updated, Seekables<?, ?> keysOrRanges)
+    {
+        if (updated.executeAt() != null && (prev == null || prev.executeAt() == null || prev.executeAt().compareTo(updated.executeAt()) < 0))
+            setMaxConflicts(maxConflicts.update(keysOrRanges != null ? keysOrRanges : updated.partialTxn().keys(), updated.executeAt()));
+    }
+
+    /**
      * This method may be invoked on a non-CommandStore thread
      */
     protected synchronized void setSafeToRead(NavigableMap<Timestamp, Ranges> newSafeToRead)
@@ -303,6 +303,9 @@
         setRedundantBefore(newRedundantBefore);
     }
 
+    /**
+     * We expect keys to be sliced to those owned by the replica in the coordination epoch
+     */
     final Timestamp preaccept(TxnId txnId, Seekables<?, ?> keys, SafeCommandStore safeStore, boolean permitFastPath)
     {
         NodeTimeService time = safeStore.time();
@@ -321,11 +324,17 @@
 
         // TODO (expected): reject if any transaction exists with a higher timestamp OR a higher epoch
         //   this permits us to agree fast path decisions across epoch changes
-        Timestamp maxConflict = safeStore.maxConflict(keys, safeStore.ranges().coordinates(txnId));
-        if (permitFastPath && txnId.compareTo(maxConflict) > 0 && txnId.epoch() >= time.epoch())
+        // TODO (expected): we should (perhaps) separate conflicts for reads and writes
+        Timestamp minNonConflicting = maxConflicts.get(keys);
+        if (permitFastPath && txnId.compareTo(minNonConflicting) >= 0 && txnId.epoch() >= time.epoch())
             return txnId;
 
-        return time.uniqueNow(maxConflict);
+        return time.uniqueNow(minNonConflicting);
+    }
+
+    public Timestamp maxConflict(Routables<?> keysOrRanges)
+    {
+        return maxConflicts.get(keysOrRanges);
     }
 
     protected void unsafeRunIn(Runnable fn)
@@ -443,6 +452,7 @@
         TxnId id = TxnId.fromValues(epoch - 1, 0, node.id());
         Timestamp before = Timestamp.minForEpoch(epoch);
         FullRoute<?> route = node.computeRoute(id, ranges);
+        // TODO (required): we need to ensure anyone we receive a reply from proposes newer timestamps for anything we don't see
         CollectDeps.withDeps(node, id, route, ranges, before, (deps, fail) -> {
             if (fail != null)
             {
@@ -452,7 +462,7 @@
             {
                 // TODO (correcness) : PreLoadContext only works with Seekables, which doesn't allow mixing Keys and Ranges... But Deps has both Keys AND Ranges!
                 // ATM all known implementations store ranges in-memory, but this will not be true soon, so this will need to be addressed
-                execute(contextFor(null, deps.txnIds(), deps.keyDeps.keys(), DEPS), safeStore -> {
+                execute(contextFor(null, deps.txnIds(), deps.keyDeps.keys(), COMMANDS), safeStore -> {
                     safeStore.registerHistoricalTransactions(deps);
                 }).begin((success, fail2) -> {
                     if (fail2 != null) fetchMajorityDeps(coordination, node, epoch, ranges);
diff --git a/accord-core/src/main/java/accord/local/Commands.java b/accord-core/src/main/java/accord/local/Commands.java
index 7f19b2a..6c8b77e 100644
--- a/accord-core/src/main/java/accord/local/Commands.java
+++ b/accord-core/src/main/java/accord/local/Commands.java
@@ -42,7 +42,6 @@
 import accord.primitives.PartialTxn;
 import accord.primitives.Participants;
 import accord.primitives.Ranges;
-import accord.primitives.Routables;
 import accord.primitives.Route;
 import accord.primitives.Seekables;
 import accord.primitives.SyncPoint;
@@ -61,10 +60,8 @@
 import static accord.api.ProgressLog.ProgressShard.UnmanagedHome;
 import static accord.api.ProgressLog.ProgressShard.Unsure;
 import static accord.local.Cleanup.ERASE;
-import static accord.local.Cleanup.TRUNCATE;
 import static accord.local.Cleanup.shouldCleanup;
 import static accord.local.Command.Truncated.erased;
-import static accord.local.Command.Truncated.invalidated;
 import static accord.local.Command.Truncated.truncatedApply;
 import static accord.local.Command.Truncated.truncatedApplyWithOutcome;
 import static accord.local.Commands.EnsureAction.Add;
@@ -72,6 +69,8 @@
 import static accord.local.Commands.EnsureAction.Set;
 import static accord.local.Commands.EnsureAction.TryAdd;
 import static accord.local.Commands.EnsureAction.TrySet;
+import static accord.local.KeyHistory.TIMESTAMPS;
+import static accord.local.PreLoadContext.contextFor;
 import static accord.local.RedundantStatus.PRE_BOOTSTRAP_OR_STALE;
 import static accord.local.SaveStatus.Applying;
 import static accord.local.SaveStatus.Erased;
@@ -84,6 +83,7 @@
 import static accord.local.Status.Committed;
 import static accord.local.Status.Durability;
 import static accord.local.Status.Invalidated;
+import static accord.local.Status.KnownExecuteAt.ExecuteAtKnown;
 import static accord.local.Status.NotDefined;
 import static accord.local.Status.PreAccepted;
 import static accord.local.Status.PreApplied;
@@ -174,7 +174,7 @@
         Invariants.checkState(validate(command.status(), command, Ranges.EMPTY, preacceptRanges, shard, route, Set, partialTxn, Set, null, Ignore));
 
         // FIXME: this should go into a consumer method
-        CommonAttributes attrs = set(safeStore, command, command, Ranges.EMPTY, preacceptRanges, shard, route, partialTxn, Set, null, Ignore);
+        CommonAttributes attrs = set(command, Ranges.EMPTY, preacceptRanges, shard, route, partialTxn, Set, null, Ignore);
         if (command.executeAt() == null)
         {
             // unlike in the Accord paper, we partition shards within a node, so that to ensure a total order we must either:
@@ -183,14 +183,14 @@
             // if we are performing recovery (i.e. non-zero ballot), do not permit a fast path decision as we want to
             // invalidate any transactions that were not completed by their initial coordinator
             // TODO (desired): limit preaccept to keys we include, to avoid inflating unnecessary state
-            Timestamp executeAt = safeStore.commandStore().preaccept(txnId, partialTxn.keys(), safeStore, ballot.equals(Ballot.ZERO));
-            command = safeCommand.preaccept(attrs, executeAt, ballot);
+            Timestamp executeAt = safeStore.commandStore().preaccept(txnId, attrs.partialTxn().keys(), safeStore, ballot.equals(Ballot.ZERO));
+            command = safeCommand.preaccept(safeStore, attrs, executeAt, ballot);
             safeStore.progressLog().preaccepted(command, shard);
         }
         else
         {
             // TODO (expected, ?): in the case that we are pre-committed but had not been preaccepted/accepted, should we inform progressLog?
-            safeCommand.markDefined(attrs, ballot);
+            safeCommand.markDefined(safeStore, attrs, ballot);
         }
 
         safeStore.notifyListeners(safeCommand);
@@ -218,7 +218,7 @@
         return true;
     }
 
-    public static AcceptOutcome accept(SafeCommandStore safeStore, TxnId txnId, Ballot ballot, PartialRoute<?> route, Seekables<?, ?> keys, @Nullable RoutingKey progressKey, Timestamp executeAt, PartialDeps partialDeps)
+    public static AcceptOutcome accept(SafeCommandStore safeStore, TxnId txnId, Ballot ballot, PartialRoute<?> route, Seekables<?, ?> keysOrRanges, @Nullable RoutingKey progressKey, Timestamp executeAt, PartialDeps partialDeps)
     {
         SafeCommand safeCommand = safeStore.get(txnId, executeAt, route);
         Command command = safeCommand.current();
@@ -256,13 +256,9 @@
         // TODO (desired, clarity/efficiency): we don't need to set the route here, and perhaps we don't even need to
         //  distributed partialDeps at all, since all we gain is not waiting for these transactions to commit during
         //  recovery. We probably don't want to directly persist a Route in any other circumstances, either, to ease persistence.
-        CommonAttributes attrs = set(safeStore, command, command, coordinateRanges, acceptRanges, shard, route, null, Ignore, partialDeps, Set);
+        CommonAttributes attrs = set(command, coordinateRanges, acceptRanges, shard, route, null, Ignore, partialDeps, Set);
 
-        // set only registers by transaction keys, which we mightn't already have received
-        if (!command.known().isDefinitionKnown())
-            safeStore.register(keys, acceptRanges, command);
-
-        command = safeCommand.accept(attrs, executeAt, ballot);
+        command = safeCommand.accept(safeStore, keysOrRanges, attrs, executeAt, ballot);
         safeStore.progressLog().accepted(command, shard);
         safeStore.notifyListeners(safeCommand);
 
@@ -299,7 +295,7 @@
 
         logger.trace("{}: accepted invalidated", command.txnId());
 
-        safeCommand.acceptInvalidated(ballot);
+        safeCommand.acceptInvalidated(safeStore, ballot);
         safeStore.notifyListeners(safeCommand);
         return AcceptOutcome.Success;
     }
@@ -348,13 +344,13 @@
             return CommitOutcome.Insufficient;
 
         // FIXME: split up set
-        CommonAttributes attrs = set(safeStore, command, command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, Set);
+        CommonAttributes attrs = set(command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, Set);
 
         logger.trace("{}: committed with executeAt: {}, deps: {}", txnId, executeAt, partialDeps);
         if (newStatus == SaveStatus.Stable)
         {
             WaitingOn waitingOn = initialiseWaitingOn(safeStore, txnId, executeAt, attrs.partialDeps(), attrs.route());
-            command = safeCommand.stable(attrs, Ballot.max(command.acceptedOrCommitted(), ballot), executeAt, waitingOn);
+            command = safeCommand.stable(safeStore, attrs, Ballot.max(command.acceptedOrCommitted(), ballot), executeAt, waitingOn);
             safeStore.progressLog().stable(command, shard);
             safeStore.agent().metricsEventsListener().onStable(command);
             // TODO (expected, safety): introduce intermediate status to avoid reentry when notifying listeners (which might notify us)
@@ -363,7 +359,7 @@
         else
         {
             Invariants.checkArgument(command.acceptedOrCommitted().compareTo(ballot) <= 0);
-            command = safeCommand.commit(attrs, ballot, executeAt);
+            command = safeCommand.commit(safeStore, attrs, ballot, executeAt);
             safeStore.progressLog().precommitted(command);
             safeStore.notifyListeners(safeCommand);
             safeStore.agent().metricsEventsListener().onCommitted(command);
@@ -397,7 +393,7 @@
         if (command.route() == null || !command.route().kind().isFullRoute())
             attrs = updateRoute(command, route);
 
-        safeCommand.precommit(attrs, executeAt);
+        safeCommand.precommit(safeStore, attrs, executeAt);
         safeStore.progressLog().precommitted(command);
         safeStore.notifyListeners(safeCommand);
         logger.trace("{}: precommitted with executeAt: {}", txnId, executeAt);
@@ -426,8 +422,8 @@
         PartialDeps none = Deps.NONE.slice(coordinateRanges);
         PartialTxn partialTxn = emptyTxn.slice(coordinateRanges, true);
         Invariants.checkState(validate(command.status(), command, Ranges.EMPTY, coordinateRanges, progressShard, route, Set, partialTxn, Set, none, Set));
-        CommonAttributes newAttributes = set(safeStore, command, command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, none, Set);
-        safeCommand.stable(newAttributes, Ballot.ZERO, localSyncId, WaitingOn.EMPTY);
+        CommonAttributes newAttributes = set(command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, none, Set);
+        safeCommand.stable(safeStore, newAttributes, Ballot.ZERO, localSyncId, WaitingOn.EMPTY);
         safeStore.notifyListeners(safeCommand);
     }
 
@@ -446,8 +442,8 @@
         // TODO (desired, consider): in the case of sync points, the coordinator is unlikely to be a home shard, do we mind this? should document at least
         ProgressShard progressShard = No;
         Invariants.checkState(validate(command.status(), command, Ranges.EMPTY, coordinateRanges, progressShard, route, Set, partialTxn, Set, partialDeps, Set));
-        CommonAttributes attrs = set(safeStore, command, command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, partialDeps, Set);
-        safeCommand.stable(attrs, Ballot.ZERO, txnId, initialiseWaitingOn(safeStore, txnId, txnId, attrs.partialDeps(), route));
+        CommonAttributes attrs = set(command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, partialDeps, Set);
+        safeCommand.stable(safeStore, attrs, Ballot.ZERO, txnId, initialiseWaitingOn(safeStore, txnId, txnId, attrs.partialDeps(), route));
         maybeExecute(safeStore, safeCommand, false, true);
     }
 
@@ -460,7 +456,7 @@
 
         // NOTE: if this is ever made a non-empty txn this will introduce a potential bug where the txn is registered against CommandsForKeys
         Txn emptyTxn = safeStore.agent().emptyTxn(localSyncId.kind(), keys);
-        safeCommand.preapplied(command, command.executeAt(), command.waitingOn(), emptyTxn.execute(localSyncId, localSyncId, null), emptyTxn.result(localSyncId, localSyncId, null));
+        safeCommand.preapplied(safeStore, command, command.executeAt(), command.waitingOn(), emptyTxn.execute(localSyncId, localSyncId, null), emptyTxn.result(localSyncId, localSyncId, null));
         maybeExecute(safeStore, safeCommand, true, false);
     }
 
@@ -485,7 +481,7 @@
         else if (command.saveStatus().isUninitialised() && !safeStore.ranges().allAt(command.txnId().epoch()).intersects(scope))
             return; // don't bother propagating the invalidation to future epochs where the replica didn't already witness the command
 
-        safeCommand.commitInvalidated();
+        safeCommand.commitInvalidated(safeStore);
         safeStore.progressLog().clear(command.txnId());
         logger.trace("{}: committed invalidated", safeCommand.txnId());
         safeStore.notifyListeners(safeCommand, command, command.durableListeners(), safeCommand.transientListeners());
@@ -516,10 +512,10 @@
         if (!validate(command.status(), command, coordinateRanges, acceptRanges, shard, route, TryAdd, partialTxn, Add, partialDeps, command.hasBeen(Committed) ? TryAdd : TrySet, safeStore))
             return ApplyOutcome.Insufficient; // TODO (expected, consider): this should probably be an assertion failure if !TrySet
 
-        CommonAttributes attrs = set(safeStore, command, command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, command.hasBeen(Committed) ? TryAdd : TrySet);
+        CommonAttributes attrs = set(command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, command.hasBeen(Committed) ? TryAdd : TrySet);
 
         WaitingOn waitingOn = !command.hasBeen(Stable) ? initialiseWaitingOn(safeStore, txnId, executeAt, attrs.partialDeps(), attrs.route()) : command.asCommitted().waitingOn();
-        safeCommand.preapplied(attrs, executeAt, waitingOn, writes, result);
+        safeCommand.preapplied(safeStore, attrs, executeAt, waitingOn, writes, result);
         safeStore.notifyListeners(safeCommand);
         logger.trace("{}: apply, status set to Executed with executeAt: {}, deps: {}", txnId, executeAt, partialDeps);
 
@@ -573,7 +569,7 @@
     {
         logger.trace("{} applied, setting status to Applied and notifying listeners", txnId);
         SafeCommand safeCommand = safeStore.get(txnId);
-        safeCommand.applied();
+        safeCommand.applied(safeStore);
         safeStore.notifyListeners(safeCommand);
     }
 
@@ -638,7 +634,8 @@
     {
         CommandStore unsafeStore = safeStore.commandStore();
         TxnId txnId = command.txnId();
-        PreLoadContext context = command.contextForSelf();
+        // TODO (expected): there is some coupling going on here - concept of TIMESTAMPS only needed if implementation tracks on apply
+        PreLoadContext context = contextFor(command.txnId(), command.partialTxn().keys(), TIMESTAMPS);
         // this is sometimes called from a listener update, which will not have the keys in context
         if (safeStore.canExecuteWith(context))
         {
@@ -684,7 +681,7 @@
         {
             case Stable:
                 // TODO (desirable, efficiency): maintain distinct ReadyToRead and ReadyToWrite states
-                command = safeCommand.readyToExecute();
+                command = safeCommand.readyToExecute(safeStore);
                 logger.trace("{}: set to ReadyToExecute", command.txnId());
                 safeStore.progressLog().readyToExecute(command);
                 safeStore.notifyListeners(safeCommand);
@@ -697,7 +694,7 @@
 
                 if (intersects)
                 {
-                    safeCommand.applying();
+                    safeCommand.applying(safeStore);
                     logger.trace("{}: applying", command.txnId());
                     apply(safeStore, executed);
                     return true;
@@ -707,7 +704,7 @@
                     // TODO (desirable, performance): This could be performed immediately upon Committed
                     //      but: if we later support transitive dependency elision this could be dangerous
                     logger.trace("{}: applying no-op", command.txnId());
-                    safeCommand.applied();
+                    safeCommand.applied(safeStore);
                     if (command.txnId().kind() == ExclusiveSyncPoint)
                     {
                         Ranges ranges = safeStore.ranges().allAt(command.txnId().epoch());
@@ -775,7 +772,7 @@
         Command dependency = dependencySafeCommand.current();
         Invariants.checkState(dependency.hasBeen(PreCommitted));
         TxnId dependencyId = dependency.txnId();
-        if (waitingId.kind().awaitsOnlyDeps() && !dependency.is(Invalidated) && dependency.executeAt().compareTo(waitingId) > 0)
+        if (waitingId.kind().awaitsOnlyDeps() && dependency.known().executeAt == ExecuteAtKnown && dependency.executeAt().compareTo(waitingId) > 0)
             waitingOn.updateExecuteAtLeast(dependency.executeAt());
 
         if (dependency.hasBeen(Truncated))
@@ -859,25 +856,47 @@
     // TODO (now): document and justify all calls
     public static void setTruncatedApply(SafeCommandStore safeStore, SafeCommand safeCommand)
     {
-        setTruncatedApply(safeStore, safeCommand, null);
+        setTruncatedApply(safeStore, safeCommand, null, null);
     }
 
-    public static void setTruncatedApply(SafeCommandStore safeStore, SafeCommand safeCommand, Route<?> maybeFullRoute)
+    public static void setTruncatedApply(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable Timestamp executeAt, Route<?> maybeFullRoute)
     {
-        purge(safeStore, safeCommand, maybeFullRoute, TRUNCATE, true);
+        Command command = safeCommand.current();
+        if (command.saveStatus().compareTo(TruncatedApply) >= 0) return;
+        FullRoute<?> route = Route.tryCastToFullRoute(maybeFullRoute);
+        if (route == null) route = Route.tryCastToFullRoute(command.route());
+        if (executeAt == null) executeAt = command.executeAtIfKnown();
+        if (route == null || executeAt == null)
+        {
+            safeCommand.update(safeStore, null, erased(command));
+        }
+        else
+        {
+            CommonAttributes attributes = command.mutable().route(route);
+            if (!safeCommand.txnId().kind().awaitsOnlyDeps())
+            {
+                safeCommand.update(safeStore, null, truncatedApply(attributes, TruncatedApply, executeAt, null, null));
+            }
+            else if (safeCommand.current().saveStatus().hasBeen(Applied))
+            {
+                Timestamp executesAtLeast = safeCommand.current().executesAtLeast();
+                if (executesAtLeast == null) safeCommand.update(safeStore, null, erased(command));
+                else safeCommand.update(safeStore, null, truncatedApply(attributes, TruncatedApply, executeAt, null, null, executesAtLeast));
+            }
+        }
     }
 
     public static void setErased(SafeCommandStore safeStore, SafeCommand safeCommand)
     {
         Listeners.Immutable durableListeners = safeCommand.current().durableListeners();
-        Command command = purge(safeStore, safeCommand, null, ERASE, true);
+        Command command = purge(safeStore, safeCommand, null, null, ERASE, true);
         safeStore.notifyListeners(safeCommand, command, durableListeners, safeCommand.transientListeners());
     }
 
     /**
      * Purge all or part of the metadata for a Commmand
      */
-    public static Command purge(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable Unseekables<?> maybeFullRoute, Cleanup cleanup, boolean notifyListeners)
+    public static Command purge(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable Seekables<?, ?> keysOrRanges, @Nullable Unseekables<?> maybeFullRoute, Cleanup cleanup, boolean notifyListeners)
     {
         Command command = safeCommand.current();
 
@@ -893,7 +912,7 @@
         Command result;
         switch (cleanup)
         {
-            default: throw new AssertionError("Unexpected status: " + cleanup);
+            default: throw new AssertionError("Unexpected cleanup: " + cleanup);
             case TRUNCATE_WITH_OUTCOME:
                 Invariants.checkArgument(!command.hasBeen(Truncated));
                 if (command.hasBeen(PreApplied))
@@ -904,8 +923,9 @@
                 // TODO (expected, consider): should we downgrade to no truncation in this case? Or are we stale?
 
             case TRUNCATE:
+                // TODO (expected): consider passing through any information we have about the reason for loading, so we can infer APPLIED if !PreCommitted
                 Invariants.checkState(command.saveStatus().compareTo(TruncatedApply) < 0);
-                if (!command.hasBeen(PreCommitted)) result = invalidated(command);
+                if (!command.hasBeen(PreCommitted)) result = erased(command);
                 else result = truncatedApply(command, Route.tryCastToFullRoute(maybeFullRoute));
                 break;
 
@@ -915,7 +935,7 @@
                 break;
         }
 
-        safeCommand.set(result);
+        safeCommand.update(safeStore, keysOrRanges, result);
         safeStore.progressLog().clear(safeCommand.txnId());
         if (notifyListeners)
             safeStore.notifyListeners(safeCommand, result, command.durableListeners(), safeCommand.transientListeners());
@@ -928,7 +948,7 @@
         if (command.saveStatus().compareTo(cleanup.appliesIfNot) >= 0)
             return false;
 
-        purge(safeStore, safeCommand, maybeFullRoute, cleanup, true);
+        purge(safeStore, safeCommand, null, maybeFullRoute, cleanup, true);
         return true;
     }
 
@@ -1182,7 +1202,7 @@
         }
     }
 
-    static Command removeRedundantDependencies(SafeCommandStore safeStore, SafeCommand safeCommand, TxnId redundant)
+    static Command removeRedundantDependencies(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable TxnId redundant)
     {
         CommandStore commandStore = safeStore.commandStore();
         Command.Committed current = safeCommand.current().asCommitted();
@@ -1192,7 +1212,8 @@
             safeStore.commandStore().removeRedundantDependencies(current.route().participants(), update);
 
         // if we are a range transaction, being redundant for this transaction does not imply we are redundant for all transactions
-        update.removeWaitingOn(redundant);
+        if (redundant != null)
+            update.removeWaitingOn(redundant);
         return safeCommand.updateWaitingOn(update);
     }
 
@@ -1255,14 +1276,14 @@
     }
 
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private static CommonAttributes set(SafeCommandStore safeStore, Command command, CommonAttributes attrs,
-                                        Ranges existingRanges, Ranges additionalRanges, ProgressShard shard, Route<?> route,
+    private static CommonAttributes set(CommonAttributes attrs,
+                                        Ranges existingRanges, Ranges additionalRanges,
+                                        ProgressShard shard, Route<?> route,
                                         @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn,
                                         @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps)
     {
         Invariants.checkState(shard != Unsure);
         Ranges allRanges = existingRanges.with(additionalRanges);
-
         attrs = attrs.mutable().route(Route.merge(attrs.route(), (Route)route));
 
         // TODO (soon): stop round-robin hashing; partition only on ranges
@@ -1275,15 +1296,6 @@
                 if (attrs.partialTxn() != null)
                 {
                     partialTxn = partialTxn.slice(allRanges, shard.isHome());
-                    if (command.txnId().kind().isGloballyVisible())
-                    {
-                        Invariants.checkState(attrs.partialTxn().covers(existingRanges));
-                        Routables.foldl(partialTxn.keys(), additionalRanges, (keyOrRange, p, v, i) -> {
-                            // TODO (expected, efficiency): we may register the same ranges more than once
-                            safeStore.register(keyOrRange, allRanges, command);
-                            return v;
-                        }, 0, 0, 1);
-                    }
                     attrs = attrs.mutable().partialTxn(attrs.partialTxn().with(partialTxn));
                     break;
                 }
@@ -1291,11 +1303,7 @@
             case Set:
             case TrySet:
                 // TODO (desired): only includeQuery if shard.isHome(); this affects state eviction and is low priority given size in C*
-                attrs = attrs.mutable().partialTxn(partialTxn = partialTxn.slice(allRanges, true));
-                // TODO (expected, efficiency): we may register the same ranges more than once
-                // TODO (desirable, efficiency): no need to register on PreAccept if already Accepted
-                if (command.txnId().kind().isGloballyVisible())
-                    safeStore.register(partialTxn.keys(), allRanges, command);
+                attrs = attrs.mutable().partialTxn(partialTxn.slice(allRanges, true));
                 break;
         }
 
diff --git a/accord-core/src/main/java/accord/local/DurableBefore.java b/accord-core/src/main/java/accord/local/DurableBefore.java
index ae872dc..9da0723 100644
--- a/accord-core/src/main/java/accord/local/DurableBefore.java
+++ b/accord-core/src/main/java/accord/local/DurableBefore.java
@@ -25,7 +25,6 @@
 import accord.api.RoutingKey;
 import accord.local.Status.Durability;
 import accord.primitives.Participants;
-import accord.primitives.Range;
 import accord.primitives.Ranges;
 import accord.primitives.TxnId;
 import accord.primitives.Unseekables;
@@ -154,20 +153,7 @@
             return DurableBefore.EMPTY;
 
         Entry entry = new Entry(majority, universal);
-        Builder builder = new Builder(ranges.get(0).endInclusive(), ranges.size() * 2);
-        Range prev = null;
-        for (int i = 0 ; i < ranges.size() ; ++i)
-        {
-            Range cur = ranges.get(i);
-            if (prev != null && !prev.end().equals(cur.start()))
-                builder.append(prev.end(), null, (a, b) -> a); // if we are equal to prev end, take the prev value not zero
-            builder.append(cur.start(), entry, (a, b) -> { throw new IllegalStateException(); });
-            prev = cur;
-        }
-        if (prev != null)
-            builder.append(prev.end(), null, (a, b) -> a); // if we are equal to prev end, take the prev value not zero
-
-        return builder.build();
+        return create(ranges, entry, Builder::new);
     }
 
     public static DurableBefore merge(DurableBefore a, DurableBefore b)
diff --git a/accord-core/src/main/java/accord/local/KeyHistory.java b/accord-core/src/main/java/accord/local/KeyHistory.java
index d5c9c3d..7f061d6 100644
--- a/accord-core/src/main/java/accord/local/KeyHistory.java
+++ b/accord-core/src/main/java/accord/local/KeyHistory.java
@@ -24,8 +24,8 @@
  */
 public enum KeyHistory
 {
-    DEPS,
-    ALL,
+    TIMESTAMPS,
+    COMMANDS,
     NONE;
 
     public boolean isNone()
diff --git a/accord-core/src/main/java/accord/local/Listeners.java b/accord-core/src/main/java/accord/local/Listeners.java
index 4afd6b4..1392971 100644
--- a/accord-core/src/main/java/accord/local/Listeners.java
+++ b/accord-core/src/main/java/accord/local/Listeners.java
@@ -23,6 +23,7 @@
 
 import accord.utils.DeterministicSet;
 
+// TODO (expected): these are immutable collections over sortable keys, so should simply use BTree (or arrays)
 public class Listeners<L extends Command.Listener> extends DeterministicSet<L>
 {
     public static Listeners EMPTY = new Listeners<>();
diff --git a/accord-core/src/main/java/accord/local/MaxConflicts.java b/accord-core/src/main/java/accord/local/MaxConflicts.java
new file mode 100644
index 0000000..3ccf3c3
--- /dev/null
+++ b/accord-core/src/main/java/accord/local/MaxConflicts.java
@@ -0,0 +1,96 @@
+/*
+ * 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 accord.local;
+
+import javax.annotation.Nonnull;
+
+import accord.api.RoutingKey;
+import accord.primitives.Ranges;
+import accord.primitives.Routables;
+import accord.primitives.Seekables;
+import accord.primitives.Timestamp;
+import accord.utils.ReducingIntervalMap;
+import accord.utils.ReducingRangeMap;
+
+// TODO (expected): track read/write conflicts separately
+public class MaxConflicts extends ReducingRangeMap<Timestamp>
+{
+    public static final MaxConflicts EMPTY = new MaxConflicts();
+
+    private MaxConflicts()
+    {
+        super();
+    }
+
+    private MaxConflicts(boolean inclusiveEnds, RoutingKey[] starts, Timestamp[] values)
+    {
+        super(inclusiveEnds, starts, values);
+    }
+
+    public Timestamp get(Seekables<?, ?> keysOrRanges)
+    {
+        return foldl(keysOrRanges, Timestamp::max, Timestamp.NONE);
+    }
+
+    public Timestamp get(Routables<?> keysOrRanges)
+    {
+        return foldl(keysOrRanges, Timestamp::max, Timestamp.NONE);
+    }
+
+    MaxConflicts update(Seekables<?, ?> keysOrRanges, Timestamp maxConflict)
+    {
+        return merge(this, create(keysOrRanges, maxConflict));
+    }
+
+    public static MaxConflicts create(Ranges ranges, @Nonnull Timestamp maxConflict)
+    {
+        if (ranges.isEmpty())
+            return MaxConflicts.EMPTY;
+
+        return create(ranges, maxConflict, MaxConflicts.Builder::new);
+    }
+
+    public static MaxConflicts create(Seekables<?, ?> keysOrRanges, @Nonnull Timestamp maxConflict)
+    {
+        if (keysOrRanges.isEmpty())
+            return MaxConflicts.EMPTY;
+
+        return create(keysOrRanges, maxConflict, Builder::new);
+    }
+
+    public static MaxConflicts merge(MaxConflicts a, MaxConflicts b)
+    {
+        return ReducingIntervalMap.merge(a, b, Timestamp::max, MaxConflicts.Builder::new);
+    }
+
+    static class Builder extends AbstractBoundariesBuilder<RoutingKey, Timestamp, MaxConflicts>
+    {
+        protected Builder(boolean inclusiveEnds, int capacity)
+        {
+            super(inclusiveEnds, capacity);
+        }
+
+        @SuppressWarnings("unchecked")
+        @Override
+        protected MaxConflicts buildInternal()
+        {
+            return new MaxConflicts(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new Timestamp[0]));
+        }
+    }
+}
diff --git a/accord-core/src/main/java/accord/local/Node.java b/accord-core/src/main/java/accord/local/Node.java
index 413055b..3bf2186 100644
--- a/accord-core/src/main/java/accord/local/Node.java
+++ b/accord-core/src/main/java/accord/local/Node.java
@@ -38,6 +38,8 @@
 import javax.annotation.Nullable;
 
 import accord.coordinate.CoordinateEphemeralRead;
+import accord.coordinate.CoordinationAdapter;
+import accord.coordinate.CoordinationAdapter.Factory.Step;
 import accord.utils.DeterministicSet;
 import accord.utils.Invariants;
 import com.google.common.annotations.VisibleForTesting;
@@ -59,12 +61,9 @@
 import accord.coordinate.Barrier;
 import accord.config.LocalConfig;
 import accord.coordinate.CoordinateTransaction;
-import accord.coordinate.Execute;
 import accord.coordinate.MaybeRecover;
 import accord.coordinate.Outcome;
-import accord.coordinate.Persist;
 import accord.coordinate.RecoverWithRoute;
-import accord.messages.Apply;
 import accord.messages.Callback;
 import accord.messages.LocalRequest;
 import accord.messages.Reply;
@@ -155,9 +154,7 @@
     private final ConfigurationService configService;
     private final TopologyManager topology;
     private final CommandStores commandStores;
-    private final Execute.Factory executionFactory;
-    private final Persist.Factory persistFactory;
-    private final Apply.Factory applyFactory;
+    private final CoordinationAdapter.Factory coordinationAdapters;
 
     private final LongSupplier nowSupplier;
     private final ToLongFunction<TimeUnit> nowTimeUnit;
@@ -175,7 +172,7 @@
     public Node(Id id, MessageSink messageSink, LocalRequest.Handler localRequestHandler,
                 ConfigurationService configService, LongSupplier nowSupplier, ToLongFunction<TimeUnit> nowTimeUnit,
                 Supplier<DataStore> dataSupplier, ShardDistributor shardDistributor, Agent agent, RandomSource random, Scheduler scheduler, TopologySorter.Supplier topologySorter,
-                Function<Node, ProgressLog.Factory> progressLogFactory, CommandStores.Factory factory, Execute.Factory executionFactory, Persist.Factory persistFactory, Apply.Factory applyFactory,
+                Function<Node, ProgressLog.Factory> progressLogFactory, CommandStores.Factory factory, CoordinationAdapter.Factory coordinationAdapters,
                 LocalConfig localConfig)
     {
         this.id = id;
@@ -183,9 +180,7 @@
         this.messageSink = messageSink;
         this.localRequestHandler = localRequestHandler;
         this.configService = configService;
-        this.executionFactory = executionFactory;
-        this.persistFactory = persistFactory;
-        this.applyFactory = applyFactory;
+        this.coordinationAdapters = coordinationAdapters;
         this.topology = new TopologyManager(topologySorter, id);
         this.nowSupplier = nowSupplier;
         this.nowTimeUnit = nowTimeUnit;
@@ -557,6 +552,9 @@
         messageSink.reply(replyingToNode, replyContext, send);
     }
 
+    /**
+     * TODO (required): Make sure we cannot re-issue the same txnid on startup
+     */
     public TxnId nextTxnId(Txn.Kind rw, Domain domain)
     {
         return new TxnId(uniqueNow(), rw, domain);
@@ -615,16 +613,21 @@
 
     public FullRoute<?> computeRoute(TxnId txnId, Seekables<?, ?> keysOrRanges)
     {
-        RoutingKey homeKey = trySelectHomeKey(txnId, keysOrRanges);
+        return computeRoute(txnId.epoch(), keysOrRanges);
+    }
+
+    public FullRoute<?> computeRoute(long epoch, Seekables<?, ?> keysOrRanges)
+    {
+        RoutingKey homeKey = trySelectHomeKey(epoch, keysOrRanges);
         if (homeKey == null)
-            homeKey = selectRandomHomeKey(txnId);
+            homeKey = selectRandomHomeKey(epoch);
 
         return keysOrRanges.toRoute(homeKey);
     }
 
-    private @Nullable RoutingKey trySelectHomeKey(TxnId txnId, Seekables<?, ?> keysOrRanges)
+    private @Nullable RoutingKey trySelectHomeKey(long epoch, Seekables<?, ?> keysOrRanges)
     {
-        Ranges owned = topology().localForEpoch(txnId.epoch()).ranges();
+        Ranges owned = topology().localForEpoch(epoch).ranges();
         int i = (int)keysOrRanges.findNextIntersection(0, owned, 0);
         return i >= 0 ? keysOrRanges.get(i).someIntersectingRoutingKey(owned) : null;
     }
@@ -670,12 +673,17 @@
 
     public RoutingKey selectRandomHomeKey(TxnId txnId)
     {
-        Ranges ranges = topology().localForEpoch(txnId.epoch()).ranges();
+        return selectRandomHomeKey(txnId.epoch());
+    }
+
+    public RoutingKey selectRandomHomeKey(long epoch)
+    {
+        Ranges ranges = topology().localForEpoch(epoch).ranges();
         // TODO (expected): should we try to pick keys in the same Keyspace in C*? Might want to adapt this to an Agent behaviour
         if (ranges.isEmpty()) // should not really happen, but pick some other replica to serve as home key
-            ranges = topology().globalForEpoch(txnId.epoch()).ranges();
+            ranges = topology().globalForEpoch(epoch).ranges();
         if (ranges.isEmpty())
-            throw illegalState("Unable to select a HomeKey as the topology does not have any ranges for epoch " + txnId.epoch());
+            throw illegalState("Unable to select a HomeKey as the topology does not have any ranges for epoch " + epoch);
         Range range = ranges.get(random.nextInt(ranges.size()));
         return range.someIntersectingRoutingKey(null);
     }
@@ -743,19 +751,9 @@
         scheduler.now(processMsg);
     }
 
-    public Execute.Factory executionFactory()
+    public <R> CoordinationAdapter<R> coordinationAdapter(TxnId txnId, Step step)
     {
-        return executionFactory;
-    }
-
-    public Persist.Factory persistFactory()
-    {
-        return persistFactory;
-    }
-
-    public Apply.Factory applyFactory()
-    {
-        return applyFactory;
+        return coordinationAdapters.get(txnId, step);
     }
 
     public Scheduler scheduler()
diff --git a/accord-core/src/main/java/accord/local/PreLoadContext.java b/accord-core/src/main/java/accord/local/PreLoadContext.java
index 0dea059..fedf6f2 100644
--- a/accord-core/src/main/java/accord/local/PreLoadContext.java
+++ b/accord-core/src/main/java/accord/local/PreLoadContext.java
@@ -94,12 +94,17 @@
      *  Both can be done without. For range transactions calculateDeps needs to be asynchronous anyway to support
      *  potentially large scans, and for register we do not need to load into memory, we can perform a blind write.
      */
+    // TODO (required): specify epochs for which we should load, so we can narrow to owned keys
     default Seekables<?, ?> keys() { return Keys.EMPTY; }
 
     default KeyHistory keyHistory() { return KeyHistory.NONE; }
 
     default boolean isSubsetOf(PreLoadContext superset)
     {
+        KeyHistory requiredHistory = keyHistory();
+        if (requiredHistory != KeyHistory.NONE && requiredHistory != superset.keyHistory())
+            return false;
+
         if (superset.keys().domain() != keys().domain() || !superset.keys().containsAll(keys()))
             return false;
 
diff --git a/accord-core/src/main/java/accord/local/RedundantBefore.java b/accord-core/src/main/java/accord/local/RedundantBefore.java
index 9697e6d..c3301a1 100644
--- a/accord-core/src/main/java/accord/local/RedundantBefore.java
+++ b/accord-core/src/main/java/accord/local/RedundantBefore.java
@@ -28,6 +28,7 @@
 import accord.primitives.Participants;
 import accord.primitives.Range;
 import accord.primitives.Ranges;
+import accord.primitives.RoutableKey;
 import accord.primitives.Seekables;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
@@ -43,6 +44,7 @@
 import static accord.local.RedundantStatus.NOT_OWNED;
 import static accord.local.RedundantStatus.PRE_BOOTSTRAP_OR_STALE;
 import static accord.local.RedundantStatus.SHARD_REDUNDANT;
+import static accord.utils.Invariants.illegalState;
 
 public class RedundantBefore extends ReducingRangeMap<RedundantBefore.Entry>
 {
@@ -88,11 +90,6 @@
         public final @Nonnull TxnId bootstrappedAt;
 
         /**
-         * The maximum of each of locallyAppliedOrInvalidatedBefore, shardAppliedOrInvalidatedBefore and bootstrappedAt.
-         */
-        public final @Nonnull TxnId minConflict;
-
-        /**
          * staleUntilAtLeast provides a minimum TxnId until which we know we will be unable to completely execute
          * transactions locally for the impacted range.
          *
@@ -108,7 +105,6 @@
             this.locallyAppliedOrInvalidatedBefore = locallyAppliedOrInvalidatedBefore;
             this.shardAppliedOrInvalidatedBefore = shardAppliedOrInvalidatedBefore;
             this.bootstrappedAt = bootstrappedAt;
-            this.minConflict = Timestamp.max(Timestamp.max(locallyAppliedOrInvalidatedBefore, shardAppliedOrInvalidatedBefore), bootstrappedAt);
             this.staleUntilAtLeast = staleUntilAtLeast;
         }
 
@@ -181,13 +177,6 @@
                                          : prev == FULLY          ? PARTIALLY : POST_BOOTSTRAP;
         }
 
-        static Timestamp getAndMergeMinConflict(Entry entry, @Nonnull Timestamp prev)
-        {
-            if (entry == null)
-                return prev;
-            return Timestamp.max(prev, entry.minConflict);
-        }
-
         static <T extends Deps> Deps.AbstractBuilder<T> collectDep(Entry entry, @Nonnull Deps.AbstractBuilder<T> prev, @Nonnull EpochSupplier minEpoch, @Nonnull EpochSupplier executeAt)
         {
             if (entry == null || entry.outOfBounds(minEpoch, executeAt))
@@ -221,6 +210,17 @@
             return executeRanges;
         }
 
+        static Ranges removeShardRedundant(Entry entry, @Nonnull Ranges notRedundant, TxnId txnId, @Nullable Timestamp executeAt)
+        {
+            if (entry == null || (executeAt == null ? entry.outOfBounds(txnId) : entry.outOfBounds(txnId, executeAt)))
+                return notRedundant;
+
+            if (txnId.compareTo(entry.shardAppliedOrInvalidatedBefore) < 0)
+                return notRedundant.subtract(Ranges.of(entry.range));
+
+            return notRedundant;
+        }
+
         RedundantStatus get(TxnId txnId)
         {
             // we have to first check bootstrappedAt, since we are not locally redundant for the covered range
@@ -309,6 +309,7 @@
     {
         super(inclusiveEnds, starts, values);
         staleRanges = extractStaleRanges(values);
+        checkParanoid(starts, values);
     }
 
     private static Ranges extractStaleRanges(Entry[] values)
@@ -375,8 +376,17 @@
         return Entry.get(entry, txnId, executeAt);
     }
 
-    public RedundantStatus status(TxnId txnId, EpochSupplier executeAt, Participants<?> participants)
+    public TxnId redundantBefore(RoutableKey key)
     {
+        Entry entry = get(key);
+        if (entry == null)
+            return TxnId.NONE;
+        return entry.shardAppliedOrInvalidatedBefore;
+    }
+
+    public RedundantStatus status(TxnId txnId, EpochSupplier executeAt, Participants<?> participants)
+    {   // TODO (required): consider how the use of txnId for executeAt affects exclusive sync points for cleanup
+        //    may want to issue synthetic sync points for local evaluation in later epochs
         if (executeAt == null) executeAt = txnId;
         return foldl(participants, Entry::getAndMerge, NOT_OWNED, txnId, executeAt, ignore -> false);
     }
@@ -391,11 +401,6 @@
         return foldl(participants, Entry::getAndMerge, PreBootstrapOrStale.NOT_OWNED, txnId, executeAt, r -> r == PARTIALLY);
     }
 
-    public Timestamp minConflict(Seekables<?, ?> participants)
-    {
-        return foldl(participants, Entry::getAndMergeMinConflict, Timestamp.NONE, ignore -> false);
-    }
-
     public <T extends Deps> Deps.AbstractBuilder<T> collectDeps(Seekables<?, ?> participants, Deps.AbstractBuilder<T> builder, EpochSupplier minEpoch, EpochSupplier executeAt)
     {
         return foldl(participants, Entry::collectDep, builder, minEpoch, executeAt, ignore -> false);
@@ -409,6 +414,15 @@
     /**
      * Subtract any ranges we consider stale or pre-bootstrap
      */
+    public Ranges removeShardRedundant(TxnId txnId, @Nonnull Timestamp executeAt, Ranges ranges)
+    {
+        Invariants.checkArgument(executeAt != null, "executeAt must not be null");
+        return foldl(ranges, Entry::removeShardRedundant, ranges, txnId, executeAt, r -> false);
+    }
+
+    /**
+     * Subtract any ranges we consider stale or pre-bootstrap
+     */
     public Ranges expectToExecute(TxnId txnId, @Nonnull Timestamp executeAt, Ranges ranges)
     {
         Invariants.checkArgument(executeAt != null, "executeAt must not be null");
@@ -468,9 +482,32 @@
         }
 
         @Override
+        public void append(RoutingKey start, RoutingKey end, @Nonnull Entry value)
+        {
+            if (value.range.start().compareTo(start) != 0 || value.range.end().compareTo(end) != 0)
+                throw illegalState();
+            super.append(start, end, value);
+        }
+
+        @Override
         protected RedundantBefore buildInternal()
         {
             return new RedundantBefore(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new Entry[0]));
         }
     }
+
+    private static void checkParanoid(RoutingKey[] starts, Entry[] values)
+    {
+        if (!Invariants.isParanoid())
+            return;
+
+        for (int i = 0 ; i < values.length ; ++i)
+        {
+            if (values[i] != null)
+            {
+                Invariants.checkArgument(starts[i].equals(values[i].range.start()));
+                Invariants.checkArgument(starts[i + 1].equals(values[i].range.end()));
+            }
+        }
+    }
 }
diff --git a/accord-core/src/main/java/accord/local/SafeCommand.java b/accord-core/src/main/java/accord/local/SafeCommand.java
index 6a3a2a0..ea02ce2 100644
--- a/accord-core/src/main/java/accord/local/SafeCommand.java
+++ b/accord-core/src/main/java/accord/local/SafeCommand.java
@@ -18,10 +18,13 @@
 
 package accord.local;
 
+import javax.annotation.Nullable;
+
 import accord.api.Result;
 import accord.local.Command.TransientListener;
 import accord.local.Command.Truncated;
 import accord.primitives.Ballot;
+import accord.primitives.Seekables;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 import accord.primitives.Writes;
@@ -55,7 +58,19 @@
         return txnId;
     }
 
-    private <C extends Command> C update(C update)
+    // TODO (expected): it isn't ideal to pass keysOrRanges for the special case of Accept. We can either:
+    //   1 - remove the special case that permits accept without the definition
+    //   2 - store some pseudo transaction with only the keys
+    //   3 - just come up with something a bit neater
+    <C extends Command> C update(SafeCommandStore safeStore, @Nullable Seekables<?, ?> keysOrRanges, C update)
+    {
+        Command prev = current();
+        set(update);
+        safeStore.update(prev, update, keysOrRanges);
+        return update;
+    }
+
+     private <C extends Command> C incidentalUpdate(C update)
     {
         set(update);
         return update;
@@ -63,7 +78,7 @@
 
     public Command addListener(Command.DurableAndIdempotentListener listener)
     {
-        return update(Command.addListener(current(), listener));
+        return incidentalUpdate(Command.addListener(current(), listener));
     }
 
     public void addAndInvokeListener(SafeCommandStore safeStore, TransientListener listener)
@@ -77,92 +92,92 @@
         Command current = current();
         if (!current.durableListeners().contains(listener))
             return current;
-        return update(Command.removeListener(current(), listener));
+        return incidentalUpdate(Command.removeListener(current(), listener));
     }
 
     public Command.Committed updateWaitingOn(Command.WaitingOn.Update waitingOn)
     {
-        return update(Command.updateWaitingOn(current().asCommitted(), waitingOn));
+        return incidentalUpdate(Command.updateWaitingOn(current().asCommitted(), waitingOn));
     }
 
     public Command updateAttributes(CommonAttributes attrs)
     {
-        return update(current().updateAttributes(attrs));
+        return incidentalUpdate(current().updateAttributes(attrs));
     }
 
-    public Command.PreAccepted preaccept(CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
+    public Command.PreAccepted preaccept(SafeCommandStore safeStore, CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
     {
-        return update(Command.preaccept(current(), attrs, executeAt, ballot));
+        return update(safeStore, null, Command.preaccept(current(), attrs, executeAt, ballot));
     }
 
-    public Command.Accepted markDefined(CommonAttributes attributes, Ballot promised)
+    public Command.Accepted markDefined(SafeCommandStore safeStore, CommonAttributes attributes, Ballot promised)
     {
-        return update(Command.markDefined(current(), attributes, promised));
+        return update(safeStore, null, Command.markDefined(current(), attributes, promised));
     }
 
     public Command updatePromised(Ballot promised)
     {
-        return update(current().updatePromised(promised));
+        return incidentalUpdate(current().updatePromised(promised));
     }
 
-    public Command.Accepted accept(CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
+    public Command.Accepted accept(SafeCommandStore safeStore, Seekables<?, ?> keysOrRanges, CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
     {
-        return update(Command.accept(current(), attrs, executeAt, ballot));
+        return update(safeStore, keysOrRanges, Command.accept(current(), attrs, executeAt, ballot));
     }
 
-    public Command.Accepted acceptInvalidated(Ballot ballot)
+    public Command.Accepted acceptInvalidated(SafeCommandStore safeStore, Ballot ballot)
     {
-        return update(Command.acceptInvalidated(current(), ballot));
+        return update(safeStore, null, Command.acceptInvalidated(current(), ballot));
     }
 
-    public Command.Committed commit(CommonAttributes attrs, Ballot ballot, Timestamp executeAt)
+    public Command.Committed commit(SafeCommandStore safeStore, CommonAttributes attrs, Ballot ballot, Timestamp executeAt)
     {
-        return update(Command.commit(current(), attrs, ballot, executeAt));
+        return update(safeStore, null, Command.commit(current(), attrs, ballot, executeAt));
     }
 
-    public Command.Committed stable(CommonAttributes attrs, Ballot ballot, Timestamp executeAt, Command.WaitingOn waitingOn)
+    public Command.Committed stable(SafeCommandStore safeStore, CommonAttributes attrs, Ballot ballot, Timestamp executeAt, Command.WaitingOn waitingOn)
     {
-        return update(Command.stable(current(), attrs, ballot, executeAt, waitingOn));
+        return update(safeStore, null, Command.stable(current(), attrs, ballot, executeAt, waitingOn));
     }
 
-    public Truncated commitInvalidated()
+    public Truncated commitInvalidated(SafeCommandStore safeStore)
     {
         Command current = current();
         if (current.hasBeen(Status.Truncated))
             return (Truncated) current;
 
-        return update(Truncated.invalidated(current));
+        return update(safeStore, null, Truncated.invalidated(current));
     }
 
-    public Command precommit(CommonAttributes attrs, Timestamp executeAt)
+    public Command precommit(SafeCommandStore safeStore, CommonAttributes attrs, Timestamp executeAt)
     {
-        return update(Command.precommit(attrs, current(), executeAt));
+        return update(safeStore, null, Command.precommit(attrs, current(), executeAt));
     }
 
-    public Command.Committed readyToExecute()
+    public Command.Committed readyToExecute(SafeCommandStore safeStore)
     {
-        return update(Command.readyToExecute(current().asCommitted()));
+        return update(safeStore, null, Command.readyToExecute(current().asCommitted()));
     }
 
-    public Command.Executed preapplied(CommonAttributes attrs, Timestamp executeAt, Command.WaitingOn waitingOn, Writes writes, Result result)
+    public Command.Executed preapplied(SafeCommandStore safeStore, CommonAttributes attrs, Timestamp executeAt, Command.WaitingOn waitingOn, Writes writes, Result result)
     {
-        return update(Command.preapplied(current(), attrs, executeAt, waitingOn, writes, result));
+        return update(safeStore, null, Command.preapplied(current(), attrs, executeAt, waitingOn, writes, result));
     }
 
-    public Command.Executed applying()
+    public Command.Executed applying(SafeCommandStore safeStore)
     {
-        return update(Command.applying(current().asExecuted()));
+        return update(safeStore, null, Command.applying(current().asExecuted()));
     }
 
-    public Command.Executed applied()
+    public Command.Executed applied(SafeCommandStore safeStore)
     {
-        return update(Command.applied(current().asExecuted()));
+        return update(safeStore, null, Command.applied(current().asExecuted()));
     }
 
     public Command.NotDefined uninitialised()
     {
         Invariants.checkArgument(current() == null);
-        return update(Command.NotDefined.uninitialised(txnId));
+        return incidentalUpdate(Command.NotDefined.uninitialised(txnId));
     }
 
     public Command initialise()
@@ -170,6 +185,6 @@
         Command current = current();
         if (!current.saveStatus().isUninitialised())
             return current;
-        return update(Command.NotDefined.notDefined(current, current.promised()));
+        return incidentalUpdate(Command.NotDefined.notDefined(current, current.promised()));
     }
 }
diff --git a/accord-core/src/main/java/accord/local/SafeCommandStore.java b/accord-core/src/main/java/accord/local/SafeCommandStore.java
index de68697..9c846ed 100644
--- a/accord-core/src/main/java/accord/local/SafeCommandStore.java
+++ b/accord-core/src/main/java/accord/local/SafeCommandStore.java
@@ -19,13 +19,8 @@
 package accord.local;
 
 import java.util.Iterator;
-import java.util.List;
-import java.util.function.Predicate;
-import java.util.function.Supplier;
 import javax.annotation.Nullable;
 
-import com.google.common.base.Predicates;
-
 import accord.api.Agent;
 import accord.api.DataStore;
 import accord.api.ProgressLog;
@@ -58,17 +53,17 @@
 {
     public interface CommandFunction<P1, I, O>
     {
-        O apply(P1 p1, Seekable keyOrRange, TxnId txnId, Timestamp executeAt, Status status, Supplier<List<TxnId>> depsSupplier, I in);
+        O apply(P1 p1, Seekable keyOrRange, TxnId txnId, Timestamp executeAt, I in);
     }
 
-    public enum TestTimestamp
+    public enum TestStartedAt
     {
         STARTED_BEFORE,
         STARTED_AFTER,
-        MAY_EXECUTE_BEFORE, // started before and uncommitted, or committed and executes before
-        EXECUTES_AFTER
+        ANY
     }
     public enum TestDep { WITH, WITHOUT, ANY_DEPS }
+    public enum TestStatus { ANY_STATUS, IS_PROPOSED, IS_STABLE }
 
     /**
      * If the transaction exists (with some associated data) in the CommandStore, return it. Otherwise return null.
@@ -98,8 +93,6 @@
         return maybeTruncate(safeCommand, command, txnId, null);
     }
 
-    public abstract void removeCommandFromSeekableDeps(Seekable seekable, TxnId txnId, Timestamp executeAt, Status status);
-
     // decidedExecuteAt == null if not yet PreCommitted
 
     /**
@@ -160,27 +153,45 @@
 
     protected abstract SafeCommand getInternal(TxnId txnId);
     protected abstract SafeCommand getInternalIfLoadedAndInitialised(TxnId txnId);
-
     public abstract boolean canExecuteWith(PreLoadContext context);
 
+    protected void update(Command prev, Command updated, @Nullable Seekables<?, ?> keysOrRanges)
+    {
+        SaveStatus oldSaveStatus = prev == null ? SaveStatus.Uninitialised : prev.saveStatus();
+        SaveStatus newSaveStatus = updated.saveStatus();
+        if (keysOrRanges == null && !newSaveStatus.known.definition.isKnown())
+            return;
+
+        if (newSaveStatus.status.equals(oldSaveStatus.status) && oldSaveStatus.known.definition.isKnown())
+            return;
+
+        TxnId txnId = updated.txnId();
+        if (!txnId.kind().isGloballyVisible())
+            return;
+
+//        if (keysOrRanges != null)
+//            keysOrRanges = keysOrRanges.slice(ranges().allBetween(updated.txnId(), updated.executeAt()));
+        commandStore().updateMaxConflicts(prev, updated, keysOrRanges);
+    }
+
+    /**
+     * Visits keys first and then ranges, both in ascending order.
+     * Within each key or range visits all visible txnids needed for the given scope in ascending order of queried timestamp.
+     * TODO (expected): no need for slice in most (all?) cases
+     */
+    public abstract <P1, T> T mapReduceActive(Seekables<?, ?> keys, Ranges slice, @Nullable Timestamp withLowerTxnId, Kinds kinds, CommandFunction<P1, T, T> map, P1 p1, T initialValue);
+
     /**
      * Visits keys first and then ranges, both in ascending order.
      * Within each key or range visits all unevicted txnids needed for the given scope in ascending order of queried timestamp.
      */
-    public <P1, T> T mapReduce(Seekables<?, ?> keys, Ranges slice, KeyHistory keyHistory,
-                                        Kinds testKind, TestTimestamp testTimestamp, Timestamp timestamp,
-                                        TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus,
-                                        CommandFunction<P1, T, T> map, P1 p1, T initialValue)
-    {
-        return mapReduce(keys, slice, keyHistory, testKind, testTimestamp, timestamp, testDep, depId, minStatus, maxStatus, map, p1, initialValue, Predicates.alwaysFalse());
-    }
-
-    public abstract <P1, T> T mapReduce(Seekables<?, ?> keys, Ranges slice, KeyHistory keyHistory,
-                                        Kinds testKind, TestTimestamp testTimestamp, Timestamp timestamp,
-                                        TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus,
-                                        CommandFunction<P1, T, T> map, P1 p1, T initialValue, Predicate<? super T> terminate);
-    protected abstract void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command);
-    protected abstract void register(Seekable keyOrRange, Ranges slice, Command command);
+    public abstract <P1, T> T mapReduceFull(Seekables<?, ?> keys, Ranges slice,
+                                            TxnId testTxnId,
+                                            Kinds testKind,
+                                            TestStartedAt testStartedAt,
+                                            TestDep testDep,
+                                            TestStatus testStatus,
+                                            CommandFunction<P1, T, T> map, P1 p1, T initialValue);
 
     public abstract CommandStore commandStore();
     public abstract DataStore dataStore();
@@ -188,14 +199,8 @@
     public abstract ProgressLog progressLog();
     public abstract NodeTimeService time();
     public abstract CommandStores.RangesForEpoch ranges();
-    public abstract Timestamp maxConflict(Seekables<?, ?> keys, Ranges slice);
-    public abstract void registerHistoricalTransactions(Deps deps);
-    public abstract void erase(SafeCommand safeCommand);
 
-    public long latestEpoch()
-    {
-        return time().epoch();
-    }
+    public abstract void registerHistoricalTransactions(Deps deps);
 
     public boolean isTruncated(Command command)
     {
diff --git a/accord-core/src/main/java/accord/local/Status.java b/accord-core/src/main/java/accord/local/Status.java
index bb44255..6d7431e 100644
--- a/accord-core/src/main/java/accord/local/Status.java
+++ b/accord-core/src/main/java/accord/local/Status.java
@@ -22,6 +22,7 @@
 import accord.primitives.Ballot;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
+import accord.utils.Invariants;
 
 import java.util.List;
 import java.util.Objects;
@@ -47,7 +48,7 @@
 {
     NotDefined        (None,      Nothing),
     PreAccepted       (PreAccept, DefinitionAndRoute),
-    AcceptedInvalidate(Accept,    Maybe,             DefinitionUnknown, ExecuteAtNotWitnessed, DepsUnknown,     Unknown), // may or may not have witnessed
+    AcceptedInvalidate(Accept,    Maybe,             DefinitionUnknown, ExecuteAtUnknown,      DepsUnknown,  Unknown), // may or may not have witnessed
     Accepted          (Accept,    Covering,          DefinitionUnknown, ExecuteAtProposed,     DepsProposed, Unknown), // may or may not have witnessed
 
     /**
@@ -124,7 +125,7 @@
      */
     public static class Known
     {
-        public static final Known Nothing            = new Known(Maybe, DefinitionUnknown, ExecuteAtNotWitnessed, DepsUnknown, Unknown);
+        public static final Known Nothing            = new Known(Maybe, DefinitionUnknown, ExecuteAtUnknown, DepsUnknown, Unknown);
         // TODO (expected): deprecate DefinitionOnly
         public static final Known DefinitionOnly     = new Known(Maybe, DefinitionKnown,   ExecuteAtUnknown, DepsUnknown, Unknown);
         public static final Known DefinitionAndRoute = new Known(Full,  DefinitionKnown,   ExecuteAtUnknown, DepsUnknown, Unknown);
@@ -155,6 +156,7 @@
             this.executeAt = executeAt;
             this.deps = deps;
             this.outcome = outcome;
+            checkInvariants();
         }
 
         public Known atLeast(Known that)
@@ -415,6 +417,13 @@
         {
             return outcome.isInvalidated();
         }
+
+        public void checkInvariants()
+        {
+            if (outcome.isInvalidated()) Invariants.checkState( deps != DepsKnown && executeAt != ExecuteAtKnown);
+            else if (outcome.isOrWasApply()) Invariants.checkState(deps != NoDeps && executeAt != NoExecuteAt);
+            Invariants.checkState(!isDefinitionKnown() || hasFullRoute());
+        }
     }
 
     public enum KnownRoute
@@ -466,11 +475,6 @@
     public enum KnownExecuteAt
     {
         /**
-         * No decision is known to have been reached. The transaction has not been witnessed so executeAt is null.
-         */
-        ExecuteAtNotWitnessed,
-
-        /**
          * No decision is known to have been reached. If executeAt is not null, it represents either when
          * the transaction was witnessed, or some earlier ExecuteAtProposed that was invalidated by AcceptedInvalidate
          */
diff --git a/accord-core/src/main/java/accord/messages/Accept.java b/accord-core/src/main/java/accord/messages/Accept.java
index 3e6aa9b..1bbc878 100644
--- a/accord-core/src/main/java/accord/messages/Accept.java
+++ b/accord-core/src/main/java/accord/messages/Accept.java
@@ -155,7 +155,7 @@
     @Override
     public KeyHistory keyHistory()
     {
-        return KeyHistory.DEPS;
+        return KeyHistory.COMMANDS;
     }
 
     @Override
@@ -181,6 +181,7 @@
 
         public final AcceptOutcome outcome;
         public final Ballot supersededBy;
+        // TODO (expected): only send back deps that weren't in those we received
         public final @Nullable PartialDeps deps;
 
         private AcceptReply(AcceptOutcome outcome)
diff --git a/accord-core/src/main/java/accord/messages/Apply.java b/accord-core/src/main/java/accord/messages/Apply.java
index cfc9f91..96d57f1 100644
--- a/accord-core/src/main/java/accord/messages/Apply.java
+++ b/accord-core/src/main/java/accord/messages/Apply.java
@@ -23,17 +23,17 @@
 import accord.api.Result;
 import accord.api.RoutingKey;
 import accord.local.Commands;
+import accord.local.KeyHistory;
 import accord.local.Node;
 import accord.local.Node.Id;
 import accord.local.SafeCommand;
 import accord.local.SafeCommandStore;
 import accord.messages.Apply.ApplyReply;
 import accord.primitives.Deps;
-import accord.primitives.Keys;
+import accord.primitives.FullRoute;
 import accord.primitives.PartialDeps;
 import accord.primitives.PartialRoute;
 import accord.primitives.PartialTxn;
-import accord.primitives.Ranges;
 import accord.primitives.Route;
 import accord.primitives.Seekables;
 import accord.primitives.Timestamp;
@@ -49,15 +49,15 @@
     public static final Factory FACTORY = Apply::new;
     public static class SerializationSupport
     {
-        public static Apply create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Kind kind, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, PartialTxn txn, Writes writes, Result result)
+        public static Apply create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Kind kind, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, PartialTxn txn, @Nullable FullRoute<?> fullRoute, Writes writes, Result result)
         {
-            return new Apply(kind, txnId, scope, waitForEpoch, keys, executeAt, deps, txn, writes, result);
+            return new Apply(kind, txnId, scope, waitForEpoch, keys, executeAt, deps, txn, fullRoute, writes, result);
         }
     }
 
     public interface Factory
     {
-        Apply create(Kind kind, Id to, Topologies participates, Topologies executes, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result);
+        Apply create(Kind kind, Id to, Topologies participates, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result);
     }
 
     public final Kind kind;
@@ -65,39 +65,40 @@
     public final Seekables<?, ?> keys;
     public final PartialDeps deps; // TODO (expected): this should be nullable, and only included if we did not send Commit (or if sending Maximal apply)
     public final @Nullable PartialTxn txn;
+    public final @Nullable FullRoute<?> fullRoute;
     public final Writes writes;
     public final Result result;
 
     public enum Kind { Minimal, Maximal }
 
-    protected Apply(Kind kind, Id to, Topologies participates, Topologies executes, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+    protected Apply(Kind kind, Id to, Topologies participates, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
     {
         super(to, participates, route, txnId);
         Invariants.checkState(txnId.kind() != Txn.Kind.Write || writes != null);
-        Ranges slice = kind == Kind.Maximal || executes == participates ? scope.covering() : executes.computeRangesForNode(to);
         // TODO (desired): it's wasteful to encode the full set of ranges owned by the recipient node;
         //     often it will be cheaper to include the FullRoute for Deps scope (or come up with some other safety-preserving encoding scheme)
         this.kind = kind;
-        this.deps = deps.slice(slice);
-        this.keys = txn.keys().slice(slice);
-        this.txn = kind == Kind.Maximal ? txn.slice(slice, true) : null;
+        this.deps = deps.slice(scope.covering());
+        this.keys = txn.keys().slice(scope.covering());
+        this.txn = kind == Kind.Maximal ? txn.slice(scope.covering(), true) : null;
+        this.fullRoute = kind == Kind.Maximal ? route : null;
         this.executeAt = executeAt;
         this.writes = writes;
         this.result = result;
     }
 
-    public static void sendMaximal(Node node, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+    public static void sendMaximal(Node node, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
     {
         Topologies executes = executes(node, route, executeAt);
         Topologies participates = participates(node, route, txnId, executeAt, executes);
-        node.send(participates.nodes(), to -> applyMaximal(FACTORY, to, participates, executes, txnId, route, txn, executeAt, deps, writes, result));
+        node.send(participates.nodes(), to -> applyMaximal(FACTORY, to, participates, txnId, route, txn, executeAt, deps, writes, result));
     }
 
-    public static void sendMaximal(Node node, Id to, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+    public static void sendMaximal(Node node, Id to, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
     {
         Topologies executes = executes(node, route, executeAt);
         Topologies participates = participates(node, route, txnId, executeAt, executes);
-        node.send(to, applyMaximal(FACTORY, to, participates, executes, txnId, route, txn, executeAt, deps, writes, result));
+        node.send(to, applyMaximal(FACTORY, to, participates, txnId, route, txn, executeAt, deps, writes, result));
     }
 
     public static Topologies executes(Node node, Unseekables<?> route, Timestamp executeAt)
@@ -110,17 +111,17 @@
         return txnId.epoch() == executeAt.epoch() ? executes : node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
     }
 
-    public static Apply applyMinimal(Factory factory, Id to, Topologies sendTo, Topologies applyTo, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
+    public static Apply applyMinimal(Factory factory, Id to, Topologies all, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
     {
-        return factory.create(Kind.Minimal, to, sendTo, applyTo, txnId, route, txn, executeAt, stableDeps, writes, result);
+        return factory.create(Kind.Minimal, to, all, txnId, route, txn, executeAt, stableDeps, writes, result);
     }
 
-    public static Apply applyMaximal(Factory factory, Id to, Topologies participates, Topologies executes, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
+    public static Apply applyMaximal(Factory factory, Id to, Topologies participates, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
     {
-        return factory.create(Kind.Maximal, to, participates, executes, txnId, route, txn, executeAt, stableDeps, writes, result);
+        return factory.create(Kind.Maximal, to, participates, txnId, route, txn, executeAt, stableDeps, writes, result);
     }
 
-    protected Apply(Kind kind, TxnId txnId, PartialRoute<?> route, long waitForEpoch, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, @Nullable PartialTxn txn, Writes writes, Result result)
+    protected Apply(Kind kind, TxnId txnId, PartialRoute<?> route, long waitForEpoch, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, @Nullable PartialTxn txn, @Nullable FullRoute<?> fullRoute, Writes writes, Result result)
     {
         super(txnId, route, waitForEpoch);
         this.kind = kind;
@@ -128,6 +129,7 @@
         this.deps = deps;
         this.keys = keys;
         this.txn = txn;
+        this.fullRoute = fullRoute;
         this.writes = writes;
         this.result = result;
     }
@@ -143,18 +145,18 @@
     @Override
     public ApplyReply apply(SafeCommandStore safeStore)
     {
-        return apply(safeStore, txn, txnId, executeAt, deps, scope, writes, result, progressKey);
+        return apply(safeStore, txn, txnId, executeAt, deps, fullRoute != null ? fullRoute : scope, writes, result, progressKey);
     }
 
-    public static ApplyReply apply(SafeCommandStore safeStore, PartialTxn txn, TxnId txnId, Timestamp executeAt, PartialDeps deps, PartialRoute<?> scope, Writes writes, Result result, RoutingKey progressKey)
+    public static ApplyReply apply(SafeCommandStore safeStore, PartialTxn txn, TxnId txnId, Timestamp executeAt, PartialDeps deps, Route<?> route, Writes writes, Result result, RoutingKey progressKey)
     {
-        SafeCommand safeCommand = safeStore.get(txnId, executeAt, scope);
-        return apply(safeStore, safeCommand, txn, txnId, executeAt, deps, scope, writes, result, progressKey);
+        SafeCommand safeCommand = safeStore.get(txnId, executeAt, route);
+        return apply(safeStore, safeCommand, txn, txnId, executeAt, deps, route, writes, result, progressKey);
     }
 
-    public static ApplyReply apply(SafeCommandStore safeStore, SafeCommand safeCommand, PartialTxn txn, TxnId txnId, Timestamp executeAt, PartialDeps deps, PartialRoute<?> scope, Writes writes, Result result, RoutingKey progressKey)
+    public static ApplyReply apply(SafeCommandStore safeStore, SafeCommand safeCommand, PartialTxn txn, TxnId txnId, Timestamp executeAt, PartialDeps deps, Route<?> route, Writes writes, Result result, RoutingKey progressKey)
     {
-        switch (Commands.apply(safeStore, safeCommand, txnId, scope, progressKey, executeAt, deps, txn, writes, result))
+        switch (Commands.apply(safeStore, safeCommand, txnId, route, progressKey, executeAt, deps, txn, writes, result))
         {
             default:
             case Insufficient:
@@ -187,8 +189,13 @@
     @Override
     public Seekables<?, ?> keys()
     {
-        if (txn == null) return Keys.EMPTY;
-        return txn.keys();
+        return keys;
+    }
+
+    @Override
+    public KeyHistory keyHistory()
+    {
+        return KeyHistory.COMMANDS;
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java b/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java
index d5c4041..ec06085 100644
--- a/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java
+++ b/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java
@@ -27,16 +27,23 @@
 import accord.api.Result;
 import accord.api.RoutingKey;
 import accord.local.CommandStore;
+import accord.local.Node;
 import accord.local.PreLoadContext;
 import accord.local.SafeCommandStore;
 import accord.messages.Apply.ApplyReply;
+import accord.primitives.Deps;
+import accord.primitives.FullRoute;
 import accord.primitives.PartialDeps;
-import accord.primitives.PartialRoute;
+import accord.primitives.PartialTxn;
+import accord.primitives.Participants;
 import accord.primitives.Ranges;
 import accord.primitives.Seekables;
+import accord.primitives.Txn;
 import accord.primitives.TxnId;
 import accord.primitives.Writes;
+import accord.topology.Topologies;
 
+import static accord.messages.TxnRequest.computeScope;
 import static accord.utils.Invariants.illegalState;
 
 /*
@@ -52,28 +59,40 @@
     @SuppressWarnings("unused")
     public static class SerializerSupport
     {
-        public static ApplyThenWaitUntilApplied create(TxnId txnId, PartialRoute<?> route, PartialDeps deps, Seekables<?, ?> partialTxnKeys, Writes writes, Result result, boolean notifyAgent)
+        public static ApplyThenWaitUntilApplied create(TxnId txnId, Participants<?> readScope, long executeAtEpoch, FullRoute<?> route, PartialTxn txn, PartialDeps deps, Writes writes, Result result, Seekables<?, ?> notify)
         {
-            return new ApplyThenWaitUntilApplied(txnId, route, deps, partialTxnKeys, writes, result, notifyAgent);
+            return new ApplyThenWaitUntilApplied(txnId, readScope, executeAtEpoch, route, txn, deps, writes, result, notify);
         }
     }
 
-    public final PartialRoute<?> route;
+    public final FullRoute<?> route;
+    public final PartialTxn txn;
     public final PartialDeps deps;
     public final Writes writes;
-    public final Result txnResult;
-    public final boolean notifyAgent;
-    public final Seekables<?, ?> partialTxnKeys;
+    public final Result result;
+    public final Seekables<?, ?> notify;
 
-    ApplyThenWaitUntilApplied(TxnId txnId, PartialRoute<?> route, PartialDeps deps, Seekables<?, ?> partialTxnKeys, Writes writes, Result txnResult, boolean notifyAgent)
+    public ApplyThenWaitUntilApplied(Node.Id to, Topologies topologies, FullRoute<?> route, TxnId txnId, Txn txn, Deps deps, Participants<?> readScope, long executeAtEpoch, Writes writes, Result result, Seekables<?, ?> notify)
     {
-        super(txnId, partialTxnKeys.toParticipants(), txnId.epoch());
+        super(to, topologies, txnId, readScope, executeAtEpoch);
+        Ranges slice = computeScope(to, topologies, null, 0, (i,r)->r, Ranges::with);
         this.route = route;
+        this.txn = txn.slice(slice, true);
+        this.deps = deps.slice(slice);
+        this.writes = writes;
+        this.result = result;
+        this.notify = notify == null ? null : notify.slice(slice);
+    }
+
+    protected ApplyThenWaitUntilApplied(TxnId txnId, Participants<?> readScope, long executeAtEpoch, FullRoute<?> route, PartialTxn txn, PartialDeps deps, Writes writes, Result result, Seekables<?, ?> notify)
+    {
+        super(txnId, readScope, executeAtEpoch);
+        this.route = route;
+        this.txn = txn;
         this.deps = deps;
         this.writes = writes;
-        this.txnResult = txnResult;
-        this.notifyAgent = notifyAgent;
-        this.partialTxnKeys = partialTxnKeys;
+        this.result = result;
+        this.notify = notify;
     }
 
     @Override
@@ -86,7 +105,7 @@
     public CommitOrReadNack apply(SafeCommandStore safeStore)
     {
         RoutingKey progressKey = TxnRequest.progressKey(node, txnId.epoch(), txnId, route);
-        ApplyReply applyReply = Apply.apply(safeStore, null, txnId, txnId, deps, route, writes, txnResult, progressKey);
+        ApplyReply applyReply = Apply.apply(safeStore, txn, txnId, txnId, deps, route, writes, result, progressKey);
         switch (applyReply)
         {
             default:
@@ -94,6 +113,7 @@
             case Insufficient:
                 throw illegalState("ApplyThenWaitUntilApplied is always sent with a maximal `Commit` so how can `Apply` have an `Insufficient` result");
             case Redundant:
+                // TODO (required): redundant is not necessarily safe for awaitsOnlyDeps commands as might need a future epoch
             case Applied:
                 // In both cases it's fine to continue to process and return a response saying
                 // things were applied
@@ -115,8 +135,9 @@
     @Override
     protected void onAllSuccess(@Nullable Ranges unavailable, @Nullable Data data, @Nullable Throwable fail)
     {
-        if (notifyAgent)
-            node.agent().onLocalBarrier(partialTxnKeys, txnId);
+        // TODO (expected): don't like the coupling going on here
+        if (notify != null)
+            node.agent().onLocalBarrier(notify, txnId);
         super.onAllSuccess(unavailable, data, fail);
     }
 
@@ -129,7 +150,7 @@
     @Override
     public String toString()
     {
-        return "WaitForDependenciesThenApply{" +
+        return "ApplyThenWaitUntilApplied{" +
                 "txnId:" + txnId +
                 '}';
     }
diff --git a/accord-core/src/main/java/accord/messages/BeginRecovery.java b/accord-core/src/main/java/accord/messages/BeginRecovery.java
index 5979bd7..c20c6f3 100644
--- a/accord-core/src/main/java/accord/messages/BeginRecovery.java
+++ b/accord-core/src/main/java/accord/messages/BeginRecovery.java
@@ -41,11 +41,11 @@
 
 import static accord.local.SafeCommandStore.TestDep.WITH;
 import static accord.local.SafeCommandStore.TestDep.WITHOUT;
-import static accord.local.SafeCommandStore.TestTimestamp.EXECUTES_AFTER;
-import static accord.local.SafeCommandStore.TestTimestamp.STARTED_AFTER;
-import static accord.local.SafeCommandStore.TestTimestamp.STARTED_BEFORE;
-import static accord.local.Status.Accepted;
-import static accord.local.Status.Committed;
+import static accord.local.SafeCommandStore.TestStartedAt.ANY;
+import static accord.local.SafeCommandStore.TestStatus.IS_STABLE;
+import static accord.local.SafeCommandStore.TestStatus.IS_PROPOSED;
+import static accord.local.SafeCommandStore.TestStartedAt.STARTED_AFTER;
+import static accord.local.SafeCommandStore.TestStartedAt.STARTED_BEFORE;
 import static accord.local.Status.Phase;
 import static accord.local.Status.PreAccepted;
 import static accord.local.Status.PreCommitted;
@@ -137,16 +137,16 @@
             //    them as a dependency (but we have to make sure we consider dependency rules, so if there's no write and only reads)
             //    we might still have new transactions block our execution.
             Ranges ranges = safeStore.ranges().allAt(txnId);
-            rejectsFastPath = hasAcceptedStartedAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
+            rejectsFastPath = hasAcceptedOrCommittedStartedAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
             if (!rejectsFastPath)
-                rejectsFastPath = hasCommittedExecutesAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
+                rejectsFastPath = hasStableExecutesAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
 
             // TODO (expected, testing): introduce some good unit tests for verifying these two functions in a real repair scenario
             // committed txns with an earlier txnid and have our txnid as a dependency
-            earlierCommittedWitness = committedStartedBeforeAndWitnessed(safeStore, txnId, ranges, partialTxn.keys());
+            earlierCommittedWitness = stableStartedBeforeAndWitnessed(safeStore, txnId, ranges, partialTxn.keys());
 
             // accepted txns with an earlier txnid that don't have our txnid as a dependency
-            earlierAcceptedNoWitness = acceptedStartedBeforeWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
+            earlierAcceptedNoWitness = acceptedOrCommittedStartedBeforeWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
         }
 
         Status status = command.status();
@@ -212,7 +212,7 @@
     @Override
     public KeyHistory keyHistory()
     {
-        return KeyHistory.ALL;
+        return KeyHistory.COMMANDS;
     }
 
     @Override
@@ -326,14 +326,14 @@
         }
     }
 
-    private static Deps acceptedStartedBeforeWithoutWitnessing(SafeCommandStore safeStore, TxnId startedBefore, Ranges ranges, Seekables<?, ?> keys)
+    private static Deps acceptedOrCommittedStartedBeforeWithoutWitnessing(SafeCommandStore safeStore, TxnId startedBefore, Ranges ranges, Seekables<?, ?> keys)
     {
         try (Deps.Builder builder = Deps.builder())
         {
             // any transaction that started
-            safeStore.mapReduce(keys, ranges, KeyHistory.ALL, startedBefore.kind().witnessedBy(), STARTED_BEFORE, startedBefore, WITHOUT, startedBefore, Accepted, PreCommitted,
-                                          (startedBefore0, keyOrRange, txnId, executeAt, status, deps, prev) -> {
-                        if (executeAt.compareTo(startedBefore) > 0)
+            safeStore.mapReduceFull(keys, ranges, startedBefore, startedBefore.kind().witnessedBy(), STARTED_BEFORE, WITHOUT, IS_PROPOSED,
+                                    (startedBefore0, keyOrRange, txnId, executeAt, prev) -> {
+                        if (executeAt.compareTo(startedBefore0) > 0)
                             builder.add(keyOrRange, txnId);
                         return builder;
                     }, startedBefore, builder);
@@ -341,17 +341,17 @@
         }
     }
 
-    private static Deps committedStartedBeforeAndWitnessed(SafeCommandStore safeStore, TxnId startedBefore, Ranges ranges, Seekables<?, ?> keys)
+    private static Deps stableStartedBeforeAndWitnessed(SafeCommandStore safeStore, TxnId startedBefore, Ranges ranges, Seekables<?, ?> keys)
     {
         try (Deps.Builder builder = Deps.builder())
         {
-            safeStore.mapReduce(keys, ranges, KeyHistory.ALL, startedBefore.kind().witnessedBy(), STARTED_BEFORE, startedBefore, WITH, startedBefore, Committed, null,
-                                          (p1, keyOrRange, txnId, executeAt, status, deps, prev) -> builder.add(keyOrRange, txnId), null, (Deps.AbstractBuilder<Deps>)builder);
+            safeStore.mapReduceFull(keys, ranges, startedBefore, startedBefore.kind().witnessedBy(), STARTED_BEFORE, WITH, IS_STABLE,
+                                    (p1, keyOrRange, txnId, executeAt, prev) -> builder.add(keyOrRange, txnId), null, (Deps.AbstractBuilder<Deps>)builder);
             return builder.build();
         }
     }
 
-    private static boolean hasAcceptedStartedAfterWithoutWitnessing(SafeCommandStore safeStore, TxnId startedAfter, Ranges ranges, Seekables<?, ?> keys)
+    private static boolean hasAcceptedOrCommittedStartedAfterWithoutWitnessing(SafeCommandStore safeStore, TxnId startedAfter, Ranges ranges, Seekables<?, ?> keys)
     {
         /*
          * The idea here is to discover those transactions that were started after us and have been Accepted
@@ -362,11 +362,11 @@
          *
          * TODO (required): consider carefully how _adding_ ranges to a CommandStore affects this
          */
-        return safeStore.mapReduce(keys, ranges, KeyHistory.ALL, startedAfter.kind().witnessedBy(), STARTED_AFTER, startedAfter, WITHOUT, startedAfter, Accepted, PreCommitted,
-                                             (p1, keyOrRange, txnId, executeAt, status, deps, prev) -> true, null, false, i -> i);
+        return safeStore.mapReduceFull(keys, ranges, startedAfter, startedAfter.kind().witnessedBy(), STARTED_AFTER, WITHOUT, IS_PROPOSED,
+                                             (p1, keyOrRange, txnId, executeAt, prev) -> true, null, false);
     }
 
-    private static boolean hasCommittedExecutesAfterWithoutWitnessing(SafeCommandStore safeStore, TxnId startedAfter, Ranges ranges, Seekables<?, ?> keys)
+    private static boolean hasStableExecutesAfterWithoutWitnessing(SafeCommandStore safeStore, TxnId executesAfter, Ranges ranges, Seekables<?, ?> keys)
     {
         /*
          * The idea here is to discover those transactions that have been decided to execute after us
@@ -375,7 +375,7 @@
          * witnessed us we are safe to propose the pre-accept timestamp regardless, whereas if any transaction
          * has not witnessed us we can safely invalidate it.
          */
-        return safeStore.mapReduce(keys, ranges, KeyHistory.ALL, startedAfter.kind().witnessedBy(), EXECUTES_AFTER, startedAfter, WITHOUT, startedAfter, Committed, null,
-                                             (p1, keyOrRange, txnId, executeAt, status, deps, prev) -> true, null, false, i -> i);
+        return safeStore.mapReduceFull(keys, ranges, executesAfter, executesAfter.kind().witnessedBy(), ANY, WITHOUT, IS_STABLE,
+                                       (p1, keyOrRange, txnId, executeAt, prev) -> true, null, false);
     }
 }
diff --git a/accord-core/src/main/java/accord/messages/CheckStatus.java b/accord-core/src/main/java/accord/messages/CheckStatus.java
index f888f48..3cc235d 100644
--- a/accord-core/src/main/java/accord/messages/CheckStatus.java
+++ b/accord-core/src/main/java/accord/messages/CheckStatus.java
@@ -37,14 +37,12 @@
 import accord.local.SafeCommandStore;
 import accord.local.SaveStatus;
 import accord.local.Status;
-import accord.primitives.AbstractKeys;
 import accord.primitives.Ballot;
 import accord.primitives.EpochSupplier;
 import accord.primitives.PartialDeps;
 import accord.primitives.PartialRoute;
 import accord.primitives.PartialTxn;
 import accord.primitives.ProgressToken;
-import accord.primitives.Range;
 import accord.primitives.Ranges;
 import accord.primitives.Routables;
 import accord.primitives.Route;
@@ -159,13 +157,13 @@
         if (!command.has(Known.DefinitionOnly) && Route.isRoute(query) && safeStore.ranges().allAt(txnId.epoch()).contains(Route.castToRoute(query).homeKey()))
             Commands.informHome(safeStore, safeCommand, Route.castToRoute(query));
 
-        Ranges ranges = safeStore.ranges().allBetween(command.txnId().epoch(), command.executeAtIfKnownOrTxnId().epoch());
-        InvalidIfNot invalidIfNotAtLeast = invalidIfNotAtLeast(safeStore);
+        InvalidIfNot invalidIfNotAtLeast = invalidIfNot(safeStore, command);
         boolean isCoordinating = isCoordinating(node, command);
         Durability durability = command.durability();
         Route<?> route = command.route();
         if (Route.isFullRoute(route))
             durability = Durability.mergeAtLeast(durability, safeStore.commandStore().durableBefore().min(txnId, route));
+        Ranges ranges = safeStore.ranges().allBetween(command.txnId().epoch(), command.executeAtIfKnownOrTxnId().epoch());
 
         switch (includeInfo)
         {
@@ -204,9 +202,11 @@
         else node.reply(replyTo, replyContext, ok, null);
     }
 
-    private InvalidIfNot invalidIfNotAtLeast(SafeCommandStore safeStore)
+    private InvalidIfNot invalidIfNot(SafeCommandStore safeStore, Command command)
     {
-        return Infer.invalidIfNotAtLeast(safeStore, txnId, query);
+        if (command.known().isDecidedToExecute())
+            return NotKnownToBeInvalid;
+        return Infer.invalidIfNot(safeStore, txnId, query);
     }
 
     public interface CheckStatusReply extends Reply
@@ -329,33 +329,7 @@
             if (keysOrRanges.isEmpty())
                 return new FoundKnownMap();
 
-            switch (keysOrRanges.domain())
-            {
-                default: throw new AssertionError("Unhandled domain type: " + keysOrRanges.domain());
-                case Range:
-                {
-                    Ranges ranges = (Ranges)keysOrRanges;
-                    Builder builder = new Builder(ranges.get(0).endInclusive(), 2 * ranges.size());
-                    for (Range range : ranges)
-                    {
-                        builder.append(range.start(), known, FoundKnown::atLeast);
-                        builder.append(range.end(), null, FoundKnown::atLeast);
-                    }
-                    return builder.build();
-                }
-                case Key:
-                {
-                    AbstractKeys<RoutingKey> keys = (AbstractKeys<RoutingKey>) keysOrRanges;
-                    Builder builder = new Builder(keys.get(0).asRange().endInclusive(), 2 * keys.size());
-                    for (RoutingKey key : keys)
-                    {
-                        Range range = key.asRange();
-                        builder.append(range.start(), known, FoundKnown::atLeast);
-                        builder.append(range.end(), null, FoundKnown::atLeast);
-                    }
-                    return builder.build();
-                }
-            }
+            return create(keysOrRanges, known, Builder::new);
         }
 
         public static FoundKnownMap merge(FoundKnownMap a, FoundKnownMap b)
@@ -430,6 +404,11 @@
             return foldl((known, prev) -> known.isTruncated(), false, i -> i);
         }
 
+        public boolean hasInvalidated()
+        {
+            return foldl((known, prev) -> known.isInvalidated(), false, i -> i);
+        }
+
         public Known knownFor(Routables<?> routables)
         {
             return validForAll.atLeast(foldlWithDefault(routables, FoundKnownMap::reduceKnownFor, FoundKnown.Nothing, null, i -> false));
diff --git a/accord-core/src/main/java/accord/messages/Commit.java b/accord-core/src/main/java/accord/messages/Commit.java
index 05fd67f..c365c9a 100644
--- a/accord-core/src/main/java/accord/messages/Commit.java
+++ b/accord-core/src/main/java/accord/messages/Commit.java
@@ -21,6 +21,7 @@
 import javax.annotation.Nullable;
 
 import accord.local.Commands;
+import accord.local.KeyHistory;
 import accord.local.Node;
 import accord.local.Node.Id;
 import accord.local.PreLoadContext;
@@ -32,7 +33,6 @@
 import accord.primitives.Ballot;
 import accord.primitives.Deps;
 import accord.primitives.FullRoute;
-import accord.primitives.Keys;
 import accord.primitives.PartialDeps;
 import accord.primitives.PartialRoute;
 import accord.primitives.PartialTxn;
@@ -58,21 +58,22 @@
 import static accord.messages.Commit.WithTxn.HasNewlyOwnedTxnRanges;
 import static accord.messages.Commit.WithTxn.HasTxn;
 import static accord.messages.Commit.WithTxn.NoTxn;
-import static accord.utils.Invariants.checkArgument;
 
 public class Commit extends TxnRequest<CommitOrReadNack>
 {
     public static class SerializerSupport
     {
-        public static Commit create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Kind kind, Ballot ballot, Timestamp executeAt, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData readData)
+        public static Commit create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Kind kind, Ballot ballot, Timestamp executeAt, Seekables<?, ?> keys, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData readData)
         {
-            return new Commit(kind, txnId, scope, waitForEpoch, ballot, executeAt, partialTxn, partialDeps, fullRoute, readData);
+            return new Commit(kind, txnId, scope, waitForEpoch, ballot, executeAt, keys, partialTxn, partialDeps, fullRoute, readData);
         }
     }
 
     public final Kind kind;
     public final Ballot ballot;
     public final Timestamp executeAt;
+    public final Seekables<?, ?> keys;
+    // TODO (expected): share keys with partialTxn and partialDeps - in memory and on wire
     public final @Nullable PartialTxn partialTxn;
     public final @Nullable PartialDeps partialDeps;
     public final @Nullable FullRoute<?> route;
@@ -139,18 +140,20 @@
 
         this.kind = kind;
         this.executeAt = executeAt;
+        this.keys = txn.keys().slice(scope.covering());
         this.partialTxn = partialTxn;
         this.partialDeps = deps.slice(scope.covering());
         this.route = sendRoute;
         this.readData = toExecuteFactory == null ? null : toExecuteFactory.apply(partialTxn != null ? partialTxn : txn, scope, partialDeps);
     }
 
-    protected Commit(Kind kind, TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Ballot ballot, Timestamp executeAt, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData readData)
+    protected Commit(Kind kind, TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Ballot ballot, Timestamp executeAt, Seekables<?, ?> keys, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData readData)
     {
         super(txnId, scope, waitForEpoch);
         this.kind = kind;
         this.ballot = ballot;
         this.executeAt = executeAt;
+        this.keys = keys;
         this.partialTxn = partialTxn;
         this.partialDeps = partialDeps;
         this.route = fullRoute;
@@ -212,25 +215,6 @@
         }
     }
 
-    public static void stableMaximalAndBlockOnDeps(Node node, Topologies topologies, TxnId txnId, Txn txn, FullRoute<?> route, Timestamp executeAt, Deps deps, Callback<ReadReply> callback)
-    {
-        checkArgument(topologies.size() == 1);
-        Topology topology = topologies.get(0);
-        for (Node.Id to : topology.nodes())
-        {
-            // To simplify making sure the agent is notified once and is notified before the barrier coordination
-            // returns a result; we never notify the agent on the coordinator as part of WaitForDependenciesThenApply execution
-            // Also always send a maximal commit since we don't block on deps often and that saves having to have an Insufficient code path
-            // going back for the Apply in `ApplyThenWaitUntilApplied
-            boolean notifyAgent = !to.equals(node.id());
-            Commit commit = new Commit(
-            CommitWithTxn, to, topology, topologies, txnId,
-            txn, route, Ballot.ZERO, executeAt, deps,
-            (maybePartialTransaction, partialRoute, partialDeps) -> new ApplyThenWaitUntilApplied(txnId, partialRoute, partialDeps, maybePartialTransaction.keys().slice(partialDeps.covering), txn.execute(txnId, txnId, null), txn.result(txnId, txnId, null), notifyAgent));
-            node.send(to, commit, callback);
-        }
-    }
-
     public static void stableMaximal(Node node, Node.Id to, Txn txn, TxnId txnId, Timestamp executeAt, FullRoute<?> route, Deps deps)
     {
         // the replica may be missing the original commit, or the additional commit, so send everything
@@ -248,7 +232,13 @@
     @Override
     public Seekables<?, ?> keys()
     {
-        return partialTxn != null ? partialTxn.keys() : Keys.EMPTY;
+        return keys;
+    }
+
+    @Override
+    public KeyHistory keyHistory()
+    {
+        return KeyHistory.COMMANDS;
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/messages/GetDeps.java b/accord-core/src/main/java/accord/messages/GetDeps.java
index 16feb38..1e77c24 100644
--- a/accord-core/src/main/java/accord/messages/GetDeps.java
+++ b/accord-core/src/main/java/accord/messages/GetDeps.java
@@ -119,7 +119,7 @@
     @Override
     public KeyHistory keyHistory()
     {
-        return KeyHistory.DEPS;
+        return KeyHistory.COMMANDS;
     }
 
     public static class GetDepsOk implements Reply
diff --git a/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java b/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java
index 030d34b..3943d5d 100644
--- a/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java
+++ b/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java
@@ -99,7 +99,7 @@
     @Override
     public String toString()
     {
-        return "GetDeps{" +
+        return "GetEphemeralReadDeps{" +
                "txnId:" + txnId +
                ", keys:" + keys +
                '}';
@@ -120,7 +120,7 @@
     @Override
     public KeyHistory keyHistory()
     {
-        return KeyHistory.DEPS;
+        return KeyHistory.COMMANDS;
     }
 
     public static class GetEphemeralReadDepsOk implements Reply
diff --git a/accord-core/src/main/java/accord/messages/GetMaxConflict.java b/accord-core/src/main/java/accord/messages/GetMaxConflict.java
new file mode 100644
index 0000000..ec59b4d
--- /dev/null
+++ b/accord-core/src/main/java/accord/messages/GetMaxConflict.java
@@ -0,0 +1,143 @@
+/*
+ * 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 accord.messages;
+
+import javax.annotation.Nonnull;
+
+import accord.local.KeyHistory;
+import accord.local.Node;
+import accord.local.SafeCommandStore;
+import accord.primitives.FullRoute;
+import accord.primitives.PartialRoute;
+import accord.primitives.Ranges;
+import accord.primitives.Seekables;
+import accord.primitives.Timestamp;
+import accord.primitives.TxnId;
+import accord.topology.Topologies;
+import accord.utils.Invariants;
+
+public class GetMaxConflict extends TxnRequest.WithUnsynced<GetMaxConflict.GetMaxConflictOk>
+{
+    public static final class SerializationSupport
+    {
+        public static GetMaxConflict create(PartialRoute<?> scope, long waitForEpoch, long minEpoch, Seekables<?, ?> keys, long executionEpoch)
+        {
+            return new GetMaxConflict(scope, waitForEpoch, minEpoch, keys, executionEpoch);
+        }
+    }
+
+    public final Seekables<?, ?> keys;
+    public final long executionEpoch;
+
+    public GetMaxConflict(Node.Id to, Topologies topologies, FullRoute<?> route, Seekables<?, ?> keys, long executionEpoch)
+    {
+        super(to, topologies, executionEpoch, route);
+        this.keys = keys.slice(scope.covering());
+        this.executionEpoch = executionEpoch;
+    }
+
+    protected GetMaxConflict(PartialRoute<?> scope, long waitForEpoch, long minEpoch,  Seekables<?, ?> keys, long executionEpoch)
+    {
+        super(TxnId.NONE, scope, waitForEpoch, minEpoch, true);
+        this.keys = keys;
+        this.executionEpoch = executionEpoch;
+    }
+
+    @Override
+    public void process()
+    {
+        node.mapReduceConsumeLocal(this, minUnsyncedEpoch, executionEpoch, this);
+    }
+
+    @Override
+    public GetMaxConflictOk apply(SafeCommandStore safeStore)
+    {
+        Ranges ranges = safeStore.ranges().allBetween(minUnsyncedEpoch, executionEpoch);
+        Timestamp maxConflict = safeStore.commandStore().maxConflict(keys.slice(ranges));
+        return new GetMaxConflictOk(maxConflict, Math.max(safeStore.time().epoch(), node.epoch()));
+    }
+
+    @Override
+    public GetMaxConflictOk reduce(GetMaxConflictOk reply1, GetMaxConflictOk reply2)
+    {
+        return new GetMaxConflictOk(Timestamp.max(reply1.maxConflict, reply2.maxConflict), Math.max(reply1.latestEpoch, reply2.latestEpoch));
+    }
+
+    @Override
+    public void accept(GetMaxConflictOk result, Throwable failure)
+    {
+        node.reply(replyTo, replyContext, result, failure);
+    }
+
+    @Override
+    public MessageType type()
+    {
+        return MessageType.GET_EPHEMERAL_READ_DEPS_REQ;
+    }
+
+    @Override
+    public String toString()
+    {
+        return "GetMaxConflict{" +
+               ", keys:" + keys +
+               '}';
+    }
+
+    @Override
+    public TxnId primaryTxnId()
+    {
+        return null;
+    }
+
+    @Override
+    public Seekables<?, ?> keys()
+    {
+        return keys;
+    }
+
+    @Override
+    public KeyHistory keyHistory()
+    {
+        return KeyHistory.NONE;
+    }
+
+    public static class GetMaxConflictOk implements Reply
+    {
+        public final Timestamp maxConflict;
+        public final long latestEpoch;
+
+        public GetMaxConflictOk(@Nonnull Timestamp maxConflict, long latestEpoch)
+        {
+            this.maxConflict = Invariants.nonNull(maxConflict);
+            this.latestEpoch = latestEpoch;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "GetMaxConflictOk(" + maxConflict + ',' + latestEpoch + '}';
+        }
+
+        @Override
+        public MessageType type()
+        {
+            return MessageType.GET_EPHEMERAL_READ_DEPS_RSP;
+        }
+    }
+}
diff --git a/accord-core/src/main/java/accord/messages/PreAccept.java b/accord-core/src/main/java/accord/messages/PreAccept.java
index 0736b4b..ad964ff 100644
--- a/accord-core/src/main/java/accord/messages/PreAccept.java
+++ b/accord-core/src/main/java/accord/messages/PreAccept.java
@@ -27,14 +27,11 @@
 import org.slf4j.LoggerFactory;
 
 import accord.local.Node.Id;
-import accord.impl.DepsBuilder;
 import accord.local.*;
 import accord.messages.TxnRequest.WithUnsynced;
 import accord.topology.Shard;
 import accord.topology.Topologies;
 
-import static accord.local.SafeCommandStore.TestDep.ANY_DEPS;
-import static accord.local.SafeCommandStore.TestTimestamp.STARTED_BEFORE;
 import static accord.primitives.Txn.Kind.ExclusiveSyncPoint;
 
 public class PreAccept extends WithUnsynced<PreAccept.PreAcceptReply> implements EpochSupplier
@@ -86,7 +83,7 @@
     @Override
     public KeyHistory keyHistory()
     {
-        return KeyHistory.DEPS;
+        return KeyHistory.COMMANDS;
     }
 
     @Override
@@ -132,7 +129,7 @@
                 // all transactions with lower txnId as expired.
                 Ranges ranges = safeStore.ranges().allBetween(minUnsyncedEpoch, txnId);
                 return new PreAcceptOk(txnId, command.executeAt(),
-                                       calculatePartialDeps(safeStore, txnId, partialTxn.keys(), EpochSupplier.constant(minUnsyncedEpoch), txnId, ranges));
+                                       calculatePartialDeps(safeStore, txnId, command.partialTxn().keys(), EpochSupplier.constant(minUnsyncedEpoch), txnId, ranges));
 
             case Truncated:
             case RejectedBallot:
@@ -248,21 +245,22 @@
     static PartialDeps calculatePartialDeps(SafeCommandStore safeStore, TxnId txnId, Seekables<?, ?> keys, EpochSupplier minEpoch, Timestamp executeAt, Ranges ranges)
     {
         // TODO (expected): do not build covering ranges; no longer especially valuable given use of FullRoute
-        DepsBuilder builder = new DepsBuilder(txnId);
-        // could use MAY_EXECUTE_BEFORE to prune those we know execute later.
         // NOTE: ExclusiveSyncPoint *relies* on STARTED_BEFORE to ensure it reports a dependency on *every* earlier TxnId that may execute after it.
         //       This is necessary for reporting to a bootstrapping replica which TxnId it must not prune from dependencies
         //       i.e. the source replica reports to the target replica those TxnId that STARTED_BEFORE and EXECUTES_AFTER.
-        safeStore.mapReduce(keys, ranges, KeyHistory.DEPS, txnId.kind().witnesses(), STARTED_BEFORE, executeAt, ANY_DEPS, null, null, null,
-                            (p1, keyOrRange, testTxnId, testExecuteAt, status, deps, in) -> {
-                                builder.add(testTxnId, keyOrRange, status, testExecuteAt, deps.get());
-                                return in;
-                            }, null, builder);
-        
-        try (Deps.AbstractBuilder<PartialDeps> redundantBuilder = new PartialDeps.Builder(ranges))
+        try (Deps.AbstractBuilder<PartialDeps> builder = new PartialDeps.Builder(ranges);
+             Deps.AbstractBuilder<PartialDeps> redundantBuilder = new PartialDeps.Builder(ranges))
         {
+            safeStore.mapReduceActive(keys, ranges, executeAt, txnId.kind().witnesses(),
+                                      (p1, keyOrRange, testTxnId, testExecuteAt, in) -> {
+                                          if (p1 == null || !testTxnId.equals(p1))
+                                              in.add(keyOrRange, testTxnId);
+                                          return in;
+                                      }, executeAt.equals(txnId) ? null : txnId, builder);
+
+            // TODO (required): make sure any sync point is in the past
             PartialDeps redundant = safeStore.commandStore().redundantBefore().collectDeps(keys, redundantBuilder, minEpoch, executeAt).build();
-            return builder.buildPartialDeps(safeStore, ranges).with(redundant);
+            return builder.build().with(redundant);
         }
     }
 
diff --git a/accord-core/src/main/java/accord/messages/Propagate.java b/accord-core/src/main/java/accord/messages/Propagate.java
index 99d4333..4a20fa1 100644
--- a/accord-core/src/main/java/accord/messages/Propagate.java
+++ b/accord-core/src/main/java/accord/messages/Propagate.java
@@ -22,6 +22,7 @@
 import accord.coordinate.Infer;
 import accord.local.Command;
 import accord.local.Commands;
+import accord.local.KeyHistory;
 import accord.local.Node;
 import accord.local.SafeCommand;
 import accord.local.SafeCommandStore;
@@ -220,6 +221,7 @@
     @Override
     public Seekables<?, ?> keys()
     {
+        // TODO (required): these may be insufficient; must make update of CommandsForKey async
         if (partialTxn != null)
             return partialTxn.keys();
 
@@ -230,6 +232,12 @@
     }
 
     @Override
+    public KeyHistory keyHistory()
+    {
+        return KeyHistory.COMMANDS;
+    }
+
+    @Override
     public void preProcess(Node on, Node.Id from, ReplyContext replyContext)
     {
         throw new UnsupportedOperationException();
@@ -267,6 +275,7 @@
         if (isShardTruncated)
         {
             // TODO (required): do not markShardStale for reads; in general optimise handling of case where we cannot recover a known no-op transaction
+            // TODO (required): permit staleness to be gated by some configuration state
             achieved = applyOrUpgradeTruncated(safeStore, safeCommand, command, executeAtIfKnown);
             if (achieved == null)
                 return null;
@@ -408,7 +417,7 @@
         if (ranges.isEmpty())
         {
             // TODO (expected): we might prefer to adopt Redundant status, and permit ourselves to later accept the result of the execution and/or definition
-            Commands.setTruncatedApply(safeStore, safeCommand, route);
+            Commands.setTruncatedApply(safeStore, safeCommand, executeAtIfKnown, route);
             return null;
         }
 
@@ -417,7 +426,7 @@
         {
             // we only coordinate this transaction, so being unable to retrieve its state does not imply any staleness
             // TODO (now): double check this doesn't stop us coordinating the transaction (it shouldn't, as doesn't imply durability)
-            Commands.setTruncatedApply(safeStore, safeCommand, route);
+            Commands.setTruncatedApply(safeStore, safeCommand, executeAtIfKnown, route);
             return null;
         }
 
@@ -443,7 +452,7 @@
             return required;
 
         // TODO (expected): we might prefer to adopt Redundant status, and permit ourselves to later accept the result of the execution and/or definition
-        Commands.setTruncatedApply(safeStore, safeCommand, route);
+        Commands.setTruncatedApply(safeStore, safeCommand, executeAtIfKnown, route);
         return null;
     }
 
diff --git a/accord-core/src/main/java/accord/messages/ReadData.java b/accord-core/src/main/java/accord/messages/ReadData.java
index d1e2529..18600a6 100644
--- a/accord-core/src/main/java/accord/messages/ReadData.java
+++ b/accord-core/src/main/java/accord/messages/ReadData.java
@@ -267,9 +267,7 @@
 
     static Ranges unavailable(SafeCommandStore safeStore, Command command)
     {
-        Timestamp executeAt = command.executeAt();
-        if (command.txnId().kind().awaitsOnlyDeps())
-            executeAt = Timestamp.nonNullOrMax(executeAt, command.asCommitted().waitingOn.executeAtLeast());
+        Timestamp executeAt = command.executesAtLeast();
         // TODO (required): for awaitsOnlyDeps commands, if we cannot infer an actual executeAtLeast we should confirm no situation where txnId is not an adequately conservative value for unavailable/unsafeToRead
         return safeStore.ranges().unsafeToReadAt(executeAt);
     }
diff --git a/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java b/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java
index fb11002..75c2126 100644
--- a/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java
+++ b/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java
@@ -26,7 +26,6 @@
 import accord.local.SafeCommand;
 import accord.local.SafeCommandStore;
 import accord.local.SaveStatus;
-import accord.local.Status;
 import accord.primitives.Deps;
 import accord.primitives.FullRoute;
 import accord.primitives.PartialDeps;
@@ -105,23 +104,38 @@
     @Override
     void read(SafeCommandStore safeStore, Command command)
     {
-        Command.WaitingOn waitingOn = command.asCommitted().waitingOn;
-        Timestamp executeAtLeast = waitingOn.executeAtLeast();
-        if (executeAtLeast != null && executeAtLeast.epoch() > executeAtEpoch)
+        long retryInLaterEpoch = retryInLaterEpoch(executeAtEpoch, safeStore, command);
+        if (retryInLaterEpoch > 0)
         {
-            long executeAtLeastEpoch = executeAtLeast.epoch();
-            Ranges removed = safeStore.ranges().removed(executeAtEpoch, executeAtLeastEpoch);
-            if (removed.intersects(command.partialTxn().keys()))
-            {
-                Ranges unavailable = unavailable(safeStore, command).with(removed).intersecting(partialTxn.keys());
-                cancel(safeStore);
-                node.reply(replyTo, replyContext, new ReadOkWithFutureEpoch(unavailable, null, executeAtLeastEpoch), null);
-                return;
-            }
+            // TODO (expected): wait for all stores' results and report only the ranges that execute later to be retried
+            cancel(safeStore);
+            node.reply(replyTo, replyContext, new ReadOkWithFutureEpoch(null, null, retryInLaterEpoch), null);
         }
         super.read(safeStore, command);
     }
 
+    static long retryInLaterEpoch(long executeAtEpoch, SafeCommandStore safeStore, Command command)
+    {
+        TxnId txnId = command.txnId();
+        if (!txnId.kind().awaitsOnlyDeps())
+            return 0;
+
+        // TODO (required): should we disambiguate between cases where a truncated command has been executed locally
+        //   versus made redundant by e.g. bootstrap, staleness etc? that *should* be handled by checking
+        //   unavailable ranges
+        Timestamp executesAtLeast = command.executesAtLeast();
+        if (executesAtLeast != null && executesAtLeast.epoch() > executeAtEpoch)
+        {
+            long executeAtLeastEpoch = executesAtLeast.epoch();
+            Ranges removed = safeStore.ranges().removed(executeAtEpoch, executeAtLeastEpoch);
+            if (removed.intersects(command.route()))
+                return executeAtLeastEpoch;
+        }
+
+        return 0;
+    }
+
+
     @Override
     protected ReadOk constructReadOk(Ranges unavailable, Data data)
     {
diff --git a/accord-core/src/main/java/accord/messages/SetGloballyDurable.java b/accord-core/src/main/java/accord/messages/SetGloballyDurable.java
index 4f76bcd..3fa3932 100644
--- a/accord-core/src/main/java/accord/messages/SetGloballyDurable.java
+++ b/accord-core/src/main/java/accord/messages/SetGloballyDurable.java
@@ -21,17 +21,22 @@
 import accord.local.PreLoadContext;
 import accord.local.DurableBefore;
 import accord.local.SafeCommandStore;
+import accord.primitives.TxnId;
 
 import static accord.messages.SimpleReply.Ok;
 
+// TODO (required): if we have epochs 1 and 2, and a syncpoint that executes on epoch 2, where the transactions have not
+//    finished executing on epoch 1, it may not be safe to mark durable on replicas on epoch 1. This is a very unlikely
+//    race condition, but must consider our behaviour - it may be simpler to wait for a sync point to execute on all epochs
+//    that haven't been closed off.
 public class SetGloballyDurable extends AbstractEpochRequest<SimpleReply>
         implements Request, PreLoadContext
 {
     public final DurableBefore durableBefore;
 
-    public SetGloballyDurable(DurableBefore durableBefore)
+    public SetGloballyDurable(TxnId txnId, DurableBefore durableBefore)
     {
-        super(null);
+        super(txnId);
         this.durableBefore = durableBefore;
     }
 
diff --git a/accord-core/src/main/java/accord/messages/TxnRequest.java b/accord-core/src/main/java/accord/messages/TxnRequest.java
index 4cd3942..f6ea2c4 100644
--- a/accord-core/src/main/java/accord/messages/TxnRequest.java
+++ b/accord-core/src/main/java/accord/messages/TxnRequest.java
@@ -51,20 +51,35 @@
             this(to, topologies, txnId, route, latestRelevantEpochIndex(to, topologies, route));
         }
 
-        private WithUnsynced(Id to, Topologies topologies, TxnId txnId, FullRoute<?> route, int startIndex)
+        protected WithUnsynced(Id to, Topologies topologies, TxnId txnId, FullRoute<?> route, int startIndex)
+        {
+            this(to, topologies, txnId, txnId.epoch(), route, startIndex);
+        }
+
+        public WithUnsynced(Id to, Topologies topologies, long epoch, FullRoute<?> route)
+        {
+            this(to, topologies, TxnId.NONE, epoch, route, latestRelevantEpochIndex(to, topologies, route), true);
+        }
+
+        protected WithUnsynced(Id to, Topologies topologies, TxnId txnId, long epoch, FullRoute<?> route, int startIndex)
+        {
+            this(to, topologies, txnId, epoch, route, startIndex, false);
+        }
+
+        protected WithUnsynced(Id to, Topologies topologies, TxnId txnId, long epoch, FullRoute<?> route, int startIndex, boolean doNotComputeProgressKey)
         {
             super(to, topologies, route, txnId, startIndex);
             this.minUnsyncedEpoch = topologies.oldestEpoch();
-            this.doNotComputeProgressKey = doNotComputeProgressKey(topologies, startIndex, txnId, waitForEpoch());
+            this.doNotComputeProgressKey = doNotComputeProgressKey || doNotComputeProgressKey(topologies, startIndex, epoch, waitForEpoch());
 
-            Ranges ranges = topologies.forEpoch(txnId.epoch()).rangesForNode(to);
+            Ranges ranges = topologies.forEpoch(epoch).rangesForNode(to);
             if (doNotComputeProgressKey)
             {
-                Invariants.checkState(!route.intersects(ranges)); // confirm dest is not a replica on txnId.epoch
+                Invariants.checkState(txnId.equals(TxnId.NONE) || !route.intersects(ranges)); // confirm dest is not a replica on txnId.epoch
             }
             else if (Invariants.isParanoid())
             {
-                long progressEpoch = Math.min(waitForEpoch(), txnId.epoch());
+                long progressEpoch = Math.min(waitForEpoch(), epoch);
                 Ranges computesRangesOn = topologies.forEpoch(progressEpoch).rangesForNode(to);
                 if (computesRangesOn == null)
                     Invariants.checkState(!route.intersects(ranges));
@@ -156,8 +171,11 @@
         return progressKey(node, waitForEpoch, txnId, scope);
     }
 
-    public static RoutingKey progressKey(Node node, long waitForEpoch, TxnId txnId, PartialRoute<?> scope)
+    public static RoutingKey progressKey(Node node, long waitForEpoch, TxnId txnId, Route<?> scope)
     {
+        if (txnId == null)
+            return null;
+
         // if waitForEpoch < txnId.epoch, then this replica's ownership is unchanged
         long progressEpoch = min(waitForEpoch, txnId.epoch());
         return node.trySelectProgressKey(progressEpoch, scope, scope.homeKey());
@@ -265,7 +283,7 @@
         return scope;
     }
 
-    private static boolean doNotComputeProgressKey(Topologies topologies, int startIndex, TxnId txnId, long waitForEpoch)
+    private static boolean doNotComputeProgressKey(Topologies topologies, int startIndex, long epoch, long waitForEpoch)
     {
         // to understand this calculation we must bear in mind the following:
         //  - startIndex is the "latest relevant" which means we skip over recent epochs where we are not owners at all,
@@ -280,7 +298,7 @@
         // (as it might be done so with stale ring information)
 
         // TODO (low priority, clarity): this would be better defined as "hasProgressKey"
-        return waitForEpoch < txnId.epoch() && startIndex > 0
-                && topologies.get(startIndex).epoch() < txnId.epoch();
+        return waitForEpoch < epoch && startIndex > 0
+                && topologies.get(startIndex).epoch() < epoch;
     }
 }
diff --git a/accord-core/src/main/java/accord/messages/WaitUntilApplied.java b/accord-core/src/main/java/accord/messages/WaitUntilApplied.java
index 2cb398f..74cd578 100644
--- a/accord-core/src/main/java/accord/messages/WaitUntilApplied.java
+++ b/accord-core/src/main/java/accord/messages/WaitUntilApplied.java
@@ -27,11 +27,11 @@
 import accord.local.SafeCommandStore;
 import accord.primitives.Participants;
 import accord.primitives.Ranges;
-import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 import accord.topology.Topologies;
 
 import static accord.local.SaveStatus.Applied;
+import static accord.local.SaveStatus.TruncatedApply;
 
 /**
  * Wait until the transaction has been applied locally
@@ -48,8 +48,8 @@
         }
     }
 
-    private static final ExecuteOn EXECUTE_ON = new ExecuteOn(Applied, Applied);
-    private long futureEpoch;
+    private static final ExecuteOn EXECUTE_ON = new ExecuteOn(Applied, TruncatedApply);
+    private long retryInLaterEpoch;
 
     public WaitUntilApplied(Node.Id to, Topologies topologies, TxnId txnId, Participants<?> readScope, long executeAtEpoch)
     {
@@ -76,17 +76,16 @@
     @Override
     void read(SafeCommandStore safeStore, Command command)
     {
-        Command.WaitingOn waitingOn = command.asCommitted().waitingOn;
-        Timestamp executeAtLeast = waitingOn.executeAtLeast();
-        if (executeAtLeast != null)
-            this.futureEpoch = Math.max(futureEpoch, executeAtLeast.epoch());
+        long retryInLaterEpoch = ReadEphemeralTxnData.retryInLaterEpoch(executeAtEpoch, safeStore, command);
+        if (retryInLaterEpoch > this.retryInLaterEpoch)
+            this.retryInLaterEpoch = retryInLaterEpoch;
         onOneSuccess(safeStore.commandStore(), unavailable(safeStore, command));
     }
 
     @Override
     protected ReadOk constructReadOk(Ranges unavailable, Data data)
     {
-        return new ReadOkWithFutureEpoch(unavailable, data, futureEpoch);
+        return new ReadOkWithFutureEpoch(unavailable, data, retryInLaterEpoch);
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/primitives/AbstractKeys.java b/accord-core/src/main/java/accord/primitives/AbstractKeys.java
index 2c699f1..c120195 100644
--- a/accord-core/src/main/java/accord/primitives/AbstractKeys.java
+++ b/accord-core/src/main/java/accord/primitives/AbstractKeys.java
@@ -31,6 +31,7 @@
 import accord.utils.ArrayBuffers.ObjectBuffers;
 import accord.utils.IndexedFold;
 import accord.utils.IndexedFoldToLong;
+import accord.utils.IndexedTriFold;
 import accord.utils.Invariants;
 import accord.utils.SortedArrays;
 import net.nicoulaj.compilecommand.annotations.Inline;
@@ -234,6 +235,18 @@
     }
 
     @Inline
+    public final long foldl(AbstractKeys<K> intersect, IndexedFoldToLong<K> fold, long param, long accumulator, long terminalValue)
+    {
+        return Routables.foldl(this, intersect, fold, param, accumulator, terminalValue);
+    }
+
+    @Inline
+    public final <P1, P2, V> V foldl(AbstractKeys<K> intersect, IndexedTriFold<P1, P2, K, V> fold, P1 p1, P2 p2, V accumulator)
+    {
+        return Routables.foldl(this, intersect, fold, p1, p2, accumulator, i -> false);
+    }
+
+    @Inline
     public final void forEach(Ranges rs, Consumer<? super K> forEach)
     {
         Routables.foldl(this, rs, (k, consumer, i) -> { consumer.accept(k); return consumer; }, forEach);
diff --git a/accord-core/src/main/java/accord/primitives/KeyDeps.java b/accord-core/src/main/java/accord/primitives/KeyDeps.java
index a53e80d..47cd345 100644
--- a/accord-core/src/main/java/accord/primitives/KeyDeps.java
+++ b/accord-core/src/main/java/accord/primitives/KeyDeps.java
@@ -515,11 +515,11 @@
         return new SortedArrayList<>(txnIds);
     }
 
-    public List<TxnId> txnIds(Key key)
+    public SortedRelationList<TxnId> txnIds(Key key)
     {
         int keyIndex = keys.indexOf(key);
         if (keyIndex < 0)
-            return Collections.emptyList();
+            return SortedRelationList.EMPTY;
 
         return txnIdsForKeyIndex(keyIndex);
     }
diff --git a/accord-core/src/main/java/accord/primitives/PartialDeps.java b/accord-core/src/main/java/accord/primitives/PartialDeps.java
index e13ce62..9367328 100644
--- a/accord-core/src/main/java/accord/primitives/PartialDeps.java
+++ b/accord-core/src/main/java/accord/primitives/PartialDeps.java
@@ -45,6 +45,8 @@
 
     // TODO (expected): we no longer need this if everyone has a FullRoute
     //      could also retain a simple bitset over the original FullRoute
+    // TODO (required) remove this and related concepts, as can cause problems with topology changes for a single store
+    //    where the store has some ranges that we participate in, and some we do not; we will not correctly construct covering in some cases
     public final Ranges covering;
 
     public PartialDeps(Ranges covering, KeyDeps keyDeps, RangeDeps rangeDeps)
diff --git a/accord-core/src/main/java/accord/primitives/RangeDeps.java b/accord-core/src/main/java/accord/primitives/RangeDeps.java
index 56ac2dd..00e047c 100644
--- a/accord-core/src/main/java/accord/primitives/RangeDeps.java
+++ b/accord-core/src/main/java/accord/primitives/RangeDeps.java
@@ -45,7 +45,10 @@
  * <p>The relationship between Range and TxnId is maintained via {@code int[]} utilising {@link RelationMultiMap}
  * functionality.
  *
- * TODO (expected): de-overlap ranges per txnId if possible cheaply, or else reduce use of partial ranges where possible
+ * TODO (required): de-overlap ranges per txnId if possible cheaply, or else reduce use of partial ranges where possible
+ * TODO (required): permit building out-of-order
+ * TODO (required): currently permitting duplicates
+ * TODO (required): randomised testing of all iteration methods (just found a double increment bug)
  */
 public class RangeDeps implements Iterable<Map.Entry<Range, TxnId>>
 {
@@ -517,7 +520,7 @@
         int start = txnIdx == 0 ? txnIds.length : txnIdsToRanges[txnIdx - 1];
         int end = txnIdsToRanges[txnIdx];
         for (int i = start; i < end ; ++i)
-            accumulate = fold.apply(p1, ranges[txnIdsToRanges[i++]], accumulate);
+            accumulate = fold.apply(p1, ranges[txnIdsToRanges[i]], accumulate);
         return accumulate;
     }
 
diff --git a/accord-core/src/main/java/accord/primitives/Ranges.java b/accord-core/src/main/java/accord/primitives/Ranges.java
index 5cc07c8..ac293dc 100644
--- a/accord-core/src/main/java/accord/primitives/Ranges.java
+++ b/accord-core/src/main/java/accord/primitives/Ranges.java
@@ -19,6 +19,9 @@
 package accord.primitives;
 
 import accord.api.RoutingKey;
+import accord.utils.IndexedFoldToLong;
+import accord.utils.IndexedTriFold;
+import net.nicoulaj.compilecommand.annotations.Inline;
 
 import javax.annotation.Nonnull;
 import java.util.ArrayList;
@@ -196,4 +199,16 @@
         return ImmutableMap.of(Boolean.TRUE, Ranges.ofSortedAndDeoverlapped(trues.toArray(new Range[0])),
                                Boolean.FALSE, Ranges.ofSortedAndDeoverlapped(falses.toArray(new Range[0])));
     }
+
+    @Inline
+    public final long foldl(Ranges intersect, IndexedFoldToLong<Range> fold, long param, long accumulator, long terminalValue)
+    {
+        return Routables.foldl(this, intersect, fold, param, accumulator, terminalValue);
+    }
+
+    @Inline
+    public final <P1, P2, V> V foldl(Ranges intersect, IndexedTriFold<P1, P2, Range, V> fold, P1 p1, P2 p2, V accumulator)
+    {
+        return Routables.foldl(this, intersect, fold, p1, p2, accumulator, i -> false);
+    }
 }
diff --git a/accord-core/src/main/java/accord/primitives/SyncPoint.java b/accord-core/src/main/java/accord/primitives/SyncPoint.java
index fedc64c..d22f375 100644
--- a/accord-core/src/main/java/accord/primitives/SyncPoint.java
+++ b/accord-core/src/main/java/accord/primitives/SyncPoint.java
@@ -33,9 +33,9 @@
 {
     public static class SerializationSupport
     {
-        public static SyncPoint construct(TxnId syncId, Deps waitFor, Seekables<?,?> keysOrRanges, RoutingKey homeKey, boolean finishedAsync)
+        public static SyncPoint construct(TxnId syncId, Deps waitFor, Seekables<?,?> keysOrRanges, RoutingKey homeKey)
         {
-            return new SyncPoint(syncId, waitFor, keysOrRanges, homeKey, finishedAsync);
+            return new SyncPoint(syncId, waitFor, keysOrRanges, homeKey);
         }
     }
 
@@ -43,25 +43,22 @@
     public final Deps waitFor;
     public final S keysOrRanges;
     public final RoutingKey homeKey;
-    public final boolean finishedAsync;
 
-    public SyncPoint(TxnId syncId, Deps waitFor, S keysOrRanges, FullRoute route, boolean finishedAsync)
+    public SyncPoint(TxnId syncId, Deps waitFor, S keysOrRanges, FullRoute<?> route)
     {
         Invariants.checkArgument(keysOrRanges.toRoute(route.homeKey()).equals(route), "Expected homeKey %s from route %s to be in ranges %s", route.homeKey(), route, keysOrRanges);
         this.syncId = syncId;
         this.waitFor = waitFor;
         this.keysOrRanges = keysOrRanges;
         this.homeKey = route.homeKey();
-        this.finishedAsync = finishedAsync;
     }
 
-    private SyncPoint(TxnId syncId, Deps waitFor, S keysOrRanges, RoutingKey homeKey, boolean finishedAsync)
+    private SyncPoint(TxnId syncId, Deps waitFor, S keysOrRanges, RoutingKey homeKey)
     {
         this.syncId = syncId;
         this.waitFor = waitFor;
         this.keysOrRanges = keysOrRanges;
         this.homeKey = homeKey;
-        this.finishedAsync = finishedAsync;
     }
 
     public FullRoute route()
diff --git a/accord-core/src/main/java/accord/primitives/Timestamp.java b/accord-core/src/main/java/accord/primitives/Timestamp.java
index 05dfe86..41c8326 100644
--- a/accord-core/src/main/java/accord/primitives/Timestamp.java
+++ b/accord-core/src/main/java/accord/primitives/Timestamp.java
@@ -37,11 +37,12 @@
      * which we may also want to retain when merging in other contexts (such as in Deps).
      */
     private static final int MERGE_FLAGS = 0x8000;
+    // TODO (required): is this the correct set of identity bits?
     private static final long IDENTITY_LSB = 0xFFFFFFFFFFFF001EL;
-    private static final int IDENTITY_FLAGS = 0x001E;
+    public static final int IDENTITY_FLAGS = 0x001E;
     public static final long MAX_EPOCH = (1L << 48) - 1;
     private static final long HLC_INCR = 1L << 16;
-    private static final long MAX_FLAGS = HLC_INCR - 1;
+    static final long MAX_FLAGS = HLC_INCR - 1;
 
     public static Timestamp fromBits(long msb, long lsb, Id node)
     {
diff --git a/accord-core/src/main/java/accord/primitives/Txn.java b/accord-core/src/main/java/accord/primitives/Txn.java
index 3675f9f..c88f471 100644
--- a/accord-core/src/main/java/accord/primitives/Txn.java
+++ b/accord-core/src/main/java/accord/primitives/Txn.java
@@ -47,9 +47,13 @@
 
 public interface Txn
 {
+    /**
+     * NOTE: we keep Read/Write adjacent to make it easier to check for non-standard flags in serialization
+     */
     enum Kind
     {
         Read,
+        Write,
 
         /**
          * A non-durable read that cannot be recovered and provides only per-key linearizability guarantees.
@@ -58,16 +62,6 @@
          */
         EphemeralRead,
 
-        Write,
-
-        /**
-         * A pseudo-transaction used only to find an executeAt that is higher than any transaction that could have
-         * reached consensus before it started.
-         *
-         * Invisible to other transactions.
-         */
-        NoOp,
-
         /**
          * A pseudo-transaction whose deps represent the complete set of transactions that may execute before it,
          * without interfering with their execution.
@@ -189,7 +183,18 @@
 
         public boolean isGloballyVisible()
         {
-            return this != EphemeralRead && this != LocalOnly;
+            switch (this)
+            {
+                default: throw new AssertionError("Unhandled Kind: " + this);
+                case EphemeralRead:
+                case LocalOnly:
+                    return false;
+                case Write:
+                case Read:
+                case ExclusiveSyncPoint:
+                case SyncPoint:
+                    return true;
+            }
         }
 
         public boolean isSyncPoint()
@@ -217,7 +222,6 @@
                 default: throw new AssertionError();
                 case EphemeralRead:
                 case Read:
-                case NoOp:
                     return Ws;
                 case Write:
                     return RorWs;
@@ -240,14 +244,13 @@
                     return AnyGloballyVisible;
                 case SyncPoint:
                 case ExclusiveSyncPoint:
-                case NoOp:
                     return SyncPoints;
             }
         }
 
         public char shortName()
         {
-            return toString().charAt(0);
+            return shortName;
         }
     }
 
diff --git a/accord-core/src/main/java/accord/primitives/TxnId.java b/accord-core/src/main/java/accord/primitives/TxnId.java
index 145ece4..5647ae9 100644
--- a/accord-core/src/main/java/accord/primitives/TxnId.java
+++ b/accord-core/src/main/java/accord/primitives/TxnId.java
@@ -166,7 +166,7 @@
         return new TxnId(epochMsb(epoch), 0, Id.NONE);
     }
 
-    private static final Pattern PARSE = Pattern.compile("\\[(?<epoch>[0-9]+),(?<hlc>[0-9]+),(?<flags>[0-9]+),(?<node>[0-9]+)]");
+    private static final Pattern PARSE = Pattern.compile("\\[(?<epoch>[0-9]+),(?<hlc>[0-9]+),(?<flags>[0-9]+)\\([KREWNSXL]\\),(?<node>[0-9]+)]");
     public static TxnId parse(String txnIdString)
     {
         Matcher m = PARSE.matcher(txnIdString);
diff --git a/accord-core/src/main/java/accord/utils/ArrayBuffers.java b/accord-core/src/main/java/accord/utils/ArrayBuffers.java
index a554559..ba633cc 100644
--- a/accord-core/src/main/java/accord/utils/ArrayBuffers.java
+++ b/accord-core/src/main/java/accord/utils/ArrayBuffers.java
@@ -372,7 +372,7 @@
 
         boolean discardInternal(B buffer, int bufferSize, int usedSize, boolean force)
         {
-            if (bufferSize > maxSize)
+            if (bufferSize == 0 || bufferSize > maxSize)
                 return true;
 
             if (bufferSize == usedSize && !force)
diff --git a/accord-core/src/main/java/accord/utils/Invariants.java b/accord-core/src/main/java/accord/utils/Invariants.java
index 43603fe..85464bf 100644
--- a/accord-core/src/main/java/accord/utils/Invariants.java
+++ b/accord-core/src/main/java/accord/utils/Invariants.java
@@ -27,6 +27,7 @@
 
 public class Invariants
 {
+    // TODO (now): configure by system parameter and turn off by default
     private static final boolean PARANOID = true;
     private static final boolean DEBUG = true;
 
@@ -41,7 +42,7 @@
 
     public static IllegalStateException illegalState(String msg)
     {
-        throw new IllegalStateException(msg);
+         throw new IllegalStateException(msg);
     }
 
     public static IllegalStateException illegalState()
@@ -54,7 +55,6 @@
         throw new IllegalArgumentException(msg);
     }
 
-
     private static void illegalArgument()
     {
         illegalArgument(null);
@@ -315,7 +315,7 @@
         }
         catch (ClassCastException e)
         {
-            throw new IllegalArgumentException(format("Unable to cast %s to %s", o, klass.getName()));
+            throw illegalArgument(format("Unable to cast %s to %s", o, klass.getName()));
         }
     }
 
diff --git a/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java b/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java
index 7a935fe..1f2c316 100644
--- a/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java
+++ b/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java
@@ -225,7 +225,8 @@
             }
 
             start = second.start();
-            if (first.hasCurrent() && first.start().compareTo(start) < 0) builder.append(first.start(), start, first.value());
+            if (first.hasCurrent() && first.start().compareTo(start) < 0 && first.value() != null)
+                builder.append(first.start(), start, builder.slice(first.start(), start, first.value()));
             Invariants.checkState(start.compareTo(second.start()) <= 0);
         }
 
@@ -304,6 +305,7 @@
         }
 
         // loop over any range covered by both
+        // TODO (expected): optimise merging of very different sized maps (i.e. for inserts)
         while (left.hasCurrent() && right.hasCurrent())
         {
             int cmp = left.end().compareTo(right.end());
@@ -460,9 +462,23 @@
             if (sameAsTailKey || sameAsTailValue)
             {
                 if (sameAsTailValue)
+                {
                     values.set(tailIdx, value == null ? null : tailValue);
+                }
+                else if (tailValue != null)
+                {
+                    values.set(tailIdx, reduce.apply(tailValue, value));
+                }
+                else if (tailIdx >= 1 && value.equals(values.get(tailIdx - 1)))
+                {
+                    // just remove the null value and start
+                    values.remove(tailIdx);
+                    starts.remove(tailIdx);
+                }
                 else
-                    values.set(tailIdx, tailValue == null ? value : reduce.apply(tailValue, value));
+                {
+                    values.set(tailIdx, value);
+                }
             }
             else
             {
diff --git a/accord-core/src/main/java/accord/utils/ReducingRangeMap.java b/accord-core/src/main/java/accord/utils/ReducingRangeMap.java
index 904586b..13beb8c 100644
--- a/accord-core/src/main/java/accord/utils/ReducingRangeMap.java
+++ b/accord-core/src/main/java/accord/utils/ReducingRangeMap.java
@@ -20,6 +20,7 @@
 import accord.api.RoutingKey;
 import accord.primitives.*;
 
+import java.util.Arrays;
 import java.util.function.BiFunction;
 import java.util.function.Predicate;
 
@@ -297,6 +298,22 @@
         return accumulator;
     }
 
+    private int find(RoutableKey key)
+    {
+        int idx = Arrays.binarySearch(starts, key);
+        if (idx < 0) idx = -2 - idx;
+        else if (inclusiveEnds) --idx;
+        return idx;
+    }
+
+    public V get(RoutableKey key)
+    {
+        int idx = find(key);
+        if (idx < 0 || idx >= values.length)
+            return null;
+        return values[idx];
+    }
+
     public static <V> ReducingRangeMap<V> create(Ranges ranges, V value)
     {
         if (value == null)
@@ -305,12 +322,86 @@
         if (ranges.isEmpty())
             return new ReducingRangeMap<>();
 
-        ReducingRangeMap.Builder<V> builder = new ReducingRangeMap.Builder<>(ranges.get(0).endInclusive(), ranges.size() * 2);
-        for (Range range : ranges)
+        return create(ranges, value, ReducingRangeMap.Builder::new);
+    }
+
+    public static <V, M extends ReducingRangeMap<V>> M create(Unseekables<?> keysOrRanges, V value, BuilderFactory<RoutingKey, V, M> builder)
+    {
+        switch (keysOrRanges.domain())
         {
-            builder.append(range.start(), value, (a, b) -> { throw new IllegalStateException(); });
-            builder.append(range.end(), null, (a, b) -> a); // if we are equal to prev end, take the prev value not zero
+            default: throw new AssertionError("Unhandled domain: " + keysOrRanges.domain());
+            case Range: return create((Ranges) keysOrRanges, value, builder);
+            case Key: return create((AbstractUnseekableKeys) keysOrRanges, value, builder);
         }
+    }
+
+    public static <V, M extends ReducingRangeMap<V>> M create(Seekables<?, ?> keysOrRanges, V value, BuilderFactory<RoutingKey, V, M> builder)
+    {
+        switch (keysOrRanges.domain())
+        {
+            default: throw new AssertionError("Unhandled domain: " + keysOrRanges.domain());
+            case Range: return create((Ranges) keysOrRanges, value, builder);
+            case Key: return create((Keys) keysOrRanges, value, builder);
+        }
+    }
+
+    public static <V, M extends ReducingRangeMap<V>> M create(Ranges ranges, V value, BuilderFactory<RoutingKey, V, M> factory)
+    {
+        if (value == null)
+            throw new IllegalArgumentException("value is null");
+
+        AbstractBoundariesBuilder<RoutingKey, V, M> builder = factory.create(ranges.get(0).endInclusive(), ranges.size() * 2);
+        for (Range cur : ranges)
+        {
+            builder.append(cur.start(), value, (a, b) -> { throw new IllegalStateException(); });
+            builder.append(cur.end(), null, (a, b) -> { throw new IllegalStateException(); });
+        }
+
+        return builder.build();
+    }
+
+    public static <V, M extends ReducingRangeMap<V>> M create(AbstractUnseekableKeys keys, V value, BuilderFactory<RoutingKey, V, M> factory)
+    {
+        if (value == null)
+            throw new IllegalArgumentException("value is null");
+
+        AbstractBoundariesBuilder<RoutingKey, V, M> builder = factory.create(keys.get(0).asRange().endInclusive(), keys.size() * 2);
+        for (int i = 0 ; i < keys.size() ; ++i)
+        {
+            Range range = keys.get(i).asRange();
+            builder.append(range.start(), value, (a, b) -> { throw new IllegalStateException(); });
+            builder.append(range.end(), null, (a, b) -> { throw new IllegalStateException(); });
+        }
+
+        return builder.build();
+    }
+
+    public static <V, M extends ReducingRangeMap<V>> M create(Keys keys, V value, BuilderFactory<RoutingKey, V, M> factory)
+    {
+        if (value == null)
+            throw new IllegalArgumentException("value is null");
+
+        RoutingKey prev = keys.get(0).toUnseekable();
+        AbstractBoundariesBuilder<RoutingKey, V, M> builder;
+        {
+            Range range = prev.asRange();
+            builder = factory.create(prev.asRange().endInclusive(), keys.size() * 2);
+            builder.append(range.start(), value, (a, b) -> { throw new IllegalStateException(); });
+            builder.append(range.end(), null, (a, b) -> { throw new IllegalStateException(); });
+        }
+
+        for (int i = 1 ; i < keys.size() ; ++i)
+        {
+            RoutingKey unseekable = keys.get(i).toUnseekable();
+            if (unseekable.equals(prev))
+                continue;
+
+            Range range = unseekable.asRange();
+            builder.append(range.start(), value, (a, b) -> { throw new IllegalStateException(); });
+            builder.append(range.end(), null, (a, b) -> { throw new IllegalStateException(); });
+            prev = unseekable;
+        }
+
         return builder.build();
     }
 
diff --git a/accord-core/src/test/java/accord/Utils.java b/accord-core/src/test/java/accord/Utils.java
index 18f1526..78b1298 100644
--- a/accord-core/src/test/java/accord/Utils.java
+++ b/accord-core/src/test/java/accord/Utils.java
@@ -32,8 +32,7 @@
 import accord.api.Scheduler;
 import accord.config.LocalConfig;
 import accord.config.MutableLocalConfig;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
 import accord.impl.InMemoryCommandStores;
 import accord.impl.IntKey;
 import accord.impl.SimpleProgressLog;
@@ -48,7 +47,6 @@
 import accord.local.Node;
 import accord.local.NodeTimeService;
 import accord.local.ShardDistributor;
-import accord.messages.Apply;
 import accord.messages.LocalRequest;
 import accord.primitives.Keys;
 import accord.primitives.Range;
@@ -176,7 +174,7 @@
                              scheduler,
                              SizeOfIntersectionSorter.SUPPLIER,
                              SimpleProgressLog::new,
-                             InMemoryCommandStores.Synchronized::new, ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+                             InMemoryCommandStores.Synchronized::new, new CoordinationAdapter.DefaultFactory(),
                              localConfig);
         awaitUninterruptibly(node.unsafeStart());
         return node;
diff --git a/accord-core/src/test/java/accord/burn/BurnTest.java b/accord-core/src/test/java/accord/burn/BurnTest.java
index af3f50b..bc8c226 100644
--- a/accord-core/src/test/java/accord/burn/BurnTest.java
+++ b/accord-core/src/test/java/accord/burn/BurnTest.java
@@ -108,12 +108,12 @@
      * Min hash value for the test domain, this value must be respected by the hash function
      * @see {@link BurnTest#hash(int)}
      */
-    private static final int HASH_RANGE_START = 0;
+    public static final int HASH_RANGE_START = 0;
     /**
      * Max hash value for the test domain, this value must be respected by the hash function
      * @see {@link BurnTest#hash(int)}
      */
-    private static final int HASH_RANGE_END = 1 << 16;
+    public static final int HASH_RANGE_END = 1 << 16;
     private static final Range[] EMPTY_RANGES = new Range[0];
 
     static List<Packet> generate(RandomSource random, MessageListener listener, Function<? super CommandStore, AsyncExecutor> executor, List<Id> clients, List<Id> nodes, int[] keys, int operations)
diff --git a/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java b/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java
index c8ea753..250f84e 100644
--- a/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java
+++ b/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java
@@ -73,6 +73,7 @@
 import static accord.impl.IntKey.range;
 import static accord.local.Status.Applied;
 import static accord.primitives.Routable.Domain.Key;
+import static accord.primitives.Routable.Domain.Range;
 import static accord.primitives.Txn.Kind.Read;
 import static accord.primitives.Txn.Kind.Write;
 import static accord.utils.Invariants.checkState;
@@ -121,7 +122,7 @@
             Node node = cluster.get(1);
             assertNotNull(node);
 
-            TxnId txnId = node.nextTxnId(Write, Key);
+            TxnId txnId = node.nextTxnId(Write, Range);
             Ranges keys = ranges(range(1, 2));
             Txn txn = writeTxn(keys);
             FullRangeRoute route = keys.toRoute(keys.get(0).someIntersectingRoutingKey(null));
@@ -219,7 +220,7 @@
             assertEquals(previousBarrierTimestamp, globalBarrierTimestamp);
 
             // Sync over nothing should work
-            SyncPoint<Ranges> syncPoint = getUninterruptibly(CoordinateSyncPoint.inclusive(node, ranges(range(99, 100)), false));
+            SyncPoint<Ranges> syncPoint = getUninterruptibly(CoordinateSyncPoint.inclusiveAndAwaitQuorum(node, ranges(range(99, 100))));
             assertEquals(node.epoch(), syncPoint.syncId.epoch());
 
             // Keys and so on for the upcoming transaction pair
@@ -247,13 +248,13 @@
                     Commands.preaccept(store, store.get(txnId, homeKey), txnId, txnId.epoch(), txn.slice(store.ranges().allAt(txnId.epoch()), true), route, null))));
 
 
-            AsyncResult<SyncPoint<Ranges>> syncInclusiveSyncFuture = CoordinateSyncPoint.inclusive(node, ranges, false);
+            AsyncResult<SyncPoint<Ranges>> syncInclusiveSyncFuture = CoordinateSyncPoint.inclusiveAndAwaitQuorum(node, ranges);
             // Shouldn't complete because it is blocked waiting for the dependency just created to apply
             sleep(500);
             assertFalse(syncInclusiveSyncFuture.isDone());
 
             // Async sync should return a result immediately since we are going to wait on the sync point transaction that was created by the sync point
-            AsyncResult<SyncPoint<Ranges>> asyncInclusiveSyncFuture = CoordinateSyncPoint.inclusive(node, ranges, true);
+            AsyncResult<SyncPoint<Ranges>> asyncInclusiveSyncFuture = CoordinateSyncPoint.inclusive(node, ranges);
             SyncPoint<Ranges> localSyncPoint = getUninterruptibly(asyncInclusiveSyncFuture);
             Semaphore localSyncOccurred = new Semaphore(0);
             node.commandStores().ifLocal(PreLoadContext.contextFor(localSyncPoint.syncId), homeKey, epoch, epoch, safeStore ->
diff --git a/accord-core/src/test/java/accord/impl/CommandsForKeyGroupUpdaterTest.java b/accord-core/src/test/java/accord/impl/CommandsForKeyGroupUpdaterTest.java
deleted file mode 100644
index 164c282..0000000
--- a/accord-core/src/test/java/accord/impl/CommandsForKeyGroupUpdaterTest.java
+++ /dev/null
@@ -1,180 +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 accord.impl;
-
-import accord.impl.CommandsForKeyGroupUpdater.Immutable;
-import accord.local.Command;
-import accord.local.SaveStatus;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
-import com.google.common.collect.ImmutableSortedSet;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-import java.util.List;
-
-public class CommandsForKeyGroupUpdaterTest
-{
-    private static class Sentinel { private static final Sentinel INSTANCE = new Sentinel(); }
-
-    private static final CommandTimeseries.CommandLoader<Sentinel> LOADER = new CommandTimeseries.CommandLoader<Sentinel>()
-    {
-        @Override public Sentinel saveForCFK(Command command) { return Sentinel.INSTANCE; }
-        @Override public TxnId txnId(Sentinel data) { throw new UnsupportedOperationException(); }
-        @Override public Timestamp executeAt(Sentinel data) { throw new UnsupportedOperationException(); }
-        @Override public SaveStatus saveStatus(Sentinel data) { throw new UnsupportedOperationException(); }
-        @Override public List<TxnId> depsIds(Sentinel data) { throw new UnsupportedOperationException(); }
-    };
-
-    private static final IntKey.Raw KEY = IntKey.key(1);
-
-    private static TxnId txnId(int i)
-    {
-        return TxnId.fromValues(0, i, 0, 1);
-    }
-
-    private static Timestamp ts(int i)
-    {
-        return Timestamp.fromValues(0, i, 0, 1);
-    }
-
-    private static CommandTimeseries<Sentinel> timeseries()
-    {
-        CommandTimeseries.Builder<Sentinel> builder = new CommandTimeseries.Builder<>(KEY, LOADER);
-        return builder.build();
-    }
-
-    private static CommandTimeseries<Sentinel> timeseries(Timestamp... timestamps)
-    {
-        CommandTimeseries.Builder<Sentinel> builder = new CommandTimeseries.Builder<>(KEY, LOADER);
-        for (Timestamp timestamp : timestamps)
-            builder.add(timestamp, Sentinel.INSTANCE);
-        return builder.build();
-    }
-
-    private static CommandTimeseries<Sentinel> timeseries(int... timestamps)
-    {
-        return timeseries(timestamps(timestamps));
-    }
-
-    private static CommandsForKey cfk(CommandTimeseries<Sentinel> commands)
-    {
-        return new CommandsForKey(KEY, commands);
-    }
-
-    private static CommandsForKey cfk(Timestamp[] byId)
-    {
-        return cfk(timeseries(byId));
-    }
-
-    private static CommandsForKey cfk()
-    {
-        return new CommandsForKey(KEY, timeseries());
-    }
-
-    private static Timestamp[] timestamps(int... values)
-    {
-        Timestamp[] timestamps = new Timestamp[values.length];
-        for (int i=0; i<values.length; i++)
-            timestamps[i] = ts(values[i]);
-        return timestamps;
-    }
-
-    private static void assertTimeseries(CommandTimeseries<?> timeseries, Timestamp... expected)
-    {
-        Assertions.assertEquals(ImmutableSortedSet.copyOf(expected), timeseries.commands.keySet());
-    }
-
-    private static void assertTimeseries(CommandTimeseries<?> timeseries, int... expected)
-    {
-        assertTimeseries(timeseries, timestamps(expected));
-    }
-
-    @Test
-    void commonTest()
-    {
-        CommandsForKeyGroupUpdater.Mutable<Sentinel> updater = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-
-        updater.common().commands().add(txnId(1), null);
-
-        Assertions.assertEquals(cfk(timeseries(1)), updater.applyToDeps(cfk()));
-        Assertions.assertEquals(cfk(timeseries(1)), updater.applyToAll(cfk()));
-    }
-
-    @Test
-    void specificPrecedence()
-    {
-        CommandsForKeyGroupUpdater.Mutable<Sentinel> updater = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-
-        updater.common().commands().add(txnId(2), null);
-        updater.common().commands().add(txnId(1), null);
-        updater.all().commands().remove(txnId(1));
-
-        Assertions.assertEquals(cfk(timeseries(1, 2)), updater.applyToDeps(cfk()));
-        Assertions.assertEquals(cfk(timeseries(2)), updater.applyToAll(cfk()));
-    }
-
-    /**
-     * A later specifc update should override a previous common update (making it specific to the opposite cfk)
-     */
-    @Test
-    void mergeOverridingSpecific()
-    {
-        CommandsForKeyGroupUpdater.Mutable<Sentinel> original = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-
-        original.common().commands().add(txnId(1), null);
-        original.common().commands().add(txnId(2), null);
-
-        CommandsForKeyGroupUpdater.Mutable<Sentinel> update = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-        update.deps().commands().remove(txnId(1));
-        update.deps().commands().add(txnId(3), null);
-
-        CommandsForKeyGroupUpdater.Mutable<Sentinel> expected = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-
-        expected.common().commands().add(txnId(2), null);
-        expected.deps().commands().remove(txnId(1));
-        expected.deps().commands().add(txnId(3), null);
-        expected.all().commands().add(txnId(1), null);
-
-//        Immutable<Sentinel> merged = original.toImmutable().mergeNewer(update.toImmutable());
-        Immutable<Sentinel> merged = Immutable.merge(original.toImmutable(), update.toImmutable(), Immutable.getFactory());
-        Assertions.assertEquals(expected.toImmutable(), merged);
-    }
-
-    @Test
-    void mergeOverridingCommon()
-    {
-        CommandsForKeyGroupUpdater.Mutable<Sentinel> original = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-        original.deps().commands().add(txnId(1), null);
-        original.deps().commands().add(txnId(2), null);
-        original.all().commands().add(txnId(2), null);
-
-        CommandsForKeyGroupUpdater.Mutable<Sentinel> update = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-        update.common().commands().remove(txnId(1));
-
-        CommandsForKeyGroupUpdater.Mutable<Sentinel> expected = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-        expected.deps().commands().add(txnId(2), null);
-        expected.all().commands().add(txnId(2), null);
-        expected.common().commands().remove(txnId(1));
-
-        Immutable<Sentinel> merged = Immutable.merge(original.toImmutable(), update.toImmutable(), Immutable.getFactory());
-        Assertions.assertEquals(expected.toImmutable(), merged);
-    }
-}
-
diff --git a/accord-core/src/test/java/accord/impl/PrefixedIntHashKey.java b/accord-core/src/test/java/accord/impl/PrefixedIntHashKey.java
index 2ee13d4..7e75acc 100644
--- a/accord-core/src/test/java/accord/impl/PrefixedIntHashKey.java
+++ b/accord-core/src/test/java/accord/impl/PrefixedIntHashKey.java
@@ -33,6 +33,7 @@
 
 import static accord.utils.Utils.toArray;
 
+// TODO (now): burn test is producing prefix keys over full integer range rather than 16bits
 public class PrefixedIntHashKey implements RoutableKey
 {
     public static final int MIN_KEY = Integer.MIN_VALUE + 1;
@@ -111,11 +112,6 @@
 
     public static final class Key extends PrefixedIntHashKey implements accord.api.Key
     {
-        private Key(int prefix, int key)
-        {
-            super(prefix, key, false);
-        }
-
         private Key(int prefix, int key, int hash)
         {
             super(prefix, key, hash);
@@ -124,9 +120,9 @@
 
     public static abstract class PrefixedIntRoutingKey extends PrefixedIntHashKey implements RoutingKey
     {
-        private PrefixedIntRoutingKey(int prefix, int key, boolean isHash)
+        private PrefixedIntRoutingKey(int prefix, int hash)
         {
-            super(prefix, key, isHash);
+            super(prefix, Integer.MIN_VALUE, hash);
         }
 
         @Override
@@ -142,7 +138,7 @@
 
         private Sentinel(int prefix, boolean isMin)
         {
-            super(prefix, isMin ? Integer.MIN_VALUE : Integer.MAX_VALUE, true);
+            super(prefix, isMin ? Integer.MIN_VALUE : Integer.MAX_VALUE);
             this.isMin = isMin;
         }
 
@@ -157,7 +153,7 @@
     {
         private Hash(int prefix, int hash)
         {
-            super(prefix, hash, true);
+            super(prefix, hash);
         }
 
         @Override
@@ -212,35 +208,14 @@
     public final int hash;
     private final boolean isHash;
 
-    private PrefixedIntHashKey(int prefix, int key, boolean isHash)
-    {
-        this.prefix = prefix;
-        this.isHash = isHash;
-        if (isHash)
-        {
-            this.key = Integer.MIN_VALUE;
-            this.hash = key;
-        }
-        else
-        {
-            this.key = key;
-            this.hash = hash(key);
-        }
-    }
-
     private PrefixedIntHashKey(int prefix, int key, int hash)
     {
         this.prefix = prefix;
-        this.isHash = false; // this constructor is only used by Key
+        this.isHash = key == Integer.MIN_VALUE; // this constructor is only used by Key
         this.key = key;
         this.hash = hash;
     }
 
-    public static Key key(int prefix, int k)
-    {
-        return new Key(prefix, k);
-    }
-
     public static Key key(int prefix, int k, int hash)
     {
         return new Key(prefix, k, hash);
@@ -305,7 +280,7 @@
         return Objects.hash(prefix, key, hash);
     }
 
-    static int hash(int key)
+    public static int hash(int key)
     {
         return CRCUtils.crc32LittleEnding(key);
     }
diff --git a/accord-core/src/test/java/accord/impl/basic/Cluster.java b/accord-core/src/test/java/accord/impl/basic/Cluster.java
index 1c61226..b949c5a 100644
--- a/accord-core/src/test/java/accord/impl/basic/Cluster.java
+++ b/accord-core/src/test/java/accord/impl/basic/Cluster.java
@@ -48,8 +48,7 @@
 import accord.burn.random.FrequentLargeRange;
 import accord.config.LocalConfig;
 import accord.config.MutableLocalConfig;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
 import accord.impl.CoordinateDurabilityScheduling;
 import accord.impl.MessageListener;
 import accord.impl.PrefixedIntHashKey;
@@ -62,7 +61,6 @@
 import accord.local.Node;
 import accord.local.NodeTimeService;
 import accord.local.ShardDistributor;
-import accord.messages.Apply;
 import accord.messages.LocalRequest;
 import accord.messages.Message;
 import accord.messages.MessageType;
@@ -299,7 +297,7 @@
                                      () -> new ListStore(id), new ShardDistributor.EvenSplit<>(8, ignore -> new PrefixedIntHashKey.Splitter()),
                                      nodeExecutor.agent(),
                                      randomSupplier.get(), sinks, SizeOfIntersectionSorter.SUPPLIER,
-                                     SimpleProgressLog::new, DelayedCommandStores.factory(sinks.pending, isLoadedCheck), ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+                                     SimpleProgressLog::new, DelayedCommandStores.factory(sinks.pending, isLoadedCheck), new CoordinationAdapter.DefaultFactory(),
                                      localConfig);
                 CoordinateDurabilityScheduling durability = new CoordinateDurabilityScheduling(node);
                 // TODO (desired): randomise
diff --git a/accord-core/src/test/java/accord/impl/list/ListAgent.java b/accord-core/src/test/java/accord/impl/list/ListAgent.java
index 7c97a2e..092fb74 100644
--- a/accord-core/src/test/java/accord/impl/list/ListAgent.java
+++ b/accord-core/src/test/java/accord/impl/list/ListAgent.java
@@ -108,4 +108,10 @@
     {
         return new Txn.InMemory(kind, keysOrRanges, new ListRead(identity(), false, Keys.EMPTY, Keys.EMPTY), new ListQuery(NONE, Integer.MIN_VALUE), null);
     }
+
+    public boolean collectMaxApplied()
+    {
+        // TODO (expected): randomise this to exercise both code paths
+        return false;
+    }
 }
diff --git a/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java b/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java
index 7917524..4e8b3e9 100644
--- a/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java
+++ b/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java
@@ -25,13 +25,17 @@
 import accord.api.Data;
 import accord.api.DataStore;
 import accord.impl.AbstractFetchCoordinator;
+import accord.impl.InMemoryCommandStore;
 import accord.local.CommandStore;
 import accord.local.Node;
 import accord.local.PreLoadContext;
+import accord.primitives.PartialDeps;
 import accord.primitives.PartialTxn;
 import accord.primitives.Ranges;
 import accord.primitives.SyncPoint;
+import accord.primitives.Timestamp;
 import accord.primitives.Txn;
+import accord.primitives.TxnId;
 import accord.utils.Timestamped;
 import accord.utils.async.AsyncResult;
 
@@ -66,4 +70,51 @@
             else fail(from, received, fail);
         }).beginAsResult());
     }
+
+    @Override
+    protected FetchRequest newFetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn)
+    {
+        if (((ListAgent)node.agent()).collectMaxApplied())
+            return new CollectMaxAppliedFetchRequest(sourceEpoch, syncId, ranges, partialDeps, partialTxn);
+
+        return super.newFetchRequest(sourceEpoch, syncId, ranges, partialDeps, partialTxn);
+    }
+
+    class CollectMaxAppliedFetchRequest extends FetchRequest
+    {
+        private transient Timestamp maxApplied;
+
+        public CollectMaxAppliedFetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn)
+        {
+            super(sourceEpoch, syncId, ranges, partialDeps, partialTxn);
+        }
+
+        @Override
+        protected void readComplete(CommandStore commandStore, Data result, Ranges unavailable)
+        {
+            Ranges slice = commandStore.unsafeRangesForEpoch().allAt(txnId).subtract(unavailable);
+            ((InMemoryCommandStore)commandStore).maxAppliedFor((Ranges)readScope, slice).begin((newMaxApplied, failure) -> {
+                if (failure != null)
+                {
+                    commandStore.agent().onUncaughtException(failure);
+                }
+                else
+                {
+                    synchronized (this)
+                    {
+                        if (maxApplied == null) maxApplied = newMaxApplied;
+                        else maxApplied = Timestamp.max(maxApplied, newMaxApplied);
+                        super.readComplete(commandStore, result, unavailable);
+                    }
+                }
+            });
+        }
+
+        @Override
+        protected Timestamp maxApplied()
+        {
+            return maxApplied;
+        }
+    }
+
 }
diff --git a/accord-core/src/test/java/accord/impl/list/ListRequest.java b/accord-core/src/test/java/accord/impl/list/ListRequest.java
index 7d796da..560cf39 100644
--- a/accord-core/src/test/java/accord/impl/list/ListRequest.java
+++ b/accord-core/src/test/java/accord/impl/list/ListRequest.java
@@ -53,6 +53,7 @@
 import static accord.local.Status.Applied;
 import static accord.local.Status.Phase.Cleanup;
 
+import static accord.local.Status.PreAccepted;
 import static accord.local.Status.PreApplied;
 import static accord.local.Status.PreCommitted;
 import static accord.utils.Invariants.illegalState;
@@ -110,9 +111,9 @@
             if (failure != null) callback.accept(null, failure);
             else if (merged.maxKnowledgeSaveStatus.is(Status.Invalidated)) callback.accept(Outcome.Invalidated, null);
             else if (merged.maxKnowledgeSaveStatus.is(Status.Truncated)) callback.accept(Outcome.Truncated, null);
-            else if (!merged.maxKnowledgeSaveStatus.hasBeen(PreCommitted) && merged.maxSaveStatus.phase == Cleanup) callback.accept(Outcome.Truncated, null);
-            else if (count == nodes().size()) callback.accept(Outcome.Lost, null);
             else if (merged.maxKnowledgeSaveStatus.hasBeen(Applied)) callback.accept(new Outcome(Outcome.Kind.Applied, (ListResult) ((CheckStatus.CheckStatusOkFull) merged).result), null);
+            else if (!merged.maxKnowledgeSaveStatus.hasBeen(PreCommitted) && merged.maxSaveStatus.phase == Cleanup) callback.accept(Outcome.Truncated, null);
+            else if (!merged.maxSaveStatus.hasBeen(PreAccepted) && count >= (1 + nodes().size())/2) callback.accept(Outcome.Lost, null);
             else callback.accept(Outcome.Other, null);
         }
 
@@ -155,15 +156,14 @@
                 else if (fail instanceof CoordinationFailed)
                 {
                     RoutingKey homeKey = ((CoordinationFailed) fail).homeKey();
-                    TxnId txnId = ((CoordinationFailed) fail).txnId();
                     if (fail instanceof Invalidated)
                     {
-                        node.reply(client, replyContext, ListResult.invalidated(client, ((Packet)replyContext).requestId, txnId), null);
+                        node.reply(client, replyContext, ListResult.invalidated(client, ((Packet)replyContext).requestId, id), null);
                         return;
                     }
 
-                    node.reply(client, replyContext, ListResult.heartBeat(client, ((Packet)replyContext).requestId, txnId), null);
-                    ((Cluster) node.scheduler()).onDone(() -> checkOnResult(homeKey, txnId, 0, null));
+                    node.reply(client, replyContext, ListResult.heartBeat(client, ((Packet)replyContext).requestId, id), null);
+                    ((Cluster) node.scheduler()).onDone(() -> checkOnResult(homeKey, id, 0, null));
                 }
                 else if (fail instanceof SimulatedFault)
                 {
diff --git a/accord-core/src/test/java/accord/impl/list/ListStore.java b/accord-core/src/test/java/accord/impl/list/ListStore.java
index e9afc8c..d669fd8 100644
--- a/accord-core/src/test/java/accord/impl/list/ListStore.java
+++ b/accord-core/src/test/java/accord/impl/list/ListStore.java
@@ -49,14 +49,18 @@
 import accord.primitives.RoutableKey;
 import accord.primitives.SyncPoint;
 import accord.primitives.Timestamp;
+import accord.primitives.TxnId;
 import accord.topology.Topologies;
 import accord.topology.Topology;
 import accord.utils.Timestamped;
 import accord.utils.async.AsyncChain;
+import accord.utils.async.AsyncChains;
 import accord.utils.async.AsyncResults;
 import org.agrona.collections.Int2ObjectHashMap;
 import org.agrona.collections.LongArrayList;
 
+import static accord.utils.Invariants.illegalState;
+
 public class ListStore implements DataStore
 {
     private static class ChangeAt
@@ -188,12 +192,24 @@
 
     private void checkAccess(Timestamp executeAt, Range range)
     {
+        if (executeAt instanceof TxnId)
+        {
+            switch (((TxnId) executeAt).kind())
+            {
+                case EphemeralRead:
+                case ExclusiveSyncPoint:
+                    // TODO (required): introduce some mechanism for EphemeralReads/ExclusiveSyncPoints to abort if local store has been expunged (and check it here)
+                    return; // safe to access later
+            }
+        }
         Ranges singleRanges = Ranges.of(range);
         if (!allowed.containsAll(singleRanges))
-            throw new IllegalStateException(String.format("Attempted to access range %s on node %s, which is not in the range %s;\nexecuteAt = %s\n%s",
-                                                          range, node, allowed,
-                                                          executeAt,
-                                                          history(singleRanges)));
+        {
+            // TODO (required): it is actually safe for a node on an old epoch to still be executing a transaction that has been executed in a later epoch,
+            //   making this check over-enthusiastic.
+            illegalState(String.format("Attempted to access range %s on node %s, which is not in the range %s;\nexecuteAt = %s\n%s",
+                                       range, node, allowed, executeAt, history(singleRanges)));
+        }
     }
 
     private String history(String type, Object key, Predicate<Ranges> test)
@@ -465,6 +481,7 @@
     }
 
     // TODO (duplication): this is 95% of accord.coordinate.CoordinateShardDurable
+    //   we already report all this information to EpochState; would be better to use that
     private static class Await extends AsyncResults.SettableResult<SyncPoint<Ranges>> implements Callback<ReadReply>
     {
         private final Node node;
@@ -484,12 +501,14 @@
             Await coordinate = new Await(node, minEpoch, sp);
             coordinate.start();
             return coordinate.recover(t -> {
-                // TODO (effecicency): backoff
                 if (t instanceof Timeout ||
                     // TODO (expected): why are we not simply handling Insufficient properly?
                     t instanceof RuntimeException && "Insufficient".equals(t.getMessage()) ||
                     t instanceof SimulatedFault)
                     return coordinate(node, minEpoch, sp);
+                // cannot loop indefinitely
+                if (t instanceof RuntimeException && "Redundant".equals(t.getMessage()))
+                    return AsyncChains.success(null);
                 return null;
             });
         }
diff --git a/accord-core/src/test/java/accord/impl/list/ListWrite.java b/accord-core/src/test/java/accord/impl/list/ListWrite.java
index ca80714..8966ec6 100644
--- a/accord-core/src/test/java/accord/impl/list/ListWrite.java
+++ b/accord-core/src/test/java/accord/impl/list/ListWrite.java
@@ -54,7 +54,7 @@
         ListStore s = (ListStore) store;
         if (!containsKey(key))
             return Writes.SUCCESS;
-        CommandsForKeys.updateLastExecutionTimestamps((AbstractSafeCommandStore<?, ?, ?, ?>) safeStore, (RoutableKey) key, executeAt, true);
+        CommandsForKeys.updateLastExecutionTimestamps((AbstractSafeCommandStore<?, ?, ?>) safeStore, (RoutableKey) key, executeAt, true);
 
         return executor.apply(safeStore.commandStore()).submit(() -> {
             int[] data = get(key);
diff --git a/accord-core/src/test/java/accord/impl/mock/MockCluster.java b/accord-core/src/test/java/accord/impl/mock/MockCluster.java
index 9e1fddb..f3b80c5 100644
--- a/accord-core/src/test/java/accord/impl/mock/MockCluster.java
+++ b/accord-core/src/test/java/accord/impl/mock/MockCluster.java
@@ -34,8 +34,7 @@
 
 import accord.NetworkFilter;
 import accord.api.MessageSink;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
 import accord.config.LocalConfig;
 import accord.impl.InMemoryCommandStores;
 import accord.impl.IntKey;
@@ -47,7 +46,6 @@
 import accord.local.Node.Id;
 import accord.local.NodeTimeService;
 import accord.local.ShardDistributor;
-import accord.messages.Apply;
 import accord.config.MutableLocalConfig;
 import accord.messages.Callback;
 import accord.messages.LocalRequest;
@@ -139,9 +137,7 @@
                              SizeOfIntersectionSorter.SUPPLIER,
                              SimpleProgressLog::new,
                              InMemoryCommandStores.SingleThread::new,
-                             ExecuteTxn.FACTORY,
-                             PersistTxn.FACTORY,
-                             Apply.FACTORY,
+                             new CoordinationAdapter.DefaultFactory(),
                              localConfig);
         awaitUninterruptibly(node.unsafeStart());
         node.onTopologyUpdate(topology, true);
diff --git a/accord-core/src/test/java/accord/local/ImmutableCommandTest.java b/accord-core/src/test/java/accord/local/ImmutableCommandTest.java
index 35afe00..a36ba8a 100644
--- a/accord-core/src/test/java/accord/local/ImmutableCommandTest.java
+++ b/accord-core/src/test/java/accord/local/ImmutableCommandTest.java
@@ -32,8 +32,7 @@
 import accord.api.RoutingKey;
 import accord.api.Scheduler;
 import accord.api.TestableConfigurationService;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
 import accord.config.LocalConfig;
 import accord.impl.InMemoryCommandStore;
 import accord.impl.InMemoryCommandStores;
@@ -46,7 +45,6 @@
 import accord.impl.mock.MockStore;
 import accord.local.Node.Id;
 import accord.local.SaveStatus.LocalExecution;
-import accord.messages.Apply;
 import accord.config.MutableLocalConfig;
 import accord.primitives.FullKeyRoute;
 import accord.primitives.Keys;
@@ -121,7 +119,7 @@
                              clock, NodeTimeService.unixWrapper(TimeUnit.MICROSECONDS, clock),
                              () -> storeSupport.data, new ShardDistributor.EvenSplit(8, ignore -> new IntKey.Splitter()), new TestAgent(), new DefaultRandom(), Scheduler.NEVER_RUN_SCHEDULED,
                              SizeOfIntersectionSorter.SUPPLIER, ignore -> ignore2 -> new NoOpProgressLog(), InMemoryCommandStores.Synchronized::new,
-                             ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+                             new CoordinationAdapter.DefaultFactory(),
                              localConfig);
         awaitUninterruptibly(node.unsafeStart());
         node.onTopologyUpdate(storeSupport.local.get(), true);
diff --git a/accord-core/src/test/java/accord/messages/PreAcceptTest.java b/accord-core/src/test/java/accord/messages/PreAcceptTest.java
index f2593d7..4344f77 100644
--- a/accord-core/src/test/java/accord/messages/PreAcceptTest.java
+++ b/accord-core/src/test/java/accord/messages/PreAcceptTest.java
@@ -23,13 +23,9 @@
 
 import java.util.List;
 import java.util.concurrent.ExecutionException;
-import java.util.function.BiFunction;
-import java.util.stream.Stream;
 
 import accord.api.RoutingKey;
 import accord.impl.AbstractSafeCommandStore;
-import accord.impl.CommandTimeseries.CommandLoader;
-import accord.impl.CommandTimeseries;
 import accord.impl.CommandsForKey;
 import accord.impl.IntKey.Raw;
 import accord.impl.IntKey;
@@ -86,11 +82,6 @@
         return PreAccept.SerializerSupport.create(txnId, route.slice(FULL_RANGE), txnId.epoch(), txnId.epoch(), false, txnId.epoch(), txn.slice(FULL_RANGE, true), route);
     }
 
-    static <T, D> Stream<T> convert(CommandTimeseries<D> timeseries, BiFunction<CommandLoader<D>, D, T> get)
-    {
-        return timeseries.all().map(d -> get.apply(timeseries.loader(), d));
-    }
-
     @Test
     void initialCommandTest() throws ExecutionException
     {
@@ -103,7 +94,7 @@
         {
             Raw key = IntKey.key(10);
             CommandStore commandStore = node.unsafeForKey(key);
-            Assertions.assertFalse(inMemory(commandStore).hasDepsCommandsForKey(key));
+            Assertions.assertFalse(inMemory(commandStore).hasCommandsForKey(key));
 
             TxnId txnId = clock.idForNode(1, ID2);
             Txn txn = writeTxn(Keys.of(key));
@@ -112,8 +103,8 @@
             preAccept.process(node, ID2, REPLY_CONTEXT);
 
             commandStore.execute(PreLoadContext.contextFor(txnId, txn.keys()), safeStore -> {
-                CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).depsCommandsForKey(key).current();
-                TxnId commandId = convert(cfk.commands(), CommandLoader::txnId).findFirst().get();
+                CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).commandsForKey(key).current();
+                TxnId commandId = cfk.findFirst();
                 Command command = safeStore.ifInitialised(commandId).current();
                 Assertions.assertEquals(Status.PreAccepted, command.status());
             });
@@ -140,7 +131,7 @@
         {
             Raw key = IntKey.key(10);
             CommandStore commandStore = node.unsafeForKey(key);
-            Assertions.assertFalse(inMemory(commandStore).hasDepsCommandsForKey(key));
+            Assertions.assertFalse(inMemory(commandStore).hasCommandsForKey(key));
 
             TxnId txnId = clock.idForNode(1, ID2);
             Txn txn = writeTxn(Keys.of(key));
@@ -163,7 +154,7 @@
         {
             Raw key = IntKey.key(10);
             CommandStore commandStore = node.unsafeForKey(key);
-            Assertions.assertFalse(inMemory(commandStore).hasDepsCommandsForKey(key));
+            Assertions.assertFalse(inMemory(commandStore).hasCommandsForKey(key));
 
             TxnId txnId = clock.idForNode(1, ID2);
             Txn txn = writeTxn(Keys.of(key));
@@ -281,8 +272,8 @@
             preAccept.process(node, ID2, REPLY_CONTEXT);
 
             commandStore.execute(PreLoadContext.contextFor(txnId, txn.keys()), safeStore -> {
-                CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).depsCommandsForKey(key).current();
-                TxnId commandId = convert(cfk.commands(), CommandLoader::txnId).findFirst().get();
+                CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).commandsForKey(key).current();
+                TxnId commandId = cfk.findFirst();
                 Command command = safeStore.ifInitialised(commandId).current();
                 Assertions.assertEquals(Status.PreAccepted, command.status());
             });
diff --git a/accord-core/src/test/java/accord/messages/ReadDataTest.java b/accord-core/src/test/java/accord/messages/ReadDataTest.java
index 6aa4dd2..34ef106 100644
--- a/accord-core/src/test/java/accord/messages/ReadDataTest.java
+++ b/accord-core/src/test/java/accord/messages/ReadDataTest.java
@@ -44,6 +44,7 @@
 import accord.impl.TopologyFactory;
 import accord.impl.mock.MockCluster;
 import accord.local.CheckedCommands;
+import accord.local.Command;
 import accord.local.CommandStore;
 import accord.local.Node;
 import accord.local.PreLoadContext;
@@ -150,7 +151,7 @@
                 CheckedCommands.accept(safe, state.txnId, Ballot.ZERO, state.partialRoute, state.partialTxn.keys(), state.progressKey, state.executeAt, state.deps);
 
                 SafeCommand safeCommand = safe.ifInitialised(state.txnId);
-                safeCommand.commit(safeCommand.current(), Ballot.ZERO, state.executeAt);
+                safeCommand.stable(safe, safeCommand.current(), Ballot.ZERO, state.executeAt, Command.WaitingOn.EMPTY);
             })));
 
             ReplyContext replyContext = state.process();
@@ -184,7 +185,7 @@
             store = stores.get(1);
             check(store.execute(PreLoadContext.contextFor(state.txnId, state.keys), safe -> {
                 SafeCommand command = safe.get(state.txnId, state.txnId, state.route);
-                command.commitInvalidated();
+                command.commitInvalidated(safe);
             }));
 
             ReplyContext replyContext = state.process();
@@ -200,7 +201,7 @@
             List<CommandStore> stores = stores(state);
             stores.forEach(store -> check(store.execute(PreLoadContext.contextFor(state.txnId, state.keys), safe -> {
                 SafeCommand command = safe.get(state.txnId, state.txnId, state.route);
-                command.commitInvalidated();
+                command.commitInvalidated(safe);
             })));
             ReplyContext replyContext = state.process();
 
diff --git a/accord-core/src/test/java/accord/primitives/DepsBuilderTest.java b/accord-core/src/test/java/accord/primitives/DepsBuilderTest.java
deleted file mode 100644
index cd56d22..0000000
--- a/accord-core/src/test/java/accord/primitives/DepsBuilderTest.java
+++ /dev/null
@@ -1,161 +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 accord.primitives;
-
-import accord.api.Agent;
-import accord.api.DataStore;
-import accord.api.Key;
-import accord.api.ProgressLog;
-import accord.impl.DepsBuilder;
-import accord.impl.DepsBuilder.Builder;
-import accord.impl.IntKey;
-import accord.local.*;
-import com.google.common.collect.ImmutableList;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-import javax.annotation.Nullable;
-import java.util.*;
-import java.util.function.Predicate;
-
-public class DepsBuilderTest
-{
-    private static final Key KEY = IntKey.key(1);
-    private static class InstrumentedSafeStore extends SafeCommandStore
-    {
-        public static class Removed
-        {
-            public final Set<TxnId> commands = new HashSet<>();
-
-            void add(TxnId txnId, Timestamp executeAt)
-            {
-                commands.add(txnId);
-            }
-        }
-
-        Map<Seekable, Removed> removed = new HashMap<>();
-
-        @Override
-        public void removeCommandFromSeekableDeps(Seekable seekable, TxnId txnId, Timestamp executeAt, Status status)
-        {
-            removed.computeIfAbsent(seekable, s -> new Removed()).add(txnId, executeAt);
-        }
-
-        @Override protected SafeCommand getInternal(TxnId txnId) { throw new UnsupportedOperationException(); }
-        @Override protected SafeCommand getInternalIfLoadedAndInitialised(TxnId txnId) { throw new UnsupportedOperationException(); }
-        @Override public boolean canExecuteWith(PreLoadContext context) { throw new UnsupportedOperationException(); }
-        @Override public <P1, T> T mapReduce(Seekables<?, ?> keys, Ranges slice, KeyHistory keyHistory, Txn.Kind.Kinds testKind, TestTimestamp testTimestamp, Timestamp timestamp, TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus, CommandFunction<P1, T, T> map, P1 p1, T initialValue, Predicate<? super T> terminate) { throw new UnsupportedOperationException(); }
-        @Override protected void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command) { throw new UnsupportedOperationException(); }
-        @Override protected void register(Seekable keyOrRange, Ranges slice, Command command) { throw new UnsupportedOperationException(); }
-        @Override public CommandStore commandStore() { throw new UnsupportedOperationException(); }
-        @Override public DataStore dataStore() { throw new UnsupportedOperationException(); }
-        @Override public Agent agent() { throw new UnsupportedOperationException(); }
-        @Override public ProgressLog progressLog() { throw new UnsupportedOperationException(); }
-        @Override public NodeTimeService time() { throw new UnsupportedOperationException(); }
-        @Override public CommandStores.RangesForEpoch ranges() { throw new UnsupportedOperationException(); }
-        @Override public Timestamp maxConflict(Seekables<?, ?> keys, Ranges slice) { throw new UnsupportedOperationException(); }
-        @Override public void registerHistoricalTransactions(Deps deps) { throw new UnsupportedOperationException(); }
-        @Override public void erase(SafeCommand safeCommand) { throw new UnsupportedOperationException(); }
-    }
-    private static final Node.Id NODE = new Node.Id(0);
-
-    private static TxnId txnId(int v)
-    {
-        return TxnId.fromValues(1, v, 0, 0);
-    }
-
-    private static List<TxnId> txnIds(int... vs)
-    {
-        ImmutableList.Builder<TxnId> builder = ImmutableList.builder();
-        Arrays.sort(vs);
-        for (int v : vs)
-            builder.add(txnId(v));
-        return builder.build();
-    }
-
-    private static Timestamp timestamp(int v)
-    {
-        return Timestamp.fromValues(1, v, NODE);
-    }
-
-    private static List<TxnId> build(SafeCommandStore safeStore, Seekable seekable, Builder builder)
-    {
-        List<TxnId> result = new ArrayList<>();
-        builder.build(safeStore, seekable, result::add);
-        result.sort(Comparator.naturalOrder());
-        return result;
-    }
-
-    private static List<TxnId> build(SafeCommandStore safeStore, Builder builder)
-    {
-        return build(safeStore, KEY, builder);
-    }
-
-    @BeforeAll
-    static void setUpClass()
-    {
-        DepsBuilder.setCanPruneUnsafe(true);
-    }
-
-    @AfterAll
-    static void tearDownClass()
-    {
-        DepsBuilder.setCanPruneUnsafe(false);
-    }
-
-    @Test
-    void basicTest()
-    {
-        InstrumentedSafeStore safeStore = new InstrumentedSafeStore();
-        Builder builder = new Builder();
-        builder.add(txnId(0), Status.Applied, timestamp(0), txnIds());
-        builder.add(txnId(1), Status.Applied, timestamp(1), txnIds(0));
-        Assertions.assertEquals(txnIds(1), build(safeStore, builder));
-    }
-
-    /**
-     * Committed deps should be included if they point to an uncommitted command
-     */
-    @Test
-    void uncommittedTest()
-    {
-        InstrumentedSafeStore safeStore = new InstrumentedSafeStore();
-        Builder builder = new Builder();
-        builder.add(txnId(0), Status.Accepted, timestamp(0), txnIds());
-        builder.add(txnId(1), Status.Applied, timestamp(1), txnIds(0));
-        builder.add(txnId(2), Status.Applied, timestamp(2), txnIds(1));
-        builder.add(txnId(3), Status.Applied, timestamp(3), txnIds(2));
-        Assertions.assertEquals(txnIds(0, 1, 2, 3), build(safeStore, builder));
-    }
-
-    /**
-     * Committed txns should be included if they point to a command with a later execution time
-     */
-    @Test
-    void executionOrderTest()
-    {
-        InstrumentedSafeStore safeStore = new InstrumentedSafeStore();
-        Builder builder = new Builder();
-        builder.add(txnId(0), Status.Applied, timestamp(2), txnIds());
-        builder.add(txnId(1), Status.Applied, timestamp(1), txnIds(0));
-        Assertions.assertEquals(txnIds(0, 1), build(safeStore, builder));
-    }
-}
diff --git a/accord-core/src/test/java/accord/topology/TopologyRandomizer.java b/accord-core/src/test/java/accord/topology/TopologyRandomizer.java
index 555969d..7b3a919 100644
--- a/accord-core/src/test/java/accord/topology/TopologyRandomizer.java
+++ b/accord-core/src/test/java/accord/topology/TopologyRandomizer.java
@@ -50,6 +50,9 @@
 import accord.utils.RandomSource;
 import org.agrona.collections.IntHashSet;
 
+import static accord.burn.BurnTest.HASH_RANGE_END;
+import static accord.burn.BurnTest.HASH_RANGE_START;
+
 
 // TODO (required, testing): add change replication factor
 public class TopologyRandomizer
@@ -348,7 +351,7 @@
         }
         List<Shard> result = new ArrayList<>(shards.length + nodes.length);
         result.addAll(Arrays.asList(shards));
-        Range[] ranges = PrefixedIntHashKey.ranges(prefix, nodes.length);
+        Range[] ranges = PrefixedIntHashKey.ranges(prefix, HASH_RANGE_START, HASH_RANGE_END, nodes.length);
         for (int i = 0; i < ranges.length; i++)
         {
             Range range = ranges[i];
diff --git a/accord-core/src/test/java/accord/utils/AccordGens.java b/accord-core/src/test/java/accord/utils/AccordGens.java
index 0186ff3..d87c2d5 100644
--- a/accord-core/src/test/java/accord/utils/AccordGens.java
+++ b/accord-core/src/test/java/accord/utils/AccordGens.java
@@ -187,9 +187,14 @@
         return prefixedIntHashKey(prefixGen, rs -> rs.nextInt(PrefixedIntHashKey.MIN_KEY, Integer.MAX_VALUE));
     }
 
-    public static Gen<Key> prefixedIntHashKey(Gen.IntGen prefixGen, Gen.IntGen valueGen)
+    public static Gen<Key> prefixedIntHashKey(Gen.IntGen prefixGen, Gen.IntGen keyGen)
     {
-        return rs -> PrefixedIntHashKey.key(prefixGen.nextInt(rs), valueGen.nextInt(rs));
+        return rs -> {
+            int prefix = prefixGen.nextInt(rs);
+            int key = keyGen.nextInt(rs);
+            int hash = PrefixedIntHashKey.hash(key);
+            return PrefixedIntHashKey.key(prefix, key, hash);
+        };
     }
 
     public static Gen<Key> prefixedIntHashKeyInsideRanges(Ranges ranges)
@@ -201,7 +206,7 @@
             // end inclusive, so +1 the result to include end and exclude start
             int hash = rs.nextInt(start.hash, end.hash) + 1;
             int key = CRCUtils.reverseCRC32LittleEnding(hash);
-            PrefixedIntHashKey.Key ret = PrefixedIntHashKey.key(start.prefix, key);
+            PrefixedIntHashKey.Key ret = PrefixedIntHashKey.key(start.prefix, key, hash);
             // we have tests to make sure this doesn't fail... just a safety check
             assert ret.hash == hash;
             return ret;
diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java b/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java
index 67ed4f3..1d9abff 100644
--- a/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java
+++ b/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java
@@ -43,8 +43,7 @@
 import accord.api.Scheduler;
 import accord.config.LocalConfig;
 import accord.config.MutableLocalConfig;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
 import accord.impl.InMemoryCommandStores;
 import accord.impl.SimpleProgressLog;
 import accord.impl.SizeOfIntersectionSorter;
@@ -53,7 +52,6 @@
 import accord.local.Node.Id;
 import accord.local.NodeTimeService;
 import accord.local.ShardDistributor;
-import accord.messages.Apply;
 import accord.messages.Callback;
 import accord.messages.LocalRequest;
 import accord.messages.Reply;
@@ -328,7 +326,7 @@
                                           MaelstromStore::new, new ShardDistributor.EvenSplit(8, ignore -> new MaelstromKey.Splitter()),
                                           MaelstromAgent.INSTANCE,
                                           randomSupplier.get(), sinks, SizeOfIntersectionSorter.SUPPLIER,
-                                          SimpleProgressLog::new, InMemoryCommandStores.SingleThread::new, ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+                                          SimpleProgressLog::new, InMemoryCommandStores.SingleThread::new, new CoordinationAdapter.DefaultFactory(),
                                           localConfig));
             }
 
diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/Main.java b/accord-maelstrom/src/main/java/accord/maelstrom/Main.java
index 50d1aee..a7ac2ce 100644
--- a/accord-maelstrom/src/main/java/accord/maelstrom/Main.java
+++ b/accord-maelstrom/src/main/java/accord/maelstrom/Main.java
@@ -34,9 +34,8 @@
 import accord.api.Scheduler;
 import accord.config.LocalConfig;
 import accord.config.MutableLocalConfig;
+import accord.coordinate.CoordinationAdapter;
 import accord.coordinate.Timeout;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
 import accord.impl.InMemoryCommandStores;
 import accord.impl.SimpleProgressLog;
 import accord.impl.SizeOfIntersectionSorter;
@@ -46,7 +45,6 @@
 import accord.local.NodeTimeService;
 import accord.local.ShardDistributor;
 import accord.maelstrom.Packet.Type;
-import accord.messages.Apply;
 import accord.messages.Callback;
 import accord.messages.LocalRequest;
 import accord.messages.Reply;
@@ -183,7 +181,7 @@
                           System::currentTimeMillis, NodeTimeService.unixWrapper(TimeUnit.MILLISECONDS, System::currentTimeMillis),
                           MaelstromStore::new, new ShardDistributor.EvenSplit(8, ignore -> new MaelstromKey.Splitter()),
                           MaelstromAgent.INSTANCE, new DefaultRandom(), scheduler, SizeOfIntersectionSorter.SUPPLIER,
-                          SimpleProgressLog::new, InMemoryCommandStores.SingleThread::new, ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+                          SimpleProgressLog::new, InMemoryCommandStores.SingleThread::new, new CoordinationAdapter.DefaultFactory(),
                           localConfig);
             awaitUninterruptibly(on.unsafeStart());
             err.println("Initialized node " + init.self);