CEP-15/Accord Immutable State Refactor

Patch by Blake Eggleston; Reviewed by David Capwell and Benedict Elliott Smith for Cassandra-18192
diff --git a/accord-core/src/main/java/accord/api/Agent.java b/accord-core/src/main/java/accord/api/Agent.java
index 0045eb0..b7bf5bc 100644
--- a/accord-core/src/main/java/accord/api/Agent.java
+++ b/accord-core/src/main/java/accord/api/Agent.java
@@ -18,8 +18,8 @@
 
 package accord.api;
 
-import accord.local.Node;
 import accord.local.Command;
+import accord.local.Node;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 
diff --git a/accord-core/src/main/java/accord/api/VisibleForImplementation.java b/accord-core/src/main/java/accord/api/VisibleForImplementation.java
new file mode 100644
index 0000000..36f1223
--- /dev/null
+++ b/accord-core/src/main/java/accord/api/VisibleForImplementation.java
@@ -0,0 +1,34 @@
+/*
+ * 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.api;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Inherited;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Indicates an element exists or is more visible than otherwise needed to support implementations
+ */
+@Documented
+@Inherited
+@Retention(RetentionPolicy.SOURCE)
+public @interface VisibleForImplementation
+{
+}
diff --git a/accord-core/src/main/java/accord/coordinate/CheckOn.java b/accord-core/src/main/java/accord/coordinate/CheckOn.java
index 0bcd756..56ea22d 100644
--- a/accord-core/src/main/java/accord/coordinate/CheckOn.java
+++ b/accord-core/src/main/java/accord/coordinate/CheckOn.java
@@ -21,14 +21,13 @@
 import java.util.function.BiConsumer;
 
 import accord.local.*;
+import accord.local.Commands.ApplyOutcome;
+import accord.local.Commands.CommitOutcome;
 import accord.primitives.*;
 import accord.utils.Invariants;
 import accord.utils.MapReduceConsume;
-import com.google.common.base.Preconditions;
 
 import accord.api.RoutingKey;
-import accord.local.Command.ApplyOutcome;
-import accord.local.Command.CommitOutcome;
 import accord.local.Node.Id;
 import accord.messages.CheckStatus.CheckStatusOk;
 import accord.messages.CheckStatus.CheckStatusOkFull;
@@ -67,7 +66,7 @@
     {
         // TODO (desired, efficiency): restore behaviour of only collecting info if e.g. Committed or Executed
         super(node, txnId, routeWithHomeKey, srcEpoch, IncludeInfo.All);
-        Preconditions.checkArgument(routeWithHomeKey.contains(route.homeKey()));
+        Invariants.checkArgument(routeWithHomeKey.contains(route.homeKey()));
         this.sufficient = sufficient;
         this.route = route;
         this.callback = callback;
@@ -109,7 +108,7 @@
     @Override
     protected void onDone(Success success, Throwable failure)
     {
-        Preconditions.checkState((success == null) != (failure == null));
+        Invariants.checkState((success == null) != (failure == null));
         if (failure != null)
         {
             callback.accept(null, failure);
@@ -174,7 +173,6 @@
         @Override
         public Void apply(SafeCommandStore safeStore)
         {
-            Command command = safeStore.command(txnId);
             switch (sufficientFor.propagate())
             {
                 default: throw new IllegalStateException();
@@ -185,30 +183,31 @@
                     throw new IllegalStateException("Invalid states to propagate");
 
                 case Invalidated:
-                    command.commitInvalidate(safeStore);
+                    Commands.commitInvalidate(safeStore, txnId);
                     break;
 
                 case Applied:
                 case PreApplied:
                     if (untilLocalEpoch >= full.executeAt.epoch())
                     {
-                        confirm(command.commit(safeStore, maxRoute, progressKey, partialTxn, full.executeAt, partialDeps));
-                        confirm(command.apply(safeStore, untilLocalEpoch, maxRoute, full.executeAt, partialDeps, full.writes, full.result));
+                        confirm(Commands.commit(safeStore, txnId, maxRoute, progressKey, partialTxn, full.executeAt, partialDeps));
+                        confirm(Commands.apply(safeStore, txnId, untilLocalEpoch, maxRoute, full.executeAt, partialDeps, full.writes, full.result));
                         break;
                     }
 
                 case Committed:
                 case ReadyToExecute:
-                    confirm(command.commit(safeStore, maxRoute, progressKey, partialTxn, full.executeAt, partialDeps));
+                    confirm(Commands.commit(safeStore, txnId, maxRoute, progressKey, partialTxn, full.executeAt, partialDeps));
                     break;
 
                 case PreCommitted:
-                    command.precommit(safeStore, full.executeAt);
+                    Commands.precommit(safeStore, txnId, full.executeAt);
                     if (!sufficientFor.definition.isKnown())
                         break;
 
                 case PreAccepted:
-                    command.preaccept(safeStore, partialTxn, maxRoute, progressKey);
+                    if (!safeStore.ranges().at(txnId.epoch()).isEmpty())
+                        Commands.preaccept(safeStore, txnId, partialTxn, maxRoute, progressKey);
                     break;
 
                 case NotWitnessed:
@@ -223,8 +222,8 @@
                 return null;
 
             Timestamp executeAt = merged.saveStatus.known.executeAt.hasDecidedExecuteAt() ? merged.executeAt : null;
-            command.setDurability(safeStore, merged.durability, homeKey, executeAt);
-            safeStore.progressLog().durable(command, null);
+            Commands.setDurability(safeStore, txnId, merged.durability, homeKey, executeAt);
+            safeStore.progressLog().durable(safeStore.command(txnId).current(), null);
             return null;
         }
 
diff --git a/accord-core/src/main/java/accord/coordinate/Coordinate.java b/accord-core/src/main/java/accord/coordinate/Coordinate.java
index 28d5e17..839f41c 100644
--- a/accord-core/src/main/java/accord/coordinate/Coordinate.java
+++ b/accord-core/src/main/java/accord/coordinate/Coordinate.java
@@ -32,7 +32,7 @@
 import accord.messages.PreAccept;
 import accord.messages.PreAccept.PreAcceptOk;
 import accord.messages.PreAccept.PreAcceptReply;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import static accord.coordinate.Propose.Invalidate.proposeInvalidate;
 import static accord.messages.Commit.Invalidate.commitInvalidate;
@@ -46,7 +46,7 @@
  *
  * TODO (desired, testing): dedicated burn test to validate outcomes
  */
-public class Coordinate extends AsyncResults.Settable<Result> implements Callback<PreAcceptReply>, BiConsumer<Result, Throwable>
+public class Coordinate extends AsyncResults.SettableResult<Result> implements Callback<PreAcceptReply>, BiConsumer<Result, Throwable>
 {
     final Node node;
     final TxnId txnId;
@@ -134,7 +134,7 @@
 
     private synchronized void onPreAccepted()
     {
-        Preconditions.checkState(!preAcceptIsDone);
+        Invariants.checkState(!preAcceptIsDone);
         preAcceptIsDone = true;
 
         if (tracker.hasFastPathAccepted())
diff --git a/accord-core/src/main/java/accord/coordinate/InformHomeOfTxn.java b/accord-core/src/main/java/accord/coordinate/InformHomeOfTxn.java
index 46011b7..dc6499c 100644
--- a/accord-core/src/main/java/accord/coordinate/InformHomeOfTxn.java
+++ b/accord-core/src/main/java/accord/coordinate/InformHomeOfTxn.java
@@ -27,13 +27,13 @@
 import accord.messages.SimpleReply;
 import accord.topology.Shard;
 import accord.primitives.TxnId;
-import accord.utils.async.AsyncResult;
+import accord.utils.async.AsyncChain;
 import accord.utils.async.AsyncResults;
 
 import static accord.coordinate.tracking.AbstractTracker.ShardOutcomes.Fail;
 import static accord.coordinate.tracking.AbstractTracker.ShardOutcomes.Success;
 
-public class InformHomeOfTxn extends AsyncResults.Settable<Void> implements Callback<SimpleReply>
+public class InformHomeOfTxn extends AsyncResults.SettableResult<Void> implements Callback<SimpleReply>
 {
     final TxnId txnId;
     final RoutingKey homeKey;
@@ -47,7 +47,7 @@
         this.tracker = new QuorumShardTracker(homeShard);
     }
 
-    public static AsyncResult<Void> inform(Node node, TxnId txnId, RoutingKey homeKey)
+    public static AsyncChain<Void> inform(Node node, TxnId txnId, RoutingKey homeKey)
     {
         return node.withEpoch(txnId.epoch(), () -> {
             Shard homeShard = node.topology().forEpoch(homeKey, txnId.epoch());
diff --git a/accord-core/src/main/java/accord/coordinate/Invalidate.java b/accord-core/src/main/java/accord/coordinate/Invalidate.java
index 6abdd91..c1bdf74 100644
--- a/accord-core/src/main/java/accord/coordinate/Invalidate.java
+++ b/accord-core/src/main/java/accord/coordinate/Invalidate.java
@@ -30,6 +30,7 @@
 import accord.local.Node.Id;
 import accord.local.Status;
 import accord.messages.Commit;
+import accord.local.*;
 import accord.primitives.*;
 import accord.topology.Topologies;
 
@@ -39,7 +40,6 @@
 import accord.messages.BeginInvalidation.InvalidateReply;
 import accord.messages.Callback;
 import accord.utils.Invariants;
-import com.google.common.base.Preconditions;
 
 import javax.annotation.Nullable;
 
@@ -189,7 +189,7 @@
                         Status witnessedByInvalidation = maxReply.status;
                         if (!witnessedByInvalidation.hasBeen(Accepted))
                         {
-                            Preconditions.checkState(tracker.all(InvalidationShardTracker::isPromised));
+                            Invariants.checkState(tracker.all(InvalidationShardTracker::isPromised));
                             if (!invalidateWith.containsAll(route))
                                 witnessedByInvalidation = null;
                         }
@@ -197,19 +197,19 @@
                     }
                     else if (homeKey != null)
                     {
-                        Preconditions.checkState(maxReply.status.hasBeen(Accepted) || tracker.all(InvalidationShardTracker::isPromised));
+                        Invariants.checkState(maxReply.status.hasBeen(Accepted) || tracker.all(InvalidationShardTracker::isPromised));
                         // if we included the home shard, and we have either a recoverable status OR have not rejected the fast path,
                         // we must have at least one response that should contain the Route
                         if (invalidateWith.contains(homeKey) && tracker.isPromisedForKey(homeKey, txnId.epoch()))
                             throw new IllegalStateException("Received replies from a node that must have known the route, but that did not include it");
 
                         // if < Accepted, we should have short-circuited to invalidation above. This guarantees no Invaldate/Recover loop, as any later status will forbid invoking Invalidate
-                        Preconditions.checkState(!(transitivelyInvokedByPriorInvalidation && !maxReply.status.hasBeen(Accepted)));
+                        Invariants.checkState(!(transitivelyInvokedByPriorInvalidation && !maxReply.status.hasBeen(Accepted)));
 
                         Status witnessedByInvalidation = maxReply.status;
                         if (!witnessedByInvalidation.hasBeen(Accepted))
                         {
-                            Preconditions.checkState(tracker.all(InvalidationShardTracker::isPromised));
+                            Invariants.checkState(tracker.all(InvalidationShardTracker::isPromised));
                             if (!invalidateWith.contains(homeKey))
                                 witnessedByInvalidation = null;
                         }
@@ -268,7 +268,7 @@
         Commit.Invalidate.commitInvalidate(node, txnId, route != null ? Unseekables.merge(route, (Unseekables)invalidateWith) : invalidateWith, txnId);
         // 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.forEachLocalSince(contextFor(txnId), invalidateWith, txnId, safeStore -> {
-            safeStore.command(txnId).commitInvalidate(safeStore);
+            Commands.commitInvalidate(safeStore, txnId);
         }).begin((s, f) -> {
             callback.accept(INVALIDATED, null);
             if (f != null) // TODO (required): consider exception handling more carefully: should we catch these prior to passing to callbacks?
diff --git a/accord-core/src/main/java/accord/coordinate/Persist.java b/accord-core/src/main/java/accord/coordinate/Persist.java
index 63b84ee..076cc66 100644
--- a/accord-core/src/main/java/accord/coordinate/Persist.java
+++ b/accord-core/src/main/java/accord/coordinate/Persist.java
@@ -56,7 +56,7 @@
     public static void persist(Node node, Topologies sendTo, Topologies applyTo, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
     {
         Persist persist = new Persist(node, applyTo, txnId, route, txn, executeAt, deps);
-        node.send(sendTo.nodes(), to -> new Apply(to, sendTo, applyTo, executeAt.epoch(), txnId, route, executeAt, deps, writes, result), persist);
+        node.send(sendTo.nodes(), to -> new Apply(to, sendTo, applyTo, executeAt.epoch(), txnId, route, txn, executeAt, deps, writes, result), persist);
     }
 
     public static void persistAndCommit(Node node, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
@@ -64,7 +64,7 @@
         Topologies sendTo = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
         Topologies applyTo = node.topology().forEpoch(route, executeAt.epoch());
         Persist persist = new Persist(node, sendTo, txnId, route, txn, executeAt, deps);
-        node.send(sendTo.nodes(), to -> new Apply(to, sendTo, applyTo, executeAt.epoch(), txnId, route, executeAt, deps, writes, result), persist);
+        node.send(sendTo.nodes(), to -> new Apply(to, sendTo, applyTo, executeAt.epoch(), txnId, route, txn, executeAt, deps, writes, result), persist);
     }
 
     private Persist(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps)
diff --git a/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java b/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java
index 45afb84..a4edb85 100644
--- a/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java
+++ b/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java
@@ -20,7 +20,6 @@
 
 import accord.coordinate.tracking.RequestStatus;
 import accord.messages.Callback;
-import com.google.common.base.Preconditions;
 
 import accord.coordinate.tracking.ReadTracker;
 import accord.local.Node;
@@ -67,6 +66,7 @@
          */
         Approve
     }
+
     protected enum Success { Quorum, Success }
 
     final Node node;
@@ -103,7 +103,7 @@
                 break;
 
             case TryAlternative:
-                Preconditions.checkState(!reply.isFinal());
+                Invariants.checkState(!reply.isFinal());
                 onSlowResponse(from);
                 break;
 
@@ -158,7 +158,7 @@
 
     protected void finishOnFailure()
     {
-        Preconditions.checkState(!isDone);
+        Invariants.checkState(!isDone);
         isDone = true;
         if (failure == null)
             failure = new Exhausted(txnId, null);
@@ -173,7 +173,7 @@
             case NoChange:
                 break;
             case Success:
-                Preconditions.checkState(!isDone);
+                Invariants.checkState(!isDone);
                 isDone = true;
                 onDone(waitingOnData == 0 ? Success.Success : Success.Quorum, null);
                 break;
diff --git a/accord-core/src/main/java/accord/coordinate/Recover.java b/accord-core/src/main/java/accord/coordinate/Recover.java
index b0ba463..5dbd819 100644
--- a/accord-core/src/main/java/accord/coordinate/Recover.java
+++ b/accord-core/src/main/java/accord/coordinate/Recover.java
@@ -55,7 +55,7 @@
 // TODO (low priority, cleanup): rename to Recover (verb); rename Recover message to not clash
 public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throwable>
 {
-    class AwaitCommit extends AsyncResults.Settable<Timestamp> implements Callback<WaitOnCommitOk>
+    class AwaitCommit extends AsyncResults.SettableResult<Timestamp> implements Callback<WaitOnCommitOk>
     {
         // TODO (desired, efficiency): this should collect the executeAt of any commit, and terminate as soon as one is found
         //                             that is earlier than TxnId for the Txn we are recovering; if all commits we wait for
diff --git a/accord-core/src/main/java/accord/coordinate/RecoverWithHomeKey.java b/accord-core/src/main/java/accord/coordinate/RecoverWithHomeKey.java
index 0e4fe60..33d01da 100644
--- a/accord-core/src/main/java/accord/coordinate/RecoverWithHomeKey.java
+++ b/accord-core/src/main/java/accord/coordinate/RecoverWithHomeKey.java
@@ -27,7 +27,7 @@
 import accord.messages.CheckStatus.IncludeInfo;
 import accord.primitives.RoutingKeys;
 import accord.primitives.TxnId;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import static accord.primitives.Route.castToFullRoute;
 import static accord.primitives.Route.isFullRoute;
@@ -47,9 +47,9 @@
         super(node, txnId, RoutingKeys.of(homeKey), txnId.epoch(), IncludeInfo.Route);
         this.witnessedByInvalidation = witnessedByInvalidation;
         // if witnessedByInvalidation == AcceptedInvalidate then we cannot assume its definition was known, and our comparison with the status is invalid
-        Preconditions.checkState(witnessedByInvalidation != Status.AcceptedInvalidate);
+        Invariants.checkState(witnessedByInvalidation != Status.AcceptedInvalidate);
         // if witnessedByInvalidation == Invalidated we should anyway not be recovering
-        Preconditions.checkState(witnessedByInvalidation != Status.Invalidated);
+        Invariants.checkState(witnessedByInvalidation != Status.Invalidated);
         this.homeKey = homeKey;
         this.callback = callback;
     }
diff --git a/accord-core/src/main/java/accord/coordinate/tracking/AbstractTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/AbstractTracker.java
index 68a3201..b479ba4 100644
--- a/accord-core/src/main/java/accord/coordinate/tracking/AbstractTracker.java
+++ b/accord-core/src/main/java/accord/coordinate/tracking/AbstractTracker.java
@@ -23,8 +23,8 @@
 import accord.topology.Topologies;
 import accord.topology.Topology;
 
+import accord.utils.Invariants;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 
 import java.util.*;
 import java.util.function.BiFunction;
@@ -82,7 +82,7 @@
 
     AbstractTracker(Topologies topologies, IntFunction<ST[]> arrayFactory, Function<Shard, ST> trackerFactory)
     {
-        Preconditions.checkArgument(topologies.totalShards() > 0);
+        Invariants.checkArgument(topologies.totalShards() > 0);
         int topologyCount = topologies.size();
         int maxShardsPerEpoch = topologies.get(0).size();
         int shardCount = maxShardsPerEpoch;
@@ -126,7 +126,7 @@
     <T extends AbstractTracker<ST, P>>
     RequestStatus recordResponse(T self, Id node, BiFunction<? super ST, P, ? extends ShardOutcome<? super T>> function, P param, int topologyLimit)
     {
-        Preconditions.checkState(self == this); // we just accept self as parameter for type safety
+        Invariants.checkState(self == this); // we just accept self as parameter for type safety
         ShardOutcomes status = NoChange;
         int maxShards = maxShardsPerEpoch();
         for (int i = 0; i < topologyLimit && !status.isTerminalState() ; ++i)
diff --git a/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java b/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java
new file mode 100644
index 0000000..e0cfa02
--- /dev/null
+++ b/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.impl;
+
+import accord.api.VisibleForImplementation;
+import accord.impl.CommandsForKey.CommandLoader;
+import accord.local.*;
+import accord.primitives.*;
+
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+public abstract class AbstractSafeCommandStore<CommandType extends SafeCommand, CommandsForKeyType extends SafeCommandsForKey> implements 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;
+    }
+
+    protected abstract CommandType getCommandInternal(TxnId txnId);
+    protected abstract void addCommandInternal(CommandType command);
+
+    protected abstract CommandsForKeyType getCommandsForKeyInternal(RoutableKey key);
+    protected abstract void addCommandsForKeyInternal(CommandsForKeyType cfk);
+
+    protected abstract CommandType getIfLoaded(TxnId txnId);
+
+    private static <K, V> V getIfLoaded(K key, Function<K, V> get, Consumer<V> add, Function<K, V> getIfLoaded)
+    {
+        V value = get.apply(key);
+        if (value != null)
+            return value;
+
+        value = getIfLoaded.apply(key);
+        if (value == null)
+            return null;
+        add.accept(value);
+        return value;
+    }
+
+    @Override
+    public CommandType ifPresent(TxnId txnId)
+    {
+        CommandType command = getCommandInternal(txnId);
+        if (command == null)
+            throw new IllegalStateException(String.format("%s was not specified in PreLoadContext", txnId));
+        if (command.isEmpty())
+            return null;
+        return command;
+    }
+
+    @Override
+    public CommandType ifLoaded(TxnId txnId)
+    {
+        CommandType command = getIfLoaded(txnId, this::getCommandInternal, this::addCommandInternal, this::getIfLoaded);
+        if (command == null)
+            return null;
+        if (command.isEmpty())
+            command.notWitnessed();
+        return command;
+    }
+
+    @Override
+    public CommandType command(TxnId txnId)
+    {
+        CommandType command = getCommandInternal(txnId);
+        if (command == null)
+            throw new IllegalStateException(String.format("%s was not specified in PreLoadContext", txnId));
+        if (command.isEmpty())
+            command.notWitnessed();
+        return command;
+    }
+
+    protected abstract CommandLoader<?> cfkLoader();
+
+    public CommandsForKeyType ifLoaded(RoutableKey key)
+    {
+        CommandsForKeyType cfk = getIfLoaded(key, this::getCommandsForKeyInternal, this::addCommandsForKeyInternal, this::getIfLoaded);
+        if (cfk == null)
+            return null;
+        if (cfk.isEmpty())
+            cfk.initialize(cfkLoader());
+        return cfk;
+    }
+
+    public CommandsForKeyType commandsForKey(RoutableKey key)
+    {
+        CommandsForKeyType cfk = getCommandsForKeyInternal(key);
+        if (cfk == null)
+            throw new IllegalStateException(String.format("%s was not specified in PreLoadContext", key));
+        if (cfk.isEmpty())
+            cfk.initialize(cfkLoader());
+        return cfk;
+    }
+
+    protected abstract CommandsForKeyType getIfLoaded(RoutableKey key);
+
+    @VisibleForImplementation
+    public CommandsForKeyType maybeCommandsForKey(RoutableKey key)
+    {
+        CommandsForKeyType cfk = getIfLoaded(key, this::getCommandsForKeyInternal, this::addCommandsForKeyInternal, this::getIfLoaded);
+        if (cfk == null || cfk.isEmpty())
+            return null;
+        return cfk;
+    }
+
+    @Override
+    public boolean canExecuteWith(PreLoadContext 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()));
+    }
+
+    protected abstract Timestamp maxConflict(Seekables<?, ?> keysOrRanges, Ranges slice);
+
+    @Override
+    public Timestamp preaccept(TxnId txnId, Seekables<?, ?> keys)
+    {
+        Timestamp max = maxConflict(keys, ranges().at(txnId.epoch()));
+        long epoch = latestEpoch();
+        long now = time().now();
+        if (txnId.compareTo(max) > 0 && txnId.epoch() >= epoch && !agent().isExpired(txnId, now))
+            return txnId;
+
+        return time().uniqueNow(max);
+    }
+
+    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 = command(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();
+
+    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) -> command(txnId).updateAttributes(attributes)));
+        }
+    }
+
+    public void complete()
+    {
+        invalidateSafeState();
+    }
+}
diff --git a/accord-core/src/main/java/accord/impl/CommandsForKey.java b/accord-core/src/main/java/accord/impl/CommandsForKey.java
index 91a77ca..9c1d20a 100644
--- a/accord-core/src/main/java/accord/impl/CommandsForKey.java
+++ b/accord-core/src/main/java/accord/impl/CommandsForKey.java
@@ -20,97 +20,419 @@
 
 import accord.api.Key;
 import accord.local.*;
-import accord.local.SafeCommandStore.CommandFunction;
-import accord.local.SafeCommandStore.TestDep;
-import accord.local.SafeCommandStore.TestKind;
-import accord.primitives.Keys;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import accord.primitives.*;
+import com.google.common.collect.ImmutableSortedMap;
 
 import javax.annotation.Nullable;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
 
-public abstract class CommandsForKey implements CommandListener
+import static accord.local.SafeCommandStore.TestDep.ANY_DEPS;
+import static accord.local.SafeCommandStore.TestDep.WITH;
+import static accord.local.SafeCommandStore.TestKind.Ws;
+import static accord.local.Status.PreAccepted;
+import static accord.local.Status.PreCommitted;
+import static accord.utils.Utils.*;
+
+public class CommandsForKey
 {
-    private static final Logger logger = LoggerFactory.getLogger(CommandsForKey.class);
-
-    public interface CommandTimeseries
+    public static class SerializerSupport
     {
-        void add(Timestamp timestamp, Command command);
-        void remove(Timestamp timestamp);
-
-        boolean isEmpty();
-
-        enum TestTimestamp { BEFORE, AFTER }
-
-        /**
-         * All commands before/after (exclusive of) the given timestamp
-         *
-         * 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.
-         *
-         * TODO (expected, efficiency): TestDep should be asynchronous; data should not be kept memory-resident as only used for recovery
-         */
-        <T> T mapReduce(TestKind testKind, TestTimestamp testTimestamp, Timestamp timestamp,
-                        TestDep testDep, @Nullable TxnId depId,
-                        @Nullable Status minStatus, @Nullable Status maxStatus,
-                        CommandFunction<T, T> map, T initialValue, T terminalValue);
-    }
-
-    public abstract Key key();
-    public abstract CommandTimeseries byId();
-    public abstract CommandTimeseries byExecuteAt();
-
-    public abstract Timestamp max();
-    protected abstract void updateMax(Timestamp timestamp);
-
-    @Override
-    public PreLoadContext listenerPreLoadContext(TxnId caller)
-    {
-        return PreLoadContext.contextFor(caller, Keys.of(key()));
-    }
-
-    @Override
-    public void onChange(SafeCommandStore safeStore, Command command)
-    {
-        logger.trace("[{}]: updating as listener in response to change on {} with status {} ({})",
-                     key(), command.txnId(), command.status(), command);
-        updateMax(command.executeAt());
-        switch (command.status())
+        public static CommandsForKey.Listener listener(Key key)
         {
-            default: throw new AssertionError();
-            case PreAccepted:
-            case NotWitnessed:
-            case Accepted:
-            case AcceptedInvalidate:
-            case PreCommitted:
-                break;
-            case Applied:
-            case PreApplied:
-            case Committed:
-            case ReadyToExecute:
-                byExecuteAt().remove(command.txnId());
-                byExecuteAt().add(command.executeAt(), command);
-                break;
-            case Invalidated:
-                byId().remove(command.txnId());
-                byExecuteAt().remove(command.txnId());
-                command.removeListener(this);
-                break;
+            return new CommandsForKey.Listener(key);
+        }
+
+        public static  <D> CommandsForKey create(Key key, Timestamp max,
+                                                 Timestamp lastExecutedTimestamp, long lastExecutedMicros, Timestamp lastWriteTimestamp,
+                                                 CommandLoader<D> loader,
+                                                 ImmutableSortedMap<Timestamp, D> byId,
+                                                 ImmutableSortedMap<Timestamp, D> byExecuteAt)
+        {
+            return new CommandsForKey(key, max, lastExecutedTimestamp, lastExecutedMicros, lastWriteTimestamp, loader, byId, byExecuteAt);
         }
     }
 
-    public void register(Command command)
+    public interface CommandLoader<D>
     {
-        updateMax(command.executeAt());
-        byId().add(command.txnId(), command);
-        byExecuteAt().add(command.txnId(), command);
-        command.addListener(this);
+        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 boolean isEmpty()
+    public static class CommandTimeseries<D>
     {
-        return byId().isEmpty();
+        public enum TestTimestamp {BEFORE, AFTER}
+
+        private final Key key;
+        protected final CommandLoader<D> loader;
+        public final ImmutableSortedMap<Timestamp, D> commands;
+
+        public CommandTimeseries(Update<D> builder)
+        {
+            this.key = builder.key;
+            this.loader = builder.loader;
+            this.commands = ensureSortedImmutable(builder.commands);
+        }
+
+        CommandTimeseries(Key key, CommandLoader<D> loader, ImmutableSortedMap<Timestamp, D> commands)
+        {
+            this.key = key;
+            this.loader = loader;
+            this.commands = commands;
+        }
+
+        public CommandTimeseries(Key key, CommandLoader<D> loader)
+        {
+            this(key, 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 key.equals(that.key) && loader.equals(that.loader) && commands.equals(that.commands);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            int hash = 1;
+            hash = 31 * hash + Objects.hashCode(key);
+            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();
+        }
+
+        /**
+         * 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 <T> T mapReduce(SafeCommandStore.TestKind testKind, TestTimestamp testTimestamp, Timestamp timestamp,
+                               SafeCommandStore.TestDep testDep, @Nullable TxnId depId,
+                               @Nullable Status minStatus, @Nullable Status maxStatus,
+                               SafeCommandStore.CommandFunction<T, T> map, T initialValue, T terminalValue)
+        {
+
+            for (D data : (testTimestamp == TestTimestamp.BEFORE ? commands.headMap(timestamp, false) : commands.tailMap(timestamp, false)).values())
+            {
+                TxnId txnId = loader.txnId(data);
+                Timestamp executeAt = loader.executeAt(data);
+                SaveStatus status = loader.saveStatus(data);
+                List<TxnId> deps = loader.depsIds(data);
+                if (testKind == Ws && txnId.isRead()) continue;
+                // 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;
+                if (minStatus != null && minStatus.compareTo(status.status) > 0)
+                    continue;
+                if (maxStatus != null && maxStatus.compareTo(status.status) < 0)
+                    continue;
+                initialValue = map.apply(key, txnId, executeAt, initialValue);
+                if (initialValue.equals(terminalValue))
+                    break;
+            }
+            return initialValue;
+        }
+
+        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();
+        }
+
+        Update<D> beginUpdate()
+        {
+            return new Update<>(this);
+        }
+
+        public CommandLoader<D> loader()
+        {
+            return loader;
+        }
+
+        public static class Update<D>
+        {
+            private final Key key;
+            protected CommandLoader<D> loader;
+            protected NavigableMap<Timestamp, D> commands;
+
+            public Update(Key key, CommandLoader<D> loader)
+            {
+                this.key = key;
+                this.loader = loader;
+                this.commands = new TreeMap<>();
+            }
+
+            public Update(CommandTimeseries<D> timeseries)
+            {
+                this.key = timeseries.key;
+                this.loader = timeseries.loader;
+                this.commands = timeseries.commands;
+            }
+
+            public CommandsForKey.CommandTimeseries.Update<D> add(Timestamp timestamp, Command command)
+            {
+                commands = ensureSortedMutable(commands);
+                commands.put(timestamp, loader.saveForCFK(command));
+                return this;
+            }
+
+            public CommandsForKey.CommandTimeseries.Update<D> remove(Timestamp timestamp)
+            {
+                commands = ensureSortedMutable(commands);
+                commands.remove(timestamp);
+                return this;
+            }
+
+            CommandTimeseries<D> build()
+            {
+                return new CommandTimeseries<>(this);
+            }
+        }
+    }
+
+    public static class Listener implements CommandListener
+    {
+        protected final Key listenerKey;
+
+        public Listener(Key listenerKey)
+        {
+            this.listenerKey = listenerKey;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            Listener that = (Listener) o;
+            return listenerKey.equals(that.listenerKey);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(listenerKey);
+        }
+
+        @Override
+        public String toString()
+        {
+            return "ListenerProxy{" + listenerKey + '}';
+        }
+
+        public Key key()
+        {
+            return listenerKey;
+        }
+
+        @Override
+        public void onChange(SafeCommandStore safeStore, SafeCommand safeCommand)
+        {
+            SafeCommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).commandsForKey(listenerKey);
+            cfk.listenerUpdate(safeCommand.current());
+        }
+
+        @Override
+        public PreLoadContext listenerPreLoadContext(TxnId caller)
+        {
+            return PreLoadContext.contextFor(caller, Keys.of(listenerKey));
+        }
+    }
+
+    // TODO (now): add validation that anything inserted into *committedBy* has everything prior in its dependencies
+    private final Key key;
+    private final Timestamp max;
+    private final Timestamp lastExecutedTimestamp;
+    private final long lastExecutedMicros;
+    private final Timestamp lastWriteTimestamp;
+    private final CommandTimeseries<?> byId;
+    private final CommandTimeseries<?> byExecuteAt;
+
+    <D> CommandsForKey(Key key, Timestamp max,
+                       Timestamp lastExecutedTimestamp,
+                       long lastExecutedMicros,
+                       Timestamp lastWriteTimestamp,
+                       CommandTimeseries<D> byId,
+                       CommandTimeseries<D> byExecuteAt)
+    {
+        this.key = key;
+        this.max = max;
+        this.lastExecutedTimestamp = lastExecutedTimestamp;
+        this.lastExecutedMicros = lastExecutedMicros;
+        this.lastWriteTimestamp = lastWriteTimestamp;
+        this.byId = byId;
+        this.byExecuteAt = byExecuteAt;
+    }
+
+    <D> CommandsForKey(Key key, Timestamp max,
+                       Timestamp lastExecutedTimestamp,
+                       long lastExecutedMicros,
+                       Timestamp lastWriteTimestamp,
+                       CommandLoader<D> loader,
+                       ImmutableSortedMap<Timestamp, D> committedById,
+                       ImmutableSortedMap<Timestamp, D> committedByExecuteAt)
+    {
+        this(key, max, lastExecutedTimestamp, lastExecutedMicros, lastWriteTimestamp,
+             new CommandTimeseries<>(key, loader, committedById),
+             new CommandTimeseries<>(key, loader, committedByExecuteAt));
+    }
+
+    public <D> CommandsForKey(Key key, CommandLoader<D> loader)
+    {
+        this.key = key;
+        this.max = Timestamp.NONE;
+        this.lastExecutedTimestamp = Timestamp.NONE;
+        this.lastExecutedMicros = 0;
+        this.lastWriteTimestamp = Timestamp.NONE;
+        this.byId = new CommandTimeseries<>(key, loader);
+        this.byExecuteAt = new CommandTimeseries<>(key, loader);
+    }
+
+    @Override
+    public String toString()
+    {
+        return "CommandsForKey@" + System.identityHashCode(this) + '{' + key + '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        CommandsForKey that = (CommandsForKey) o;
+        return lastExecutedMicros == that.lastExecutedMicros
+                && key.equals(that.key)
+                && Objects.equals(max, that.max)
+                && Objects.equals(lastExecutedTimestamp, that.lastExecutedTimestamp)
+                && Objects.equals(lastWriteTimestamp, that.lastWriteTimestamp)
+                && byId.equals(that.byId)
+                && byExecuteAt.equals(that.byExecuteAt);
+    }
+
+    @Override
+    public final int hashCode()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public final CommandListener asListener()
+    {
+        return new Listener(key());
+    }
+
+    public Key key()
+    {
+        return key;
+    }
+
+    public Timestamp max()
+    {
+        return max;
+    }
+
+    public Timestamp lastExecutedTimestamp()
+    {
+        return lastExecutedTimestamp;
+    }
+
+    public long lastExecutedMicros()
+    {
+        return lastExecutedMicros;
+    }
+
+    public Timestamp lastWriteTimestamp()
+    {
+        return lastWriteTimestamp;
+    }
+
+    public CommandTimeseries<?> byId()
+    {
+        return byId;
+    }
+
+    public CommandTimeseries<?> byExecuteAt()
+    {
+        return byExecuteAt;
+    }
+
+    public void forWitnessed(Timestamp minTs, Timestamp maxTs, Consumer<TxnId> consumer)
+    {
+        byId.between(minTs, maxTs, status -> status.hasBeen(PreAccepted)).forEach(consumer);
+        byExecuteAt.between(minTs, maxTs, status -> status.hasBeen(PreCommitted)).forEach(consumer);
+    }
+
+    private static long getTimestampMicros(Timestamp timestamp)
+    {
+        return timestamp.hlc();
+    }
+
+
+    private void validateExecuteAtTime(Timestamp executeAt, boolean isForWriteTxn)
+    {
+        if (executeAt.compareTo(lastWriteTimestamp) < 0)
+            throw new IllegalArgumentException(String.format("%s is less than the most recent write timestamp %s", executeAt, lastWriteTimestamp));
+
+        int cmp = executeAt.compareTo(lastExecutedTimestamp);
+        // execute can be in the past if it's for a read and after the most recent write
+        if (cmp == 0 || (!isForWriteTxn && cmp < 0))
+            return;
+        if (cmp < 0)
+            throw new IllegalArgumentException(String.format("%s is less than the most recent executed timestamp %s", executeAt, lastExecutedTimestamp));
+        else
+            throw new IllegalArgumentException(String.format("%s is greater than the most recent executed timestamp, cfk should be updated", executeAt, lastExecutedTimestamp));
+    }
+
+    public int nowInSecondsFor(Timestamp executeAt, boolean isForWriteTxn)
+    {
+        validateExecuteAtTime(executeAt, isForWriteTxn);
+        // we use the executeAt time instead of the monotonic database timestamp to prevent uneven
+        // ttl expiration in extreme cases, ie 1M+ writes/second to a key causing timestamps to overflow
+        // into the next second on some keys and not others.
+        return Math.toIntExact(TimeUnit.MICROSECONDS.toSeconds(getTimestampMicros(lastExecutedTimestamp)));
+    }
+
+    public long timestampMicrosFor(Timestamp executeAt, boolean isForWriteTxn)
+    {
+        validateExecuteAtTime(executeAt, isForWriteTxn);
+        return lastExecutedMicros;
     }
 }
diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommand.java b/accord-core/src/main/java/accord/impl/InMemoryCommand.java
deleted file mode 100644
index 8c49b94..0000000
--- a/accord-core/src/main/java/accord/impl/InMemoryCommand.java
+++ /dev/null
@@ -1,342 +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.Result;
-import accord.api.RoutingKey;
-import accord.local.*;
-import accord.local.Status.Durability;
-import accord.local.Status.Known;
-import accord.primitives.*;
-import accord.primitives.Txn.Kind;
-import accord.utils.Invariants;
-
-import javax.annotation.Nullable;
-import java.util.*;
-
-import static accord.local.Status.Durability.Local;
-import static accord.local.Status.Durability.NotDurable;
-
-public class InMemoryCommand extends Command
-{
-    public final CommandStore commandStore;
-    private final TxnId txnId;
-
-    private Route<?> route;
-    private RoutingKey homeKey, progressKey;
-    private PartialTxn partialTxn;
-    private Ballot promised = Ballot.ZERO, accepted = Ballot.ZERO;
-    private Timestamp executeAt;
-    private @Nullable PartialDeps partialDeps = null;
-    private Writes writes;
-    private Result result;
-
-    private SaveStatus status = SaveStatus.NotWitnessed;
-
-    private Durability durability = NotDurable; // only set on home shard
-
-    private NavigableSet<TxnId> waitingOnCommit;
-    private NavigableMap<Timestamp, TxnId> waitingOnApply;
-
-    private final Listeners listeners = new Listeners();
-
-    public InMemoryCommand(CommandStore commandStore, TxnId txnId)
-    {
-        this.commandStore = commandStore;
-        this.txnId = txnId;
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        InMemoryCommand command = (InMemoryCommand) o;
-        return commandStore == command.commandStore
-                && txnId.equals(command.txnId)
-                && Objects.equals(homeKey, command.homeKey)
-                && Objects.equals(progressKey, command.progressKey)
-                && Objects.equals(partialTxn, command.partialTxn)
-                && promised.equals(command.promised)
-                && accepted.equals(command.accepted)
-                && Objects.equals(executeAt, command.executeAt)
-                && partialDeps.equals(command.partialDeps)
-                && Objects.equals(writes, command.writes)
-                && Objects.equals(result, command.result)
-                && status == command.status
-                && durability == command.durability
-                && Objects.equals(waitingOnCommit, command.waitingOnCommit)
-                && Objects.equals(waitingOnApply, command.waitingOnApply)
-                && Objects.equals(listeners, command.listeners);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return Objects.hash(commandStore, txnId, partialTxn, promised, accepted, executeAt, partialDeps, writes, result, status, waitingOnCommit, waitingOnApply, listeners);
-    }
-
-    @Override
-    public TxnId txnId()
-    {
-        return txnId;
-    }
-
-    @Override
-    public RoutingKey homeKey()
-    {
-        return homeKey;
-    }
-
-    @Override
-    protected void setHomeKey(RoutingKey key)
-    {
-        this.homeKey = key;
-    }
-
-    @Override
-    public RoutingKey progressKey()
-    {
-        return progressKey;
-    }
-
-    @Override
-    protected void setProgressKey(RoutingKey key)
-    {
-        this.progressKey = key;
-    }
-
-    @Override
-    public Route<?> route()
-    {
-        return route;
-    }
-
-    @Override
-    protected void setRoute(Route<?> route)
-    {
-        this.route = Invariants.checkArgument(route, !route.isEmpty());
-    }
-
-    @Override
-    public PartialTxn partialTxn()
-    {
-        return partialTxn;
-    }
-
-    @Override
-    protected void setPartialTxn(PartialTxn txn)
-    {
-        this.partialTxn = txn;
-    }
-
-    @Override
-    public Ballot promised()
-    {
-        return promised;
-    }
-
-    @Override
-    public void setPromised(Ballot ballot)
-    {
-        this.promised = ballot;
-    }
-
-    @Override
-    public Ballot accepted()
-    {
-        return accepted;
-    }
-
-    @Override
-    public void setAccepted(Ballot ballot)
-    {
-        this.accepted = ballot;
-    }
-
-    @Override
-    public Timestamp executeAt()
-    {
-        return executeAt;
-    }
-
-    @Override
-    public void setExecuteAt(Timestamp timestamp)
-    {
-        this.executeAt = timestamp;
-    }
-
-    @Override
-    public @Nullable PartialDeps partialDeps()
-    {
-        return partialDeps;
-    }
-
-    @Override
-    public void setPartialDeps(@Nullable PartialDeps deps)
-    {
-        this.partialDeps = deps;
-    }
-
-    @Override
-    public Writes writes()
-    {
-        return writes;
-    }
-
-    @Override
-    public void setWrites(Writes writes)
-    {
-        this.writes = writes;
-    }
-
-    @Override
-    public Result result()
-    {
-        return result;
-    }
-
-    @Override
-    public void setResult(Result result)
-    {
-        this.result = result;
-    }
-
-    @Override
-    public SaveStatus saveStatus()
-    {
-        return status;
-    }
-
-    @Override
-    public void setSaveStatus(SaveStatus status)
-    {
-        this.status = status;
-    }
-
-    @Override
-    public Known known()
-    {
-        return status.known;
-    }
-
-    @Override
-    public Durability durability()
-    {
-        if (status.compareTo(SaveStatus.PreApplied) >= 0 && durability == NotDurable)
-            return Local; // not necessary anywhere, but helps for logical consistency
-        return durability;
-    }
-
-    @Override
-    public void setDurability(Durability v)
-    {
-        durability = v;
-    }
-
-    @Override
-    public Command addListener(CommandListener listener)
-    {
-        listeners.add(listener);
-        return this;
-    }
-
-    @Override
-    public void removeListener(CommandListener listener)
-    {
-        listeners.remove(listener);
-    }
-
-    @Override
-    public void notifyListeners(SafeCommandStore safeStore)
-    {
-        listeners.forEach(this, safeStore);
-    }
-
-    @Override
-    public void addWaitingOnCommit(TxnId txnId)
-    {
-        if (waitingOnCommit == null)
-            waitingOnCommit = new TreeSet<>();
-
-        waitingOnCommit.add(txnId);
-    }
-
-    @Override
-    public void removeWaitingOnCommit(TxnId txnId)
-    {
-        if (waitingOnCommit == null)
-            return;
-        waitingOnCommit.remove(txnId);
-    }
-
-    @Override
-    public TxnId firstWaitingOnCommit()
-    {
-        return isWaitingOnCommit() ? waitingOnCommit.first() : null;
-    }
-
-    @Override
-    public void addWaitingOnApplyIfAbsent(TxnId txnId, Timestamp executeAt)
-    {
-        if (waitingOnApply == null)
-            waitingOnApply = new TreeMap<>();
-
-        waitingOnApply.put(executeAt, txnId);
-    }
-
-    public boolean isWaitingOnApply()
-    {
-        return waitingOnApply != null && !waitingOnApply.isEmpty();
-    }
-
-    public boolean isWaitingOnCommit()
-    {
-        return waitingOnCommit != null && !waitingOnCommit.isEmpty();
-    }
-
-    @Override
-    public boolean isWaitingOnDependency()
-    {
-        return isWaitingOnCommit() || isWaitingOnApply();
-    }
-
-    @Override
-    public void removeWaitingOn(TxnId txnId, Timestamp executeAt)
-    {
-        if (waitingOnCommit != null)
-            waitingOnCommit.remove(txnId);
-
-        if (waitingOnApply != null)
-            waitingOnApply.remove(executeAt);
-    }
-
-    @Override
-    public TxnId firstWaitingOnApply(@Nullable TxnId ifExecutesBefore)
-    {
-        if (!isWaitingOnApply())
-            return null;
-
-        Map.Entry<Timestamp, TxnId> first = waitingOnApply.firstEntry();
-        if (ifExecutesBefore == null || first.getKey().compareTo(ifExecutesBefore) < 0)
-            return first.getValue();
-
-        return null;
-    }
-}
diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
index 60c51f3..e881b91 100644
--- a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
+++ b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
@@ -18,28 +18,22 @@
 
 package accord.impl;
 
+import accord.api.*;
 import accord.local.*;
-import accord.local.SyncCommandStores.SyncCommandStore; // java8 fails compilation if this is in correct position
-import accord.api.Agent;
-import accord.api.DataStore;
-import accord.api.Key;
-import accord.api.ProgressLog;
-import accord.impl.InMemoryCommandStore.SingleThread.AsyncState;
-import accord.impl.InMemoryCommandStore.Synchronized.SynchronizedState;
 import accord.local.CommandStores.RangesForEpochHolder;
 import accord.local.CommandStores.RangesForEpoch;
-import accord.impl.CommandsForKey.CommandTimeseries;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 import accord.primitives.*;
 import accord.utils.Invariants;
 import accord.utils.async.AsyncChain;
 import accord.utils.async.AsyncChains;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
 import java.util.*;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
+import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
 import java.util.function.BiFunction;
@@ -48,102 +42,526 @@
 
 import static accord.local.SafeCommandStore.TestDep.*;
 import static accord.local.SafeCommandStore.TestKind.Ws;
-import static accord.local.Status.Committed;
+import static accord.local.Status.*;
 import static accord.primitives.Routables.Slice.Minimal;
 
-public class InMemoryCommandStore
+public abstract class InMemoryCommandStore implements CommandStore
 {
-    public static abstract class State implements SafeCommandStore
+    private static final Logger logger = LoggerFactory.getLogger(InMemoryCommandStore.class);
+
+    private final int id;
+    private final NodeTimeService time;
+    private final Agent agent;
+    private final DataStore store;
+    private final ProgressLog progressLog;
+    private final RangesForEpochHolder rangesForEpochHolder;
+
+    private final NavigableMap<TxnId, GlobalCommand> commands = new TreeMap<>();
+    private final NavigableMap<RoutableKey, GlobalCommandsForKey> commandsForKey = new TreeMap<>();
+    private final CFKLoader cfkLoader = new CFKLoader();
+    // TODO (find library, efficiency): this is obviously super inefficient, need some range map
+
+    private final TreeMap<TxnId, RangeCommand> rangeCommands = new TreeMap<>();
+
+    private InMemorySafeStore current;
+
+    public InMemoryCommandStore(int id, NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, RangesForEpochHolder rangesForEpochHolder)
     {
-        private final NodeTimeService time;
-        private final Agent agent;
-        private final DataStore store;
-        private final ProgressLog progressLog;
-        private final RangesForEpochHolder rangesForEpochHolder;
-        private RangesForEpoch rangesForEpoch;
+        this.id = id;
+        this.time = time;
+        this.agent = agent;
+        this.store = store;
+        this.progressLog = progressLogFactory.create(this);
+        this.rangesForEpochHolder = rangesForEpochHolder;
+    }
 
-        private final CommandStore commandStore;
-        private final NavigableMap<TxnId, Command> commands = new TreeMap<>();
-        private final NavigableMap<RoutableKey, InMemoryCommandsForKey> commandsForKey = new TreeMap<>();
-        // TODO (find library, efficiency): this is obviously super inefficient, need some range map
-        private final TreeMap<TxnId, RangeCommand> rangeCommands = new TreeMap<>();
+    @Override
+    public int id()
+    {
+        return id;
+    }
 
-        static class RangeCommand
+    @Override
+    public Agent agent()
+    {
+        return agent;
+    }
+
+    public GlobalCommand ifPresent(TxnId txnId)
+    {
+        return commands.get(txnId);
+    }
+
+    public GlobalCommand command(TxnId txnId)
+    {
+        return commands.computeIfAbsent(txnId, GlobalCommand::new);
+    }
+
+    public boolean hasCommand(TxnId txnId)
+    {
+        return commands.containsKey(txnId);
+    }
+
+    public GlobalCommandsForKey ifPresent(Key key)
+    {
+        return commandsForKey.get(key);
+    }
+
+    public GlobalCommandsForKey commandsForKey(Key key)
+    {
+        return commandsForKey.computeIfAbsent(key, GlobalCommandsForKey::new);
+    }
+
+    public boolean hasCommandsForKey(Key key)
+    {
+        return commandsForKey.containsKey(key);
+    }
+
+    public CFKLoader cfkLoader()
+    {
+        return cfkLoader;
+    }
+
+    public void forEpochCommands(Ranges ranges, long epoch, Consumer<Command> consumer)
+    {
+        Timestamp minTimestamp = Timestamp.minForEpoch(epoch);
+        Timestamp maxTimestamp = Timestamp.maxForEpoch(epoch);
+        for (Range range : ranges)
         {
-            final Command command;
-            Ranges ranges;
+            Iterable<GlobalCommandsForKey> keyCommands = commandsForKey.subMap(range.start(), range.startInclusive(),
+                                                                                 range.end(), range.endInclusive()).values();
 
-            RangeCommand(Command command)
+            for (GlobalCommandsForKey commands : keyCommands)
             {
-                this.command = command;
+                if (commands.isEmpty())
+                    continue;
+                commands.value().forWitnessed(minTimestamp, maxTimestamp, txnId -> consumer.accept(command(txnId).value()));
             }
 
-            void update(Ranges add)
+            rangeCommands.forEach((txnId, rangeCommand) -> {
+                if (!rangeCommand.ranges.intersects(range))
+                    return;
+                Command command = rangeCommand.command.value();
+                Invariants.nonNull(command);
+                if (!command.hasBeen(PreAccepted))
+                    return;
+
+                Timestamp timestamp = command.hasBeen(PreCommitted) ? command.executeAt() : txnId;
+                if (timestamp.compareTo(maxTimestamp) > 0)
+                    return;
+                if (timestamp.compareTo(minTimestamp) < 0)
+                    return;
+                consumer.accept(command);
+            });
+        }
+    }
+
+    public void forCommittedInEpoch(Ranges ranges, long epoch, Consumer<Command> consumer)
+    {
+        Timestamp minTimestamp = Timestamp.minForEpoch(epoch);
+        Timestamp maxTimestamp = Timestamp.maxForEpoch(epoch);
+        for (Range range : ranges)
+        {
+            Iterable<GlobalCommandsForKey> keyCommands = commandsForKey.subMap(range.start(),
+                                                                                 range.startInclusive(),
+                                                                                 range.end(),
+                                                                                 range.endInclusive()).values();
+            for (GlobalCommandsForKey commands : keyCommands)
             {
-                if (ranges == null) ranges = add;
-                else ranges = ranges.with(add);
+                if (commands.isEmpty())
+                    continue;
+                commands.value().byExecuteAt()
+                        .between(minTimestamp, maxTimestamp, status -> status.hasBeen(Committed))
+                        .forEach(txnId -> consumer.accept(command(txnId).value()));
+            }
+
+            rangeCommands.forEach((txnId, rangeCommand) -> {
+                if (!rangeCommand.ranges.intersects(range))
+                    return;
+                Command command = rangeCommand.command.value();
+                if (command == null || !command.hasBeen(Committed))
+                    return;
+                Timestamp executeAt = command.executeAt();
+                if (executeAt.compareTo(maxTimestamp) > 0)
+                    return;
+                if (executeAt.compareTo(minTimestamp) < 0)
+                    return;
+                consumer.accept(command);
+            });
+        }
+    }
+
+    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 -> {
+                    SafeCommandsForKey cfk = safeStore.commandsForKey(key);
+                    CommandListener listener = cfk.register(command.current()).asListener();
+                    mutable.addListener(listener);
+                });
+                return mutable;
+            case Range:
+                rangeCommands.computeIfAbsent(command.txnId(), ignore -> new RangeCommand(commands.get(command.txnId())))
+                        .update((Ranges)keysOrRanges);
+        }
+        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 -> {
+                    SafeCommandsForKey cfk = safeStore.commandsForKey(key);
+                    CommandListener listener = cfk.register(command.current()).asListener();
+                    mutable.addListener(listener);
+                });
+                return mutable;
+            case Range:
+                rangeCommands.computeIfAbsent(command.txnId(), ignore -> new RangeCommand(commands.get(command.txnId())))
+                        .update(Ranges.of((Range)keyOrRange));
+        }
+        return attrs;
+    }
+
+    private <O> O mapReduceForKey(InMemorySafeStore safeStore, Routables<?, ?> keysOrRanges, Ranges slice, BiFunction<CommandsForKey, O, O> map, O accumulate, O terminalValue)
+    {
+        switch (keysOrRanges.domain()) {
+            default:
+                throw new AssertionError();
+            case Key:
+                AbstractKeys<Key, ?> keys = (AbstractKeys<Key, ?>) keysOrRanges;
+                for (Key key : keys)
+                {
+                    if (!slice.contains(key)) continue;
+                    SafeCommandsForKey forKey = safeStore.ifLoaded(key);
+                    if (forKey.current() == null)
+                        continue;
+                    accumulate = map.apply(forKey.current(), accumulate);
+                    if (accumulate.equals(terminalValue))
+                        return accumulate;
+                }
+                break;
+            case Range:
+                Ranges ranges = (Ranges) keysOrRanges;
+                Ranges sliced = ranges.slice(slice, Minimal);
+                for (Range range : sliced)
+                {
+                    for (GlobalCommandsForKey forKey : commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive()).values())
+                    {
+                        if (forKey.value() == null)
+                            continue;
+                        accumulate = map.apply(forKey.value(), accumulate);
+                        if (accumulate.equals(terminalValue))
+                            return accumulate;
+                    }
+                }
+        }
+        return accumulate;
+    }
+
+    private void forEach(Seekables<?, ?> keysOrRanges, Ranges slice, Consumer<RoutableKey> forEach)
+    {
+        switch (keysOrRanges.domain()) {
+            default:
+                throw new AssertionError();
+            case Key:
+                AbstractKeys<Key, ?> keys = (AbstractKeys<Key, ?>) keysOrRanges;
+                keys.forEach(slice, key -> forEach.accept(key));
+                break;
+            case Range:
+                Ranges ranges = (Ranges) keysOrRanges;
+                ranges.slice(slice).forEach(range -> {
+                    commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive())
+                            .keySet().forEach(forEach);
+                });
+        }
+    }
+
+    private void forEach(Routable keyOrRange, Ranges slice, Consumer<RoutableKey> forEach)
+    {
+        switch (keyOrRange.domain())
+        {
+            default: throw new AssertionError();
+            case Key:
+                Key key = (Key) keyOrRange;
+                if (slice.contains(key))
+                    forEach.accept(key);
+                break;
+            case Range:
+                Range range = (Range) keyOrRange;
+                Ranges.of(range).slice(slice).forEach(r -> {
+                    commandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive())
+                            .keySet().forEach(forEach);
+                });
+        }
+    }
+
+
+    protected InMemorySafeStore createCommandStore(PreLoadContext context, RangesForEpoch ranges, Map<TxnId, InMemorySafeCommand> commands, Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKeys)
+    {
+        return new InMemorySafeStore(this, cfkLoader, ranges, context, commands, commandsForKeys);
+    }
+
+    protected final InMemorySafeStore createCommandStore(PreLoadContext context, RangesForEpoch ranges)
+    {
+        Map<TxnId, InMemorySafeCommand> commands = new HashMap<>();
+        Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKeys = new HashMap<>();
+        for (TxnId txnId : context.txnIds())
+            commands.put(txnId, command(txnId).createSafeReference());
+
+        for (Seekable seekable : context.keys())
+        {
+            switch (seekable.domain())
+            {
+                case Key:
+                    RoutableKey key = (RoutableKey) seekable;
+                    commandsForKeys.put(key, commandsForKey((Key) key).createSafeReference());
+                    break;
+                case Range:
+                    // load range cfks here
             }
         }
+        return createCommandStore(context, ranges, commands, commandsForKeys);
+    }
 
-        public State(NodeTimeService time, Agent agent, DataStore store, ProgressLog progressLog, RangesForEpochHolder rangesForEpoch, CommandStore commandStore)
+    public SafeCommandStore beginOperation(PreLoadContext context)
+    {
+        if (current != null)
+            throw new IllegalStateException("Another operation is in progress or it's store was not cleared");
+        current = createCommandStore(context, rangesForEpochHolder.get());
+        return current;
+    }
+
+    public void completeOperation(SafeCommandStore store)
+    {
+        if (store != current)
+            throw new IllegalStateException("This operation has already been cleared");
+        try
         {
-            this.time = time;
-            this.agent = agent;
-            this.store = store;
-            this.progressLog = progressLog;
-            this.rangesForEpochHolder = rangesForEpoch;
+            current.complete();
+        }
+        catch (Throwable t)
+        {
+            logger.error("Exception completing operation", t);
+            throw t;
+        }
+        finally
+        {
+            current = null;
+        }
+    }
+
+    private <T> T executeInContext(InMemoryCommandStore commandStore, PreLoadContext preLoadContext, Function<? super SafeCommandStore, T> function, boolean isDirectCall)
+    {
+
+        SafeCommandStore safeStore = commandStore.beginOperation(preLoadContext);
+        try
+        {
+            return function.apply(safeStore);
+        }
+        catch (Throwable t)
+        {
+            if (isDirectCall) logger.error("Uncaught exception", t);
+            throw t;
+        }
+        finally
+        {
+            commandStore.completeOperation(safeStore);
+        }
+    }
+
+    protected <T> T executeInContext(InMemoryCommandStore commandStore, PreLoadContext context, Function<? super SafeCommandStore, T> function)
+    {
+        return executeInContext(commandStore, context, function, true);
+
+    }
+
+    protected <T> void executeInContext(InMemoryCommandStore commandStore, PreLoadContext context, Function<? super SafeCommandStore, T> function, BiConsumer<? super T, Throwable> callback)
+    {
+        try
+        {
+            T result = executeInContext(commandStore, context, function, false);
+            callback.accept(result, null);
+        }
+        catch (Throwable t)
+        {
+            logger.error("Uncaught exception", t);
+            callback.accept(null, t);
+        }
+    }
+
+    class RangeCommand
+    {
+        final GlobalCommand command;
+        Ranges ranges;
+
+        RangeCommand(GlobalCommand command)
+        {
+            this.command = command;
+        }
+
+        void update(Ranges add)
+        {
+            if (ranges == null) ranges = add;
+            else ranges = ranges.with(add);
+        }
+    }
+
+    class CFKLoader implements CommandsForKey.CommandLoader<TxnId>
+    {
+        private Command loadForCFK(TxnId data)
+        {
+            GlobalCommand globalCommand = ifPresent(data);
+            if (globalCommand != null)
+                return globalCommand.value();
+            throw new IllegalStateException("Could not find command for CFK for " + data);
+        }
+
+        @Override
+        public TxnId txnId(TxnId txnId)
+        {
+            return loadForCFK(txnId).txnId();
+        }
+
+        @Override
+        public Timestamp executeAt(TxnId txnId)
+        {
+            return loadForCFK(txnId).executeAt();
+        }
+
+        @Override
+        public SaveStatus saveStatus(TxnId txnId)
+        {
+            return loadForCFK(txnId).saveStatus();
+        }
+
+        @Override
+        public List<TxnId> depsIds(TxnId data)
+        {
+            PartialDeps deps = loadForCFK(data).partialDeps();
+            return deps != null ? deps.txnIds() : Collections.emptyList();
+        }
+
+        @Override
+        public TxnId saveForCFK(Command command)
+        {
+            return command.txnId();
+        }
+    }
+
+    public static abstract class GlobalState<V>
+    {
+        private V value;
+
+        public V value()
+        {
+            return value;
+        }
+
+        boolean isEmpty()
+        {
+            return value == null;
+        }
+
+        public GlobalState<V> value(V value)
+        {
+            this.value = value;
+            return this;
+        }
+    }
+
+    public static class GlobalCommand extends GlobalState<Command>
+    {
+        private final TxnId txnId;
+
+        public GlobalCommand(TxnId txnId)
+        {
+            this.txnId = txnId;
+        }
+
+        public InMemorySafeCommand createSafeReference()
+        {
+            return new InMemorySafeCommand(txnId, this);
+        }
+    }
+
+    public static class GlobalCommandsForKey extends GlobalState<CommandsForKey>
+    {
+        private final Key key;
+
+        public GlobalCommandsForKey(RoutableKey key)
+        {
+            this.key = (Key) key;
+        }
+
+        public InMemorySafeCommandsForKey createSafeReference()
+        {
+            return new InMemorySafeCommandsForKey(key, this);
+        }
+    }
+
+    static class InMemorySafeStore extends AbstractSafeCommandStore<InMemorySafeCommand, InMemorySafeCommandsForKey>
+    {
+        private final InMemoryCommandStore commandStore;
+        private final Map<TxnId, InMemorySafeCommand> commands;
+        private final Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey;
+        private final RangesForEpoch ranges;
+        private final CFKLoader cfkLoader;
+
+        public InMemorySafeStore(InMemoryCommandStore commandStore, CFKLoader cfkLoader, RangesForEpoch ranges, PreLoadContext context, Map<TxnId, InMemorySafeCommand> commands, Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey)
+        {
+            super(context);
             this.commandStore = commandStore;
+            this.commands = commands;
+            this.commandsForKey = commandsForKey;
+            this.ranges = Invariants.nonNull(ranges);
+            this.cfkLoader = cfkLoader;
         }
 
         @Override
-        public Command ifPresent(TxnId txnId)
-        {
-            return commands.get(txnId);
-        }
-
-        // TODO (required): mimic caching to test C* behaviour
-        @Override
-        public Command ifLoaded(TxnId txnId)
+        protected InMemorySafeCommand getCommandInternal(TxnId txnId)
         {
             return commands.get(txnId);
         }
 
         @Override
-        public Command command(TxnId txnId)
+        protected void addCommandInternal(InMemorySafeCommand command)
         {
-            return commands.computeIfAbsent(txnId, id -> new InMemoryCommand(commandStore, id));
+            commands.put(command.txnId(), command);
         }
 
-        public boolean hasCommand(TxnId txnId)
-        {
-            return commands.containsKey(txnId);
-        }
-
-        public CommandsForKey commandsForKey(Key key)
-        {
-            return commandsForKey.computeIfAbsent(key, k -> new InMemoryCommandsForKey((Key) k));
-        }
-
-        public boolean hasCommandsForKey(Key key)
-        {
-            return commandsForKey.containsKey(key);
-        }
-
-        public CommandsForKey maybeCommandsForKey(Key key)
+        @Override
+        protected InMemorySafeCommandsForKey getCommandsForKeyInternal(RoutableKey key)
         {
             return commandsForKey.get(key);
         }
 
         @Override
-        public void addAndInvokeListener(TxnId txnId, CommandListener listener)
+        protected void addCommandsForKeyInternal(InMemorySafeCommandsForKey cfk)
         {
-            command(txnId).addListener(listener);
+            commandsForKey.put(cfk.key(), (InMemorySafeCommandsForKey) cfk);
         }
 
         @Override
-        public DataStore dataStore()
+        protected InMemorySafeCommand getIfLoaded(TxnId txnId)
         {
-            return store;
+            GlobalCommand global = commandStore.ifPresent(txnId);
+            return global != null ? global.createSafeReference() : null;
+        }
+
+        @Override
+        protected InMemorySafeCommandsForKey getIfLoaded(RoutableKey key)
+        {
+            GlobalCommandsForKey global = commandStore.ifPresent((Key) key);
+            return global != null ? global.createSafeReference() : null;
         }
 
         @Override
@@ -153,107 +571,60 @@
         }
 
         @Override
+        public DataStore dataStore()
+        {
+            return commandStore.store;
+        }
+
+        @Override
         public Agent agent()
         {
-            return agent;
+            return commandStore.agent;
         }
 
         @Override
         public ProgressLog progressLog()
         {
-            return progressLog;
+            return commandStore.progressLog;
         }
 
         @Override
         public RangesForEpoch ranges()
         {
-            Invariants.checkState(rangesForEpoch != null);
-            return rangesForEpoch;
+            return ranges;
         }
 
         @Override
         public long latestEpoch()
         {
-            return time.epoch();
+            return commandStore.time.epoch();
+
         }
 
         @Override
-        public Timestamp preaccept(TxnId txnId, Seekables<?, ?> keys)
+        public Timestamp maxConflict(Seekables<?, ?> keysOrRanges, Ranges slice)
         {
-            Timestamp max = maxConflict(keys, ranges().at(txnId.epoch()));
-            long epoch = latestEpoch();
-            if (txnId.compareTo(max) > 0 && txnId.epoch() >= epoch && !agent.isExpired(txnId, time.now()))
-                return txnId;
-
-            return time.uniqueNow(max);
-        }
-
-        void refreshRanges()
-        {
-            rangesForEpoch = rangesForEpochHolder.get();
+            Timestamp timestamp = commandStore.mapReduceForKey(this, keysOrRanges, slice, (forKey, prev) -> Timestamp.max(forKey.max(), prev), Timestamp.NONE, null);
+            Seekables<?, ?> sliced = keysOrRanges.slice(slice, Minimal);
+            for (RangeCommand command : commandStore.rangeCommands.values())
+            {
+                if (command.ranges.intersects(sliced))
+                    timestamp = Timestamp.max(timestamp, command.command.value().executeAt());
+            }
+            return timestamp;
         }
 
         @Override
         public NodeTimeService time()
         {
-            return time;
-        }
-
-        private Timestamp maxConflict(Seekables<?, ?> keysOrRanges, Ranges slice)
-        {
-            Timestamp timestamp = mapReduceForKey(keysOrRanges, slice, (forKey, prev) -> Timestamp.max(forKey.max(), prev), Timestamp.NONE, null);
-            Seekables<?, ?> sliced = keysOrRanges.slice(slice, Minimal);
-            for (RangeCommand command : rangeCommands.values())
-            {
-                if (command.ranges.intersects(sliced))
-                    timestamp = Timestamp.max(timestamp, command.command.executeAt());
-            }
-            return timestamp;
-        }
-
-        public void forEpochCommands(Ranges ranges, long epoch, Consumer<Command> consumer)
-        {
-            Timestamp minTimestamp = Timestamp.minForEpoch(epoch);
-            Timestamp maxTimestamp = Timestamp.maxForEpoch(epoch);
-            for (Range range : ranges)
-            {
-                Iterable<InMemoryCommandsForKey> rangeCommands = commandsForKey.subMap(
-                        range.start(), range.startInclusive(),
-                        range.end(), range.endInclusive()
-                ).values();
-
-                for (InMemoryCommandsForKey commands : rangeCommands)
-                {
-                    commands.forWitnessed(minTimestamp, maxTimestamp, consumer);
-                }
-            }
-        }
-
-        public void forCommittedInEpoch(Ranges ranges, long epoch, Consumer<Command> consumer)
-        {
-            Timestamp minTimestamp = Timestamp.minForEpoch(epoch);
-            Timestamp maxTimestamp = Timestamp.maxForEpoch(epoch);
-            for (Range range : ranges)
-            {
-                Iterable<InMemoryCommandsForKey> rangeCommands = commandsForKey.subMap(range.start(),
-                        range.startInclusive(),
-                        range.end(),
-                        range.endInclusive()).values();
-                for (InMemoryCommandsForKey commands : rangeCommands)
-                {
-                    commands.byExecuteAt()
-                            .between(minTimestamp, maxTimestamp)
-                            .filter(command -> command.hasBeen(Committed))
-                            .forEach(consumer);
-                }
-            }
+            return commandStore.time;
         }
 
         @Override
         public <T> T mapReduce(Seekables<?, ?> keysOrRanges, Ranges slice, TestKind testKind, TestTimestamp testTimestamp, Timestamp timestamp, TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus, CommandFunction<T, T> map, T accumulate, T terminalValue)
         {
-            accumulate = mapReduceForKey(keysOrRanges, slice, (forKey, prev) -> {
-                CommandTimeseries timeseries;
+            accumulate = commandStore.mapReduceForKey(this, keysOrRanges, slice, (forKey, prev) -> {
+                CommandsForKey.CommandTimeseries<?> timeseries;
                 switch (testTimestamp)
                 {
                     default: throw new AssertionError();
@@ -265,17 +636,17 @@
                     case MAY_EXECUTE_BEFORE:
                         timeseries = forKey.byExecuteAt();
                 }
-                CommandTimeseries.TestTimestamp remapTestTimestamp;
+                CommandsForKey.CommandTimeseries.TestTimestamp remapTestTimestamp;
                 switch (testTimestamp)
                 {
                     default: throw new AssertionError();
                     case STARTED_AFTER:
                     case EXECUTES_AFTER:
-                        remapTestTimestamp = CommandTimeseries.TestTimestamp.AFTER;
+                        remapTestTimestamp = CommandsForKey.CommandTimeseries.TestTimestamp.AFTER;
                         break;
                     case STARTED_BEFORE:
                     case MAY_EXECUTE_BEFORE:
-                        remapTestTimestamp = CommandTimeseries.TestTimestamp.BEFORE;
+                        remapTestTimestamp = CommandsForKey.CommandTimeseries.TestTimestamp.BEFORE;
                 }
                 return timeseries.mapReduce(testKind, remapTestTimestamp, timestamp, testDep, depId, minStatus, maxStatus, map, prev, terminalValue);
             }, accumulate, terminalValue);
@@ -286,414 +657,177 @@
             // TODO (find lib, efficiency): this is super inefficient, need to store Command in something queryable
             Seekables<?, ?> sliced = keysOrRanges.slice(slice, Minimal);
             Map<Range, List<Command>> collect = new TreeMap<>(Range::compare);
-            for (RangeCommand rangeCommand : rangeCommands.values())
-            {
-                Command command = rangeCommand.command;
+            commandStore.rangeCommands.forEach(((txnId, rangeCommand) -> {
+                Command command = rangeCommand.command.value();
+                Invariants.nonNull(command);
                 switch (testTimestamp)
                 {
                     default: throw new AssertionError();
                     case STARTED_AFTER:
-                        if (command.txnId().compareTo(timestamp) < 0) continue;
+                        if (command.txnId().compareTo(timestamp) < 0) return;
                         else break;
                     case STARTED_BEFORE:
-                        if (command.txnId().compareTo(timestamp) > 0) continue;
+                        if (command.txnId().compareTo(timestamp) > 0) return;
                         else break;
                     case EXECUTES_AFTER:
-                        if (command.executeAt().compareTo(timestamp) < 0) continue;
+                        if (command.executeAt().compareTo(timestamp) < 0) return;
                         else break;
                     case MAY_EXECUTE_BEFORE:
                         Timestamp compareTo = command.known().executeAt.hasDecidedExecuteAt() ? command.executeAt() : command.txnId();
-                        if (compareTo.compareTo(timestamp) > 0) continue;
+                        if (compareTo.compareTo(timestamp) > 0) return;
                         else break;
                 }
 
                 if (minStatus != null && command.status().compareTo(minStatus) < 0)
-                    continue;
+                    return;
 
                 if (maxStatus != null && command.status().compareTo(maxStatus) > 0)
-                    continue;
+                    return;
 
                 if (testKind == Ws && command.txnId().rw().isRead())
-                    continue;
+                    return;
 
                 if (testDep != ANY_DEPS)
                 {
                     if (!command.known().deps.hasProposedOrDecidedDeps())
-                        continue;
+                        return;
 
                     if ((testDep == WITH) == !command.partialDeps().contains(depId))
-                        continue;
+                        return;
                 }
 
                 if (!rangeCommand.ranges.intersects(sliced))
-                    continue;
+                    return;
 
                 Routables.foldl(rangeCommand.ranges, sliced, (r, in, i) -> {
                     // TODO (easy, efficiency): pass command as a parameter to Fold
                     List<Command> list = in.computeIfAbsent(r, ignore -> new ArrayList<>());
                     if (list.isEmpty() || list.get(list.size() - 1) != command)
-                            list.add(command);
+                        list.add(command);
                     return in;
                 }, collect);
-            }
-
+            }));
             for (Map.Entry<Range, List<Command>> e : collect.entrySet())
             {
                 for (Command command : e.getValue())
-                    accumulate = map.apply(e.getKey(), command.txnId(), command.executeAt(), accumulate);
+                {
+                    T initial = accumulate;
+                    accumulate = map.apply(e.getKey(), command.txnId(), command.executeAt(), initial);
+                }
             }
 
             return accumulate;
         }
 
         @Override
-        public void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command)
+        public CommonAttributes completeRegistration(Seekables<?, ?> keysOrRanges, Ranges slice, InMemorySafeCommand command, CommonAttributes attrs)
         {
-            switch (keysOrRanges.domain())
-            {
-                default: throw new AssertionError();
-                case Key:
-                    forEach(keysOrRanges, slice, forKey -> forKey.register(command));
-                    break;
-                case Range:
-                    rangeCommands.computeIfAbsent(command.txnId(), ignore -> new RangeCommand(command))
-                            .update((Ranges)keysOrRanges);
-            }
+            return commandStore.register(this, keysOrRanges, slice, command, attrs);
         }
 
         @Override
-        public void register(Seekable keyOrRange, Ranges slice, Command command)
+        public CommonAttributes completeRegistration(Seekable keyOrRange, Ranges slice, InMemorySafeCommand command, CommonAttributes attrs)
         {
-            switch (keyOrRange.domain())
-            {
-                default: throw new AssertionError();
-                case Key:
-                    forEach(keyOrRange, slice, forKey -> forKey.register(command));
-                    break;
-                case Range:
-                    rangeCommands.computeIfAbsent(command.txnId(), ignore -> new RangeCommand(command))
-                            .update(Ranges.of((Range)keyOrRange));
-            }
+            return commandStore.register(this, keyOrRange, slice, command, attrs);
         }
 
-        private <O> O mapReduceForKey(Routables<?, ?> keysOrRanges, Ranges slice, BiFunction<CommandsForKey, O, O> map, O accumulate, O terminalValue)
+        public CommandsForKey.CommandLoader<?> cfkLoader()
         {
-            switch (keysOrRanges.domain()) {
-                default:
-                    throw new AssertionError();
-                case Key:
-                    AbstractKeys<Key, ?> keys = (AbstractKeys<Key, ?>) keysOrRanges;
-                    for (Key key : keys)
-                    {
-                        if (!slice.contains(key)) continue;
-                        CommandsForKey forKey = commandsForKey(key);
-                        accumulate = map.apply(forKey, accumulate);
-                        if (accumulate.equals(terminalValue))
-                            return accumulate;
-                    }
-                    break;
-                case Range:
-                    Ranges ranges = (Ranges) keysOrRanges;
-                    Ranges sliced = ranges.slice(slice, Minimal);
-                    for (Range range : sliced)
-                    {
-                        for (CommandsForKey forKey : commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive()).values())
-                        {
-                            accumulate = map.apply(forKey, accumulate);
-                            if (accumulate.equals(terminalValue))
-                                return accumulate;
-                        }
-                    }
-            }
-            return accumulate;
+            return cfkLoader;
         }
 
-        private void forEach(Seekables<?, ?> keysOrRanges, Ranges slice, Consumer<CommandsForKey> forEach)
+        @Override
+        protected void invalidateSafeState()
         {
-            switch (keysOrRanges.domain()) {
-                default:
-                    throw new AssertionError();
-                case Key:
-                    AbstractKeys<Key, ?> keys = (AbstractKeys<Key, ?>) keysOrRanges;
-                    keys.forEach(slice, key -> forEach.accept(commandsForKey(key)));
-                    break;
-                case Range:
-                    Ranges ranges = (Ranges) keysOrRanges;
-                    ranges.slice(slice).forEach(range -> {
-                        commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive())
-                                .values().forEach(forEach);
-                    });
-            }
+            commands.values().forEach(SafeState::invalidate);
+            commandsForKey.values().forEach(SafeState::invalidate);
         }
 
-        private void forEach(Routable keyOrRange, Ranges slice, Consumer<CommandsForKey> forEach)
+        @Override
+        public void complete()
         {
-            switch (keyOrRange.domain())
-            {
-                default: throw new AssertionError();
-                case Key:
-                    Key key = (Key) keyOrRange;
-                    if (slice.contains(key))
-                        forEach.accept(commandsForKey(key));
-                    break;
-                case Range:
-                    Range range = (Range) keyOrRange;
-                    Ranges.of(range).slice(slice).forEach(r -> {
-                        commandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive())
-                                .values().forEach(forEach);
-                    });
-            }
+            postExecute();
+            super.complete();
         }
     }
 
-    public static class Synchronized extends SyncCommandStore
+    public static class Synchronized extends InMemoryCommandStore
     {
-        public static class SynchronizedState extends State implements SyncCommandStores.SafeSyncCommandStore
+        Runnable active = null;
+        final Queue<Runnable> queue = new ConcurrentLinkedQueue<>();
+
+        public Synchronized(int id, NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, RangesForEpochHolder rangesForEpochHolder)
         {
-            public SynchronizedState(NodeTimeService time, Agent agent, DataStore store, ProgressLog progressLog, RangesForEpochHolder rangesForEpoch, CommandStore commandStore)
-            {
-                super(time, agent, store, progressLog, rangesForEpoch, commandStore);
-            }
+            super(id, time, agent, store, progressLogFactory, rangesForEpochHolder);
+        }
 
-            @Override
-            public AsyncChain<Void> execute(PreLoadContext context, Consumer<? super SafeCommandStore> consumer)
-            {
-                return submit(context, i -> { consumer.accept(i); return null; });
-            }
+        private synchronized void maybeRun()
+        {
+            if (active != null)
+                return;
 
-            @Override
-            public synchronized <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> function)
+            active = queue.poll();
+            while (active != null)
             {
-                return new AsyncChains.Head<T>()
+                try
                 {
-                    @Override
-                    public void begin(BiConsumer<? super T, Throwable> callback)
-                    {
-                        T result = null;
-                        Throwable failure = null;
-                        synchronized (SynchronizedState.this)
-                        {
-                            try
-                            {
-                                result = function.apply(SynchronizedState.this);
-                            }
-                            catch (Throwable t)
-                            {
-                                failure = t;
-                            }
-                        }
-                        callback.accept(result, failure);
-                    }
-                };
-            }
-
-            public synchronized <T> T executeSync(PreLoadContext context, Function<? super SafeCommandStore, T> function)
-            {
-                return function.apply(this);
+                    active.run();
+                }
+                catch (Throwable t)
+                {
+                    logger.error("Uncaught exception", t);
+                }
+                active = queue.poll();
             }
         }
 
-        final SynchronizedState state;
-
-        public Synchronized(int id, NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, RangesForEpochHolder rangesForEpoch)
+        private void enqueueAndRun(Runnable runnable)
         {
-            super(id);
-            this.state = new SynchronizedState(time, agent, store, progressLogFactory.create(this), rangesForEpoch, this);
-        }
-
-        @Override
-        public Agent agent()
-        {
-            return state.agent();
-        }
-
-        private SynchronizedState safeStore()
-        {
-            state.refreshRanges();
-            return state;
+            boolean result = queue.add(runnable);
+            if (!result)
+                throw new IllegalStateException("could not add item to queue");
+            maybeRun();
         }
 
         @Override
         public AsyncChain<Void> execute(PreLoadContext context, Consumer<? super SafeCommandStore> consumer)
         {
-            return safeStore().execute(context, consumer);
+            return submit(context, i -> { consumer.accept(i); return null; });
         }
 
         @Override
         public <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> function)
         {
-            return safeStore().submit(context, function);
-        }
-
-        @Override
-        public <T> T executeSync(PreLoadContext context, Function<? super SafeCommandStore, T> function)
-        {
-            return safeStore().executeSync(context, function);
+            return new AsyncChains.Head<T>()
+            {
+                @Override
+                protected void start(BiConsumer<? super T, Throwable> callback)
+                {
+                    enqueueAndRun(() -> executeInContext(InMemoryCommandStore.Synchronized.this, context, function, callback));
+                }
+            };
         }
 
         @Override
         public void shutdown() {}
     }
 
-    public static class SingleThread extends CommandStore
+    public static class SingleThread extends InMemoryCommandStore
     {
-        class AsyncState extends State implements SafeCommandStore
-        {
-            public AsyncState(NodeTimeService time, Agent agent, DataStore store, ProgressLog progressLog, RangesForEpochHolder rangesForEpoch, CommandStore commandStore)
-            {
-                super(time, agent, store, progressLog, rangesForEpoch, commandStore);
-            }
-
-            @Override
-            public AsyncChain<Void> execute(PreLoadContext context, Consumer<? super SafeCommandStore> consumer)
-            {
-                return submit(context, i -> { consumer.accept(i); return null; });
-            }
-
-            @Override
-            public <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> function)
-            {
-                return AsyncChains.ofCallable(executor, () -> function.apply(this));
-            }
-        }
+        private final AtomicReference<Thread> expectedThread = new AtomicReference<>();
         private final ExecutorService executor;
-        private final AsyncState state;
 
-        public SingleThread(int id, NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, RangesForEpochHolder rangesForEpoch)
+        public SingleThread(int id, NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, RangesForEpochHolder rangesForEpochHolder)
         {
-            super(id);
-            executor = Executors.newSingleThreadExecutor(r -> {
+            super(id, time, agent, store, progressLogFactory, rangesForEpochHolder);
+            this.executor = Executors.newSingleThreadExecutor(r -> {
                 Thread thread = new Thread(r);
                 thread.setName(CommandStore.class.getSimpleName() + '[' + time.id() + ']');
                 return thread;
             });
-            state = newState(time, agent, store, progressLogFactory, rangesForEpoch);
         }
 
-        AsyncState newState(NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, RangesForEpochHolder rangesForEpoch)
-        {
-            return new AsyncState(time, agent, store, progressLogFactory.create(this), rangesForEpoch, this);
-        }
-
-        @Override
-        public Agent agent()
-        {
-            return state.agent();
-        }
-
-        private State safeStore()
-        {
-            state.refreshRanges();
-            return state;
-        }
-
-        @Override
-        public AsyncChain<Void> execute(PreLoadContext context, Consumer<? super SafeCommandStore> consumer)
-        {
-            return safeStore().execute(context, consumer);
-        }
-
-        @Override
-        public <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> function)
-        {
-            return safeStore().submit(context, function);
-        }
-
-        @Override
-        public void shutdown()
-        {
-            executor.shutdown();
-        }
-    }
-
-    public static class Debug extends SingleThread
-    {
-        class DebugState extends AsyncState
-        {
-            public DebugState(NodeTimeService time, Agent agent, DataStore store, ProgressLog progressLog, RangesForEpochHolder rangesForEpoch, CommandStore commandStore)
-            {
-                super(time, agent, store, progressLog, rangesForEpoch, commandStore);
-            }
-
-            @Override
-            public Command ifPresent(TxnId txnId)
-            {
-                assertThread();
-                return super.ifPresent(txnId);
-            }
-
-            @Override
-            public Command ifLoaded(TxnId txnId)
-            {
-                assertThread();
-                return super.ifLoaded(txnId);
-            }
-
-            @Override
-            public Command command(TxnId txnId)
-            {
-                assertThread();
-                return super.command(txnId);
-            }
-
-            @Override
-            public boolean hasCommand(TxnId txnId)
-            {
-                assertThread();
-                return super.hasCommand(txnId);
-            }
-
-            @Override
-            public CommandsForKey commandsForKey(Key key)
-            {
-                assertThread();
-                return super.commandsForKey(key);
-            }
-
-            @Override
-            public boolean hasCommandsForKey(Key key)
-            {
-                assertThread();
-                return super.hasCommandsForKey(key);
-            }
-
-            @Override
-            public CommandsForKey maybeCommandsForKey(Key key)
-            {
-                assertThread();
-                return super.maybeCommandsForKey(key);
-            }
-
-            @Override
-            public void addAndInvokeListener(TxnId txnId, CommandListener listener)
-            {
-                assertThread();
-                super.addAndInvokeListener(txnId, listener);
-            }
-
-            @Override
-            public void forEpochCommands(Ranges ranges, long epoch, Consumer<Command> consumer)
-            {
-                assertThread();
-                super.forEpochCommands(ranges, epoch, consumer);
-            }
-
-            @Override
-            public void forCommittedInEpoch(Ranges ranges, long epoch, Consumer<Command> consumer)
-            {
-                assertThread();
-                super.forCommittedInEpoch(ranges, epoch, consumer);
-            }
-        }
-
-        private final AtomicReference<Thread> expectedThread = new AtomicReference<>();
-
-        public Debug(int id, NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, RangesForEpochHolder rangesForEpoch)
-        {
-            super(id, time, agent, store, progressLogFactory, rangesForEpoch);
-        }
-
-        private void assertThread()
+        void assertThread()
         {
             Thread current = Thread.currentThread();
             Thread expected;
@@ -709,19 +843,180 @@
         }
 
         @Override
-        AsyncState newState(NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, RangesForEpochHolder rangesForEpoch)
+        public AsyncChain<Void> execute(PreLoadContext context, Consumer<? super SafeCommandStore> consumer)
         {
-            return new DebugState(time, agent, store, progressLogFactory.create(this), rangesForEpoch, this);
+            return submit(context, i -> { consumer.accept(i); return null; });
+        }
+
+        @Override
+        public <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> function)
+        {
+            return AsyncChains.ofCallable(executor, () -> executeInContext(this, context, function));
+        }
+
+        @Override
+        public void shutdown()
+        {
+            executor.shutdown();
         }
     }
 
-    public static State inMemory(CommandStore unsafeStore)
+    public static class Debug extends SingleThread
     {
-        return (unsafeStore instanceof Synchronized) ? ((Synchronized) unsafeStore).safeStore() : ((SingleThread) unsafeStore).safeStore();
+        class DebugSafeStore extends InMemorySafeStore
+        {
+            public DebugSafeStore(InMemoryCommandStore commandStore, CFKLoader cfkLoader, RangesForEpoch ranges, PreLoadContext context, Map<TxnId, InMemorySafeCommand> commands, Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey)
+            {
+                super(commandStore, cfkLoader, ranges, context, commands, commandsForKey);
+            }
+
+            @Override
+            public InMemorySafeCommand ifPresent(TxnId txnId)
+            {
+                assertThread();
+                return super.ifPresent(txnId);
+            }
+
+            @Override
+            public InMemorySafeCommand ifLoaded(TxnId txnId)
+            {
+                assertThread();
+                return super.ifLoaded(txnId);
+            }
+
+            @Override
+            public InMemorySafeCommand command(TxnId txnId)
+            {
+                assertThread();
+                return super.command(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, RangesForEpochHolder rangesForEpochHolder)
+        {
+            super(id, time, agent, store, progressLogFactory, rangesForEpochHolder);
+        }
+
+        @Override
+        protected InMemorySafeStore createCommandStore(PreLoadContext context, RangesForEpoch ranges, Map<TxnId, InMemorySafeCommand> commands, Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKeys)
+        {
+            return new DebugSafeStore(this, cfkLoader(), ranges, context, commands, commandsForKeys);
+        }
+
     }
 
-    public static State inMemory(SafeCommandStore safeStore)
+    public static InMemoryCommandStore inMemory(CommandStore unsafeStore)
     {
-        return (safeStore instanceof SynchronizedState) ? ((SynchronizedState) safeStore) : ((AsyncState) safeStore);
+        return (InMemoryCommandStore) unsafeStore;
+    }
+
+    public static InMemoryCommandStore inMemory(SafeCommandStore safeStore)
+    {
+        return inMemory(safeStore.commandStore());
+    }
+
+    /**
+     * methods useful for troubleshooting burn test failures. Shouldn't be used anywhere
+     */
+    public static class Utils
+    {
+        private static String prefix(int level, boolean verbose)
+        {
+            if (level == 0 || !verbose)
+                return "";
+
+            StringBuilder prefix = new StringBuilder();
+            for (int i=0; i<level; i++)
+                prefix.append("-> ");
+            prefix.append(' ');
+            return prefix.toString();
+        }
+
+        private static String suffix(boolean blockingOnCommit, boolean blockingOnApply)
+        {
+            if (blockingOnApply)
+                return " <Blocking On Apply>";
+            if (blockingOnCommit)
+                return " <Blocking On Commit>";
+            return "";
+        }
+
+        private static void log(String prefix, String suffix, String msg, Object... args)
+        {
+            logger.info(prefix + msg + suffix, args);
+        }
+
+        private static void log(String prefix, String suffix, Command command)
+        {
+            log(prefix, suffix, "{} {}", command.txnId(), command.saveStatus());
+        }
+
+        private static void logDependencyGraph(InMemoryCommandStore commandStore, TxnId txnId, Set<TxnId> visited, boolean verbose, int level, boolean blockingOnCommit, boolean blockingOnApply)
+        {
+            String prefix = prefix(level, verbose);
+            boolean previouslyVisited = !visited.add(txnId); // prevents infinite loops if command deps overlap
+            String suffix = suffix(blockingOnCommit, blockingOnApply);
+            if (previouslyVisited) suffix = suffix + " -- PREVIOUSLY VISITED";
+            GlobalCommand global = commandStore.commands.get(txnId);
+            if (global == null || global.isEmpty())
+            {
+                log(prefix, suffix, "{} NOT FOUND", txnId);
+                return;
+            }
+
+            Command command = global.value();
+            PartialDeps partialDeps = command.partialDeps();
+            List<TxnId> deps = partialDeps != null ? partialDeps.txnIds() : Collections.emptyList();
+            if (command.hasBeen(Committed))
+            {
+                Command.Committed committed = command.asCommitted();
+                if (level == 0 || verbose || !committed.isWaitingOnDependency())
+                    log(prefix, suffix, command);
+
+                Set<TxnId> waitingOnCommit = committed.waitingOnCommit();
+                Set<TxnId> waitingOnApply = new HashSet<>(committed.waitingOnApply().values());
+
+                if (committed.isWaitingOnDependency() && !previouslyVisited)
+                    deps.forEach(depId -> logDependencyGraph(commandStore, depId, visited, verbose, level+1, waitingOnCommit.contains(depId), waitingOnApply.contains(depId)));
+            }
+            else
+            {
+                log(prefix, suffix, command);
+                if (!previouslyVisited)
+                    deps.forEach(depId -> logDependencyGraph(commandStore, depId, visited, verbose, level+1, false, false));
+            }
+        }
+
+        public static void logDependencyGraph(CommandStore commandStore, TxnId txnId, boolean verbose)
+        {
+            logger.info("Logging dependencies on for {}, verbose: {}", txnId, verbose);
+            InMemoryCommandStore inMemoryCommandStore = (InMemoryCommandStore) commandStore;
+            logger.info("Node: {}, CommandStore #{}", inMemoryCommandStore.time.id(), commandStore.id());
+            Set<TxnId> visited = new HashSet<>();
+            logDependencyGraph(inMemoryCommandStore, txnId, visited, verbose, 0, false, false);
+        }
+
+        /**
+         * Recursively follows and prints dependencies starting from the given txnId. Useful in tracking down
+         * the root causes of hung burn tests
+         */
+        public static void logDependencyGraph(CommandStore commandStore, TxnId txnId)
+        {
+            logDependencyGraph(commandStore, txnId, true);
+        }
     }
 }
diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommandStores.java b/accord-core/src/main/java/accord/impl/InMemoryCommandStores.java
index 9ab4219..163318a 100644
--- a/accord-core/src/main/java/accord/impl/InMemoryCommandStores.java
+++ b/accord-core/src/main/java/accord/impl/InMemoryCommandStores.java
@@ -23,45 +23,18 @@
 import accord.api.DataStore;
 import accord.api.ProgressLog;
 import accord.local.CommandStore;
-import accord.primitives.Routables;
-import accord.utils.MapReduce;
-
-import java.util.function.BiFunction;
-import java.util.function.Function;
 
 public class InMemoryCommandStores
 {
-    public static class Synchronized extends SyncCommandStores
+    public static class Synchronized extends CommandStores
     {
         public Synchronized(NodeTimeService time, Agent agent, DataStore store, ShardDistributor shardDistributor, ProgressLog.Factory progressLogFactory)
         {
             super(time, agent, store, shardDistributor, progressLogFactory, InMemoryCommandStore.Synchronized::new);
         }
-
-        public <T> T mapReduce(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduce<? super SafeCommandStore, T> map)
-        {
-            return super.mapReduce(context, keys, minEpoch, maxEpoch, map, SyncMapReduceAdapter.instance());
-        }
-
-        public <T> T mapReduce(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, Function<? super SafeCommandStore, T> map, BiFunction<T, T, T> reduce)
-        {
-            return mapReduce(context, keys, minEpoch, maxEpoch, new MapReduce<SafeCommandStore, T>() {
-                @Override
-                public T apply(SafeCommandStore in)
-                {
-                    return map.apply(in);
-                }
-
-                @Override
-                public T reduce(T o1, T o2)
-                {
-                    return reduce.apply(o1, o2);
-                }
-            });
-        }
     }
 
-    public static class SingleThread extends AsyncCommandStores
+    public static class SingleThread extends CommandStores
     {
         public SingleThread(NodeTimeService time, Agent agent, DataStore store, ShardDistributor shardDistributor, ProgressLog.Factory progressLogFactory)
         {
diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommandsForKey.java b/accord-core/src/main/java/accord/impl/InMemoryCommandsForKey.java
deleted file mode 100644
index b2a1a65..0000000
--- a/accord-core/src/main/java/accord/impl/InMemoryCommandsForKey.java
+++ /dev/null
@@ -1,160 +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.local.Command;
-import accord.local.SafeCommandStore.CommandFunction;
-import accord.local.SafeCommandStore.TestDep;
-import accord.local.SafeCommandStore.TestKind;
-import accord.local.Status;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
-
-import javax.annotation.Nullable;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-import java.util.function.Consumer;
-import java.util.stream.Stream;
-
-import static accord.local.SafeCommandStore.TestDep.*;
-import static accord.local.SafeCommandStore.TestKind.Ws;
-import static accord.local.Status.KnownDeps.DepsUnknown;
-import static accord.local.Status.PreAccepted;
-import static accord.local.Status.PreCommitted;
-
-public class InMemoryCommandsForKey extends CommandsForKey
-{
-    public static class InMemoryCommandTimeseries implements CommandTimeseries
-    {
-        private final NavigableMap<Timestamp, Command> commands = new TreeMap<>();
-        private final Key key;
-
-        public InMemoryCommandTimeseries(Key key)
-        {
-            this.key = key;
-        }
-
-        @Override
-        public void add(Timestamp timestamp, Command command)
-        {
-            if (commands.containsKey(timestamp) && !commands.get(timestamp).equals(command))
-                throw new IllegalStateException(String.format("Attempting to overwrite command at timestamp %s %s with %s.",
-                                                              timestamp, commands.get(timestamp), command));
-
-            commands.put(timestamp, command);
-        }
-
-        @Override
-        public void remove(Timestamp timestamp)
-        {
-            commands.remove(timestamp);
-        }
-
-        @Override
-        public boolean isEmpty()
-        {
-            return commands.isEmpty();
-        }
-
-        @Override
-        public <T> T mapReduce(TestKind testKind, TestTimestamp testTimestamp, Timestamp timestamp,
-                               TestDep testDep, @Nullable TxnId depId,
-                               @Nullable Status minStatus, @Nullable Status maxStatus,
-                               CommandFunction<T, T> map, T initialValue, T terminalValue)
-        {
-
-            for (Command cmd : (testTimestamp == TestTimestamp.BEFORE ? commands.headMap(timestamp, false) : commands.tailMap(timestamp, false)).values())
-            {
-                if (testKind == Ws && cmd.txnId().isRead()) continue;
-                    // If we don't have any dependencies, we treat a dependency filter as a mismatch
-                if (testDep != ANY_DEPS && (!cmd.known().deps.hasProposedOrDecidedDeps() || (cmd.partialDeps().contains(depId) != (testDep == WITH))))
-                    continue;
-                if (minStatus != null && minStatus.compareTo(cmd.status()) > 0)
-                    continue;
-                if (maxStatus != null && maxStatus.compareTo(cmd.status()) < 0)
-                    continue;
-                initialValue = map.apply(key, cmd.txnId(), cmd.executeAt(), initialValue);
-                if (initialValue.equals(terminalValue))
-                    break;
-            }
-            return initialValue;
-        }
-
-        public Stream<Command> between(Timestamp min, Timestamp max)
-        {
-            return commands.subMap(min, true, max, true).values().stream();
-        }
-
-        public Stream<Command> all()
-        {
-            return commands.values().stream();
-        }
-    }
-
-    // TODO (now): add validation that anything inserted into *committedBy* has everything prior in its dependencies
-    private final Key key;
-    private final InMemoryCommandTimeseries byId;
-    private final InMemoryCommandTimeseries byExecuteAt;
-
-    private Timestamp max = Timestamp.NONE;
-
-    public InMemoryCommandsForKey(Key key)
-    {
-        this.key = key;
-        this.byId = new InMemoryCommandTimeseries(key);
-        this.byExecuteAt = new InMemoryCommandTimeseries(key);
-    }
-
-    @Override
-    public Key key()
-    {
-        return key;
-    }
-
-    @Override
-    public Timestamp max()
-    {
-        return max;
-    }
-
-    @Override
-    public void updateMax(Timestamp timestamp)
-    {
-        max = Timestamp.max(max, timestamp);
-    }
-
-    @Override
-    public InMemoryCommandTimeseries byId()
-    {
-        return byId;
-    }
-
-    @Override
-    public InMemoryCommandTimeseries byExecuteAt()
-    {
-        return byExecuteAt;
-    }
-
-    public void forWitnessed(Timestamp minTs, Timestamp maxTs, Consumer<Command> consumer)
-    {
-        byId.between(minTs, maxTs).filter(cmd -> cmd.hasBeen(PreAccepted)).forEach(consumer);
-        byExecuteAt.between(minTs, maxTs).filter(cmd -> cmd.hasBeen(PreCommitted)).forEach(consumer);
-    }
-}
diff --git a/accord-core/src/main/java/accord/impl/InMemorySafeCommand.java b/accord-core/src/main/java/accord/impl/InMemorySafeCommand.java
new file mode 100644
index 0000000..512f170
--- /dev/null
+++ b/accord-core/src/main/java/accord/impl/InMemorySafeCommand.java
@@ -0,0 +1,68 @@
+/*
+ * 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.InMemoryCommandStore.GlobalCommand;
+import accord.local.Command;
+import accord.local.SafeCommand;
+import accord.primitives.TxnId;
+
+public class InMemorySafeCommand extends SafeCommand implements SafeState<Command>
+{
+    private boolean invalidated;
+    private final GlobalCommand global;
+
+    public InMemorySafeCommand(TxnId txnId, GlobalCommand global)
+    {
+        super(txnId);
+        this.global = global;
+    }
+
+    @Override
+    public Command current()
+    {
+        checkNotInvalidated();
+        return global.value();
+    }
+
+    @Override
+    protected void set(Command update)
+    {
+        checkNotInvalidated();
+        global.value(update);
+    }
+
+    @Override
+    public void invalidate()
+    {
+        invalidated = true;
+    }
+
+    @Override
+    public boolean invalidated()
+    {
+        return invalidated;
+    }
+
+    private void checkNotInvalidated()
+    {
+        if (invalidated())
+            throw new IllegalStateException("Cannot access invalidated " + this);
+    }
+}
diff --git a/accord-core/src/main/java/accord/impl/InMemorySafeCommandsForKey.java b/accord-core/src/main/java/accord/impl/InMemorySafeCommandsForKey.java
new file mode 100644
index 0000000..7eebf62
--- /dev/null
+++ b/accord-core/src/main/java/accord/impl/InMemorySafeCommandsForKey.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.impl;
+
+import accord.api.Key;
+import accord.impl.InMemoryCommandStore.GlobalCommandsForKey;
+
+public class InMemorySafeCommandsForKey extends SafeCommandsForKey
+{
+    private boolean invalidated = false;
+    private final GlobalCommandsForKey global;
+
+    public InMemorySafeCommandsForKey(Key key, GlobalCommandsForKey global)
+    {
+        super(key);
+        this.global = global;
+    }
+
+    @Override
+    public CommandsForKey current()
+    {
+        return global.value();
+    }
+
+    @Override
+    protected void set(CommandsForKey update)
+    {
+        global.value(update);
+    }
+
+    @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
new file mode 100644
index 0000000..30651b4
--- /dev/null
+++ b/accord-core/src/main/java/accord/impl/SafeCommandsForKey.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.impl;
+
+import accord.api.Key;
+import accord.api.VisibleForImplementation;
+import accord.impl.CommandsForKey.CommandLoader;
+import accord.impl.CommandsForKey.CommandTimeseries;
+import accord.local.Command;
+import accord.primitives.Timestamp;
+import accord.utils.Invariants;
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class SafeCommandsForKey implements SafeState<CommandsForKey>
+{
+    private static final Logger logger = LoggerFactory.getLogger(SafeCommandsForKey.class);
+
+    private final Key key;
+
+    public SafeCommandsForKey(Key key)
+    {
+        this.key = key;
+    }
+
+    protected abstract void set(CommandsForKey update);
+
+    public Key key()
+    {
+        return key;
+    }
+
+    private CommandsForKey update(CommandsForKey update)
+    {
+        set(update);
+        return update;
+    }
+
+    public CommandsForKey initialize(CommandLoader<?> loader)
+    {
+        return update(new CommandsForKey(key, loader));
+    }
+
+    @VisibleForTesting
+    @VisibleForImplementation
+    public static Timestamp updateMax(CommandsForKey cfk, Timestamp timestamp)
+    {
+        Invariants.checkArgument(cfk != null || timestamp != null);
+        if (cfk == null)
+            return timestamp;
+        if (timestamp == null)
+            return cfk.max();
+        return Timestamp.max(cfk.max(), timestamp);
+    }
+
+    @VisibleForTesting
+    @VisibleForImplementation
+    public <D> CommandsForKey updateMax(Timestamp timestamp)
+    {
+        CommandsForKey current = current();
+        return update(new CommandsForKey(current.key(),
+                                         updateMax(current, timestamp),
+                                         current.lastExecutedTimestamp(),
+                                         current.lastExecutedMicros(),
+                                         current.lastWriteTimestamp(),
+                                         (CommandTimeseries<D>) current().byId(),
+                                         (CommandTimeseries<D>) current().byExecuteAt()));
+    }
+
+    public <D> CommandsForKey register(Command command)
+    {
+        CommandsForKey current = current();
+        CommandTimeseries.Update<D> byId = (CommandTimeseries.Update<D>) current().byId().beginUpdate();
+        CommandTimeseries.Update<D> byExecuteAt = (CommandTimeseries.Update<D>) current().byExecuteAt().beginUpdate();
+        return update(new CommandsForKey(current.key(),
+                                         updateMax(current, command.executeAt()),
+                                         current.lastExecutedTimestamp(),
+                                         current.lastExecutedMicros(),
+                                         current.lastWriteTimestamp(),
+                                         byId.add(command.txnId(), command).build(),
+                                         byExecuteAt.add(command.txnId(), command).build() ));
+    }
+
+    public <D> CommandsForKey listenerUpdate(Command command)
+    {
+        if (logger.isTraceEnabled())
+            logger.trace("[{}]: updating as listener in response to change on {} with status {} ({})",
+                         key(), command.txnId(), command.status(), command);
+
+        CommandsForKey current = current();
+        CommandTimeseries.Update<D> byId = (CommandTimeseries.Update<D>) current().byId().beginUpdate();
+        CommandTimeseries.Update<D> byExecuteAt = (CommandTimeseries.Update<D>) current().byExecuteAt().beginUpdate();
+
+        // 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 NotWitnessed:
+            case Accepted:
+            case AcceptedInvalidate:
+            case PreCommitted:
+                byId.add(command.txnId(), command);
+                byExecuteAt.add(command.txnId(), command);
+                break;
+            case Applied:
+            case PreApplied:
+            case Committed:
+            case ReadyToExecute:
+                byId.add(command.txnId(), command);
+                byExecuteAt.remove(command.txnId());
+                byExecuteAt.add(command.executeAt(), command);
+                break;
+            case Invalidated:
+                byId.remove(command.txnId());
+                byExecuteAt.remove(command.txnId());
+                break;
+        }
+
+        return update(new CommandsForKey(current.key(),
+                                         updateMax(current, command.executeAt()),
+                                         current.lastExecutedTimestamp(),
+                                         current.lastExecutedMicros(),
+                                         current.lastWriteTimestamp(),
+                                         byId.build(),
+                                         byExecuteAt.build()));
+    }
+
+    @VisibleForImplementation
+    public <D> CommandsForKey updateLastExecutionTimestamps(Timestamp executeAt, boolean isForWriteTxn)
+    {
+        CommandsForKey current = current();
+
+        Timestamp lastWrite = current.lastWriteTimestamp();
+
+        if (executeAt.compareTo(lastWrite) < 0)
+            throw new IllegalArgumentException(String.format("%s is less than the most recent write timestamp %s", executeAt, lastWrite));
+
+        Timestamp lastExecuted = current.lastExecutedTimestamp();
+        int cmp = executeAt.compareTo(lastExecuted);
+        // execute can be in the past if it's for a read and after the most recent write
+        if (cmp == 0 || (!isForWriteTxn && cmp < 0))
+            return current;
+        if (cmp < 0)
+            throw new IllegalArgumentException(String.format("%s is less than the most recent executed timestamp %s", executeAt, lastExecuted));
+
+        long micros = executeAt.hlc();
+        long lastMicros = current.lastExecutedMicros();
+
+        Timestamp lastExecutedTimestamp = executeAt;
+        long lastExecutedMicros = Math.max(micros, lastMicros + 1);
+        Timestamp lastWriteTimestamp = isForWriteTxn ? executeAt : current.lastWriteTimestamp();
+
+        return update(new CommandsForKey(current.key(),
+                                         current.max(),
+                                         lastExecutedTimestamp,
+                                         lastExecutedMicros,
+                                         lastWriteTimestamp,
+                                         (CommandTimeseries<D>) current.byId(),
+                                         (CommandTimeseries<D>) current.byExecuteAt()));
+    }
+}
diff --git a/accord-core/src/main/java/accord/impl/SafeState.java b/accord-core/src/main/java/accord/impl/SafeState.java
new file mode 100644
index 0000000..187bace
--- /dev/null
+++ b/accord-core/src/main/java/accord/impl/SafeState.java
@@ -0,0 +1,34 @@
+/*
+ * 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;
+
+/**
+ * State scoped to a single request that references global state
+ */
+public interface SafeState<T>
+{
+    T current();
+    void invalidate();
+    boolean invalidated();
+
+    default boolean isEmpty()
+    {
+        return current() == null;
+    }
+}
diff --git a/accord-core/src/main/java/accord/impl/SimpleProgressLog.java b/accord-core/src/main/java/accord/impl/SimpleProgressLog.java
index b1c5b5f..f768115 100644
--- a/accord-core/src/main/java/accord/impl/SimpleProgressLog.java
+++ b/accord-core/src/main/java/accord/impl/SimpleProgressLog.java
@@ -36,6 +36,7 @@
 import accord.local.Status.Known;
 import accord.primitives.*;
 import accord.utils.Invariants;
+import accord.utils.async.AsyncChain;
 import accord.utils.async.AsyncResult;
 
 import accord.api.ProgressLog;
@@ -253,8 +254,7 @@
                                                 if (token.durability.isDurable())
                                                 {
                                                     commandStore.execute(contextFor(txnId), safeStore -> {
-                                                        Command cmd = safeStore.command(txnId);
-                                                        cmd.setDurability(safeStore, token.durability, homeKey, null);
+                                                        Command cmd = Commands.setDurability(safeStore, txnId, token.durability, homeKey, null);
                                                         safeStore.progressLog().durable(txnId, cmd.maxUnseekables(), null);
                                                     }).begin(commandStore.agent());
                                                 }
@@ -423,6 +423,11 @@
                     if (notAwareOfDurability == null && !maybeReady(node, command))
                         return;
 
+                    // whenReady callbacks may run in maybeReady method, and those callbacks may set the progress to Done,
+                    // in which case this logic should no-op
+                    if (progress() == Done)
+                        return;
+
                     setProgress(Investigating);
                     if (notAwareOfDurability.isEmpty())
                     {
@@ -468,6 +473,11 @@
 
                 void record(Known known)
                 {
+                    // invalidation coordination callback may fire
+                    // before invalidation is committed locally
+                    if (progress() == Done)
+                        return;
+
                     if (blockedUntil.isSatisfiedBy(known))
                         setProgress(NoneExpected);
                 }
@@ -554,8 +564,8 @@
                 void run(Command command)
                 {
                     // make sure a quorum of the home shard is aware of the transaction, so we can rely on it to ensure progress
-                    AsyncResult<Void> inform = inform(node, txnId, command.homeKey());
-                    inform.addCallback((success, fail) -> {
+                    AsyncChain<Void> inform = inform(node, txnId, command.homeKey());
+                    inform.begin((success, fail) -> {
                         commandStore.execute(PreLoadContext.empty(), ignore -> {
                             if (progress() == Done)
                                 return;
@@ -827,7 +837,7 @@
                     {
                         commandStore.execute(contextFor(run.txnId()), safeStore -> {
                             if (run.shouldRun()) // could have been completed by a callback
-                                run.run(safeStore.command(run.txnId()));
+                                run.run(safeStore.command(run.txnId()).current());
                         }).begin(commandStore.agent());
                     }
                 }
diff --git a/accord-core/src/main/java/accord/local/AsyncCommandStores.java b/accord-core/src/main/java/accord/local/AsyncCommandStores.java
deleted file mode 100644
index a6b0300..0000000
--- a/accord-core/src/main/java/accord/local/AsyncCommandStores.java
+++ /dev/null
@@ -1,92 +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.local;
-
-import accord.api.Agent;
-import accord.api.DataStore;
-import accord.api.ProgressLog;
-import accord.primitives.Routables;
-import accord.utils.MapReduce;
-import accord.utils.MapReduceConsume;
-import accord.utils.async.AsyncChain;
-import accord.utils.async.AsyncChains;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.stream.IntStream;
-
-public class AsyncCommandStores extends CommandStores<CommandStore>
-{
-    static class AsyncMapReduceAdapter<O> implements MapReduceAdapter<CommandStore, AsyncChain<O>, List<AsyncChain<O>>, O>
-    {
-        private static final AsyncChain<?> SUCCESS = AsyncChains.success(null);
-        private static final AsyncMapReduceAdapter INSTANCE = new AsyncMapReduceAdapter<>();
-        public static <O> AsyncMapReduceAdapter<O> instance() { return INSTANCE; }
-
-        @Override
-        public List<AsyncChain<O>> allocate()
-        {
-            return new ArrayList<>();
-        }
-
-        @Override
-        public AsyncChain<O> apply(MapReduce<? super SafeCommandStore, O> map, CommandStore commandStore, PreLoadContext context)
-        {
-            return commandStore.submit(context, map);
-        }
-
-        @Override
-        public List<AsyncChain<O>> reduce(MapReduce<? super SafeCommandStore, O> reduce, List<AsyncChain<O>> chains, AsyncChain<O> next)
-        {
-            chains.add(next);
-            return chains;
-        }
-
-        @Override
-        public void consume(MapReduceConsume<?, O> reduceAndConsume, AsyncChain<O> chain)
-        {
-            chain.begin(reduceAndConsume);
-        }
-
-        @Override
-        public AsyncChain<O> reduce(MapReduce<?, O> reduce, List<AsyncChain<O>> futures)
-        {
-            if (futures.isEmpty())
-                return (AsyncChain<O>) SUCCESS;
-            return AsyncChains.reduce(futures, reduce::reduce);
-        }
-    }
-
-    public AsyncCommandStores(NodeTimeService time, Agent agent, DataStore store, ShardDistributor shardDistributor, ProgressLog.Factory progressLogFactory, CommandStore.Factory shardFactory)
-    {
-        super(time, agent, store, shardDistributor, progressLogFactory, shardFactory);
-    }
-
-    @Override
-    public <O> void mapReduceConsume(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
-    {
-        mapReduceConsume(context, keys, minEpoch, maxEpoch, mapReduceConsume, AsyncMapReduceAdapter.INSTANCE);
-    }
-
-    @Override
-    public <O> void mapReduceConsume(PreLoadContext context, IntStream commandStoreIds, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
-    {
-        mapReduceConsume(context, commandStoreIds, mapReduceConsume, AsyncMapReduceAdapter.INSTANCE);
-    }
-}
diff --git a/accord-core/src/main/java/accord/local/Command.java b/accord-core/src/main/java/accord/local/Command.java
index 8fc1f29..f8f6d00 100644
--- a/accord-core/src/main/java/accord/local/Command.java
+++ b/accord-core/src/main/java/accord/local/Command.java
@@ -18,86 +18,284 @@
 
 package accord.local;
 
-import accord.api.*;
-import accord.local.Status.Durability;
-import accord.local.Status.Known;
-import accord.primitives.*;
-import accord.primitives.Writes;
-import accord.utils.Invariants;
-import accord.utils.async.AsyncCallbacks;
-import accord.utils.async.AsyncChain;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import static accord.local.Status.*;
-import static accord.local.Status.Known.*;
-import static accord.local.Status.Known.Done;
-import static accord.local.Status.Known.ExecuteAtOnly;
-import static accord.primitives.Route.isFullRoute;
-import static accord.utils.Utils.listOf;
-
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-
-import accord.api.ProgressLog.ProgressShard;
-import accord.primitives.Ranges;
-import accord.primitives.Ballot;
-import accord.primitives.PartialDeps;
-import accord.primitives.PartialTxn;
-import accord.primitives.Route;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
+import accord.api.Data;
 import accord.api.Result;
 import accord.api.RoutingKey;
+import accord.api.VisibleForImplementation;
+import accord.primitives.*;
+import accord.utils.Invariants;
+import accord.utils.Utils;
+import accord.utils.async.AsyncChain;
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedSet;
 
-import static accord.api.ProgressLog.ProgressShard.Home;
-import static accord.api.ProgressLog.ProgressShard.Local;
-import static accord.api.ProgressLog.ProgressShard.No;
-import static accord.api.ProgressLog.ProgressShard.Unsure;
-import static accord.local.Command.EnsureAction.Add;
-import static accord.local.Command.EnsureAction.Check;
-import static accord.local.Command.EnsureAction.Ignore;
-import static accord.local.Command.EnsureAction.Set;
-import static accord.local.Command.EnsureAction.TrySet;
+import javax.annotation.Nullable;
+import java.util.*;
 
-public abstract class Command implements CommandListener, BiConsumer<SafeCommandStore, CommandListener>, PreLoadContext
+import static accord.local.Status.Durability.Local;
+import static accord.local.Status.Durability.NotDurable;
+import static accord.local.Status.Known.DefinitionOnly;
+import static accord.utils.Utils.*;
+import static java.lang.String.format;
+
+public abstract class Command implements CommonAttributes
 {
-    private static final Logger logger = LoggerFactory.getLogger(Command.class);
-
-    public abstract TxnId txnId();
-
-    // TODO (desirable, API consistency): should any of these calls be replaced by corresponding known() registers?
-    public boolean hasBeen(Status status)
+    static PreLoadContext contextForCommand(Command command)
     {
-        return status().hasBeen(status);
+        Invariants.checkState(command.hasBeen(Status.PreAccepted) && command.partialTxn() != null);
+        return command instanceof PreLoadContext ? (PreLoadContext) command : PreLoadContext.contextFor(command.txnId(), command.partialTxn().keys());
     }
 
-    public boolean has(Known known)
+    private static Status.Durability durability(Status.Durability durability, SaveStatus status)
     {
-        return known.isSatisfiedBy(saveStatus().known);
+        if (status.compareTo(SaveStatus.PreApplied) >= 0 && durability == NotDurable)
+            return Local; // not necessary anywhere, but helps for logical consistency
+        return durability;
     }
 
-    public boolean has(Definition definition)
+    @VisibleForImplementation
+    public static class SerializerSupport
     {
-        return known().definition.compareTo(definition) >= 0;
+        public static NotWitnessed notWitnessed(CommonAttributes attributes, Ballot promised)
+        {
+            return NotWitnessed.notWitnessed(attributes, promised);
+        }
+
+        public static PreAccepted preaccepted(CommonAttributes common, Timestamp executeAt, Ballot promised)
+        {
+            return PreAccepted.preAccepted(common, executeAt, promised);
+        }
+
+        public static Accepted accepted(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted)
+        {
+            return Accepted.accepted(common, status, executeAt, promised, accepted);
+        }
+
+        public static Committed committed(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted, ImmutableSortedSet<TxnId> waitingOnCommit, ImmutableSortedMap<Timestamp, TxnId> waitingOnApply)
+        {
+            return Committed.committed(common, status, executeAt, promised, accepted, waitingOnCommit, waitingOnApply);
+        }
+
+        public static Executed executed(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted, ImmutableSortedSet<TxnId> waitingOnCommit, ImmutableSortedMap<Timestamp, TxnId> waitingOnApply, Writes writes, Result result)
+        {
+            return Executed.executed(common, status, executeAt, promised, accepted, waitingOnCommit, waitingOnApply, writes, result);
+        }
     }
 
-    public boolean has(Outcome outcome)
+    private static SaveStatus validateCommandClass(SaveStatus status, Class<?> expected, Class<?> actual)
     {
-        return known().outcome.compareTo(outcome) >= 0;
+        if (actual != expected)
+        {
+            throw new IllegalStateException(format("Cannot instantiate %s for status %s. %s expected",
+                                                   actual.getSimpleName(), status, expected.getSimpleName()));
+        }
+        return status;
     }
 
-    public boolean is(Status status)
+    private static SaveStatus validateCommandClass(SaveStatus status, Class<?> klass)
     {
-        return status() == status;
+        switch (status.status)
+        {
+            case NotWitnessed:
+                return validateCommandClass(status, NotWitnessed.class, klass);
+            case PreAccepted:
+                return validateCommandClass(status, PreAccepted.class, klass);
+            case AcceptedInvalidate:
+            case Accepted:
+            case PreCommitted:
+                return validateCommandClass(status, Accepted.class, klass);
+            case Committed:
+            case ReadyToExecute:
+                return validateCommandClass(status, Committed.class, klass);
+            case PreApplied:
+            case Applied:
+            case Invalidated:
+                return validateCommandClass(status, Executed.class, klass);
+            default:
+                throw new IllegalStateException("Unhandled status " + status);
+        }
     }
 
+    public static class Listener implements CommandListener
+    {
+        protected final TxnId listenerId;
+
+        public Listener(TxnId listenerId)
+        {
+            this.listenerId = listenerId;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            Listener that = (Listener) o;
+            return listenerId.equals(that.listenerId);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(listenerId);
+        }
+
+        @Override
+        public String toString()
+        {
+            return "ListenerProxy{" + listenerId + '}';
+        }
+
+        public TxnId txnId()
+        {
+            return listenerId;
+        }
+
+        @Override
+        public void onChange(SafeCommandStore safeStore, SafeCommand safeCommand)
+        {
+            Commands.listenerUpdate(safeStore, safeStore.command(listenerId), safeCommand);
+        }
+
+        @Override
+        public PreLoadContext listenerPreLoadContext(TxnId caller)
+        {
+            return PreLoadContext.contextFor(Utils.listOf(listenerId, caller), Keys.EMPTY);
+        }
+    }
+
+    private abstract static class AbstractCommand extends Command
+    {
+        private final TxnId txnId;
+        private final SaveStatus status;
+        private final Status.Durability durability;
+        private final RoutingKey homeKey;
+        private final RoutingKey progressKey;
+        private final Route<?> route;
+        private final Ballot promised;
+        private final Listeners.Immutable listeners;
+
+        private AbstractCommand(TxnId txnId, SaveStatus status, Status.Durability durability, RoutingKey homeKey, RoutingKey progressKey, Route<?> route, Ballot promised, Listeners.Immutable listeners)
+        {
+            this.txnId = txnId;
+            this.status = validateCommandClass(status, getClass());
+            this.durability = durability;
+            this.homeKey = homeKey;
+            this.progressKey = progressKey;
+            this.route = route;
+            this.promised = promised;
+            this.listeners = listeners;
+        }
+
+        private AbstractCommand(CommonAttributes common, SaveStatus status, Ballot promised)
+        {
+            this.txnId = common.txnId();
+            this.status = validateCommandClass(status, getClass());
+            this.durability = common.durability();
+            this.homeKey = common.homeKey();
+            this.progressKey = common.progressKey();
+            this.route = common.route();
+            this.promised = promised;
+            this.listeners = common.listeners();
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            Command command = (Command) o;
+            return txnId.equals(command.txnId())
+                    && status == command.saveStatus()
+                    && durability == command.durability()
+                    && Objects.equals(homeKey, command.homeKey())
+                    && Objects.equals(progressKey, command.progressKey())
+                    && Objects.equals(route, command.route())
+                    && Objects.equals(promised, command.promised())
+                    && listeners.equals(command.listeners());
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Command@" + System.identityHashCode(this) + '{' + txnId + ':' + status + '}';
+        }
+
+        @Override
+        public TxnId txnId()
+        {
+            return txnId;
+        }
+
+        @Override
+        public final RoutingKey homeKey()
+        {
+            return homeKey;
+        }
+
+        @Override
+        public final RoutingKey progressKey()
+        {
+            return progressKey;
+        }
+
+        @Override
+        public final Route<?> route()
+        {
+            return route;
+        }
+
+        @Override
+        public Ballot promised()
+        {
+            return promised;
+        }
+
+        @Override
+        public Status.Durability durability()
+        {
+            return Command.durability(durability, saveStatus());
+        }
+
+        @Override
+        public Listeners.Immutable listeners()
+        {
+            if (listeners == null)
+                return Listeners.Immutable.EMPTY;
+            return listeners;
+        }
+
+        @Override
+        public final SaveStatus saveStatus()
+        {
+            return status;
+        }
+    }
+
+    @Override
+    public final int hashCode()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * If this is the home shard, we require that this is a Route for all states &gt; NotWitnessed;
+     * otherwise for the local progress shard this is ordinarily a PartialRoute, and for other shards this is not set,
+     * so that there is only one copy per node that can be consulted to construct the full set of involved keys.
+     *
+     * If hasBeen(Committed) this must contain the keys for both txnId.epoch and executeAt.epoch
+     */
+    public abstract Route<?> route();
+
+    /**
+     * A key nominated to be the primary shard within this node for managing progress of the command.
+     * It is nominated only as of txnId.epoch, and may be null (indicating that this node does not monitor
+     * the progress of this command).
+     *
+     * Preferentially, this is homeKey on nodes that replicate it, and otherwise any key that is replicated, as of txnId.epoch
+     */
+    public abstract RoutingKey progressKey();
+
     /**
      * homeKey is a global value that defines the home shard - the one tasked with ensuring the transaction is finished.
      * progressKey is a local value that defines the local shard responsible for ensuring progress on the transaction.
@@ -110,1014 +308,41 @@
      *  Once this restriction is imposed, we no longer need to pass around Routable.Domain with TxnId.
      */
     public abstract RoutingKey homeKey();
-    protected abstract void setHomeKey(RoutingKey key);
-
-    public abstract RoutingKey progressKey();
-    protected abstract void setProgressKey(RoutingKey key);
-
-    /**
-     * If this is the home shard, we require that this is a Route for all states &gt; NotWitnessed;
-     * otherwise for the local progress shard this is ordinarily a PartialRoute, and for other shards this is not set,
-     * so that there is only one copy per node that can be consulted to construct the full set of involved keys.
-     *
-     * If hasBeen(Committed) this must contain the keys for both txnId.epoch and executeAt.epoch
-     */
-    public abstract @Nullable Route<?> route();
-    protected abstract void setRoute(Route<?> route);
-
-    public abstract PartialTxn partialTxn();
-    protected abstract void setPartialTxn(PartialTxn txn);
-
+    public abstract TxnId txnId();
     public abstract Ballot promised();
-    protected abstract void setPromised(Ballot ballot);
-
-    public abstract Ballot accepted();
-    protected abstract void setAccepted(Ballot ballot);
-
-    public abstract Timestamp executeAt();
-    protected abstract void setExecuteAt(Timestamp timestamp);
-
-    /**
-     * While !hasBeen(Committed), used only as a register for Accept state, used by Recovery
-     * If hasBeen(Committed), represents the full deps owned by this range for execution at both txnId.epoch
-     * AND executeAt.epoch so that it may be used for Recovery (which contacts only txnId.epoch topology),
-     * but also for execution.
-     */
-    public abstract PartialDeps partialDeps();
-    protected abstract void setPartialDeps(PartialDeps deps);
-
-    public abstract Writes writes();
-    protected abstract void setWrites(Writes writes);
-
-    public abstract Result result();
-    protected abstract void setResult(Result result);
-
+    public abstract Status.Durability durability();
+    public abstract Listeners.Immutable listeners();
     public abstract SaveStatus saveStatus();
-    protected abstract void setSaveStatus(SaveStatus status);
 
-    public Status status() { return saveStatus().status; }
-    protected void setStatus(Status status) { setSaveStatus(SaveStatus.get(status, known())); }
-
-    public Known known() { return saveStatus().known; }
-
-    public abstract Durability durability();
-    public abstract void setDurability(Durability v);
-
-    public abstract Command addListener(CommandListener listener);
-    public abstract void removeListener(CommandListener listener);
-    protected abstract void notifyListeners(SafeCommandStore safeStore);
-
-    protected abstract void addWaitingOnCommit(TxnId txnId);
-    protected abstract void removeWaitingOnCommit(TxnId txnId);
-    protected abstract TxnId firstWaitingOnCommit();
-
-    protected abstract void addWaitingOnApplyIfAbsent(TxnId txnId, Timestamp executeAt);
-    protected abstract TxnId firstWaitingOnApply(@Nullable TxnId ifExecutesBefore);
-
-    protected abstract void removeWaitingOn(TxnId txnId, Timestamp executeAt);
-    protected abstract boolean isWaitingOnDependency();
-
-    public boolean hasBeenWitnessed()
+    static boolean isSameClass(Command command, Class<? extends Command> klass)
     {
-        return partialTxn() != null;
+        return command.getClass() == klass;
     }
 
-    @Override
-    public Iterable<TxnId> txnIds()
+    private static void checkNewBallot(Ballot current, Ballot next, String name)
     {
-        return Collections.singleton(txnId());
+        if (next.compareTo(current) < 0)
+            throw new IllegalArgumentException(format("Cannot update %s ballot from %s to %s. New ballot is less than current", name, current, next));
     }
 
-    @Override
-    public Seekables<?, ?> keys()
+    private static void checkPromised(Command command, Ballot ballot)
     {
-        // TODO (expected, consider): when do we need this, and will it always be sufficient?
-        return partialTxn().keys();
+        checkNewBallot(command.promised(), ballot, "promised");
     }
 
-    public void setDurability(SafeCommandStore safeStore, Durability durability, RoutingKey homeKey, @Nullable Timestamp executeAt)
+    private static void checkAccepted(Command command, Ballot ballot)
     {
-        updateHomeKey(safeStore, homeKey);
-        if (executeAt != null && hasBeen(PreCommitted) && !this.executeAt().equals(executeAt))
-            safeStore.agent().onInconsistentTimestamp(this, this.executeAt(), executeAt);
-        setDurability(durability);
+        checkNewBallot(command.accepted(), ballot, "accepted");
     }
 
-    public enum AcceptOutcome
+    private static void checkSameClass(Command command, Class<? extends Command> klass, String errorMsg)
     {
-        Success, Redundant, RejectedBallot
-    }
-
-    public AcceptOutcome preaccept(SafeCommandStore safeStore, PartialTxn partialTxn, Route<?> route, @Nullable RoutingKey progressKey)
-    {
-        return preacceptOrRecover(safeStore, partialTxn, route, progressKey, Ballot.ZERO);
-    }
-
-    public AcceptOutcome recover(SafeCommandStore safeStore, PartialTxn partialTxn, Route<?> route, @Nullable RoutingKey progressKey, Ballot ballot)
-    {
-        return preacceptOrRecover(safeStore, partialTxn, route, progressKey, ballot);
-    }
-
-    private AcceptOutcome preacceptOrRecover(SafeCommandStore safeStore, PartialTxn partialTxn, Route<?> route, @Nullable RoutingKey progressKey, Ballot ballot)
-    {
-        int compareBallots = promised().compareTo(ballot);
-        if (compareBallots > 0)
-        {
-            logger.trace("{}: skipping preaccept - higher ballot witnessed ({})", txnId(), promised());
-            return AcceptOutcome.RejectedBallot;
-        }
-        else if (compareBallots < 0)
-        {
-            // save the new ballot as a promise
-            setPromised(ballot);
-        }
-
-        if (known().definition.isKnown())
-        {
-            Invariants.checkState(status() == Invalidated || executeAt() != null);
-            logger.trace("{}: skipping preaccept - already known ({})", txnId(), status());
-            // in case of Ballot.ZERO, we must either have a competing recovery coordinator or have late delivery of the
-            // preaccept; in the former case we should abandon coordination, and in the latter we have already completed
-            return ballot.equals(Ballot.ZERO) ? AcceptOutcome.Redundant : AcceptOutcome.Success;
-        }
-
-        Ranges coordinateRanges = coordinateRanges(safeStore);
-        Invariants.checkState(!coordinateRanges.isEmpty());
-        ProgressShard shard = progressShard(safeStore, route, progressKey, coordinateRanges);
-        if (!validate(Ranges.EMPTY, coordinateRanges, shard, route, Set, partialTxn, Set, null, Ignore))
-            throw new IllegalStateException();
-
-        if (executeAt() == null)
-        {
-            TxnId txnId = txnId();
-            // unlike in the Accord paper, we partition shards within a node, so that to ensure a total order we must either:
-            //  - use a global logical clock to issue new timestamps; or
-            //  - assign each shard _and_ process a unique id, and use both as components of the timestamp
-            // 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
-            if (ballot.equals(Ballot.ZERO)) setExecuteAt(safeStore.preaccept(txnId, partialTxn.keys()));
-            else setExecuteAt(safeStore.time().uniqueNow(txnId));
-
-            if (status() == NotWitnessed)
-                setStatus(PreAccepted);
-            safeStore.progressLog().preaccepted(this, shard);
-        }
-        else
-        {
-            // TODO (expected, ?): in the case that we are pre-committed but had not been preaccepted/accepted, should we inform progressLog?
-            setSaveStatus(SaveStatus.enrich(saveStatus(), DefinitionOnly));
-        }
-        set(safeStore, Ranges.EMPTY, coordinateRanges, shard, route, partialTxn, Set, null, Ignore);
-
-        notifyListeners(safeStore);
-        return AcceptOutcome.Success;
-    }
-
-    public boolean preacceptInvalidate(Ballot ballot)
-    {
-        if (promised().compareTo(ballot) > 0)
-        {
-            logger.trace("{}: skipping preacceptInvalidate - witnessed higher ballot ({})", txnId(), promised());
-            return false;
-        }
-        setPromised(ballot);
-        return true;
-    }
-
-    public AcceptOutcome accept(SafeCommandStore safeStore, Ballot ballot, PartialRoute<?> route, Seekables<?, ?> keys, @Nullable RoutingKey progressKey, Timestamp executeAt, PartialDeps partialDeps)
-    {
-        if (this.promised().compareTo(ballot) > 0)
-        {
-            logger.trace("{}: skipping accept - witnessed higher ballot ({} > {})", txnId(), promised(), ballot);
-            return AcceptOutcome.RejectedBallot;
-        }
-
-        if (hasBeen(PreCommitted))
-        {
-            logger.trace("{}: skipping accept - already committed ({})", txnId(), status());
-            return AcceptOutcome.Redundant;
-        }
-
-        TxnId txnId = txnId();
-        Ranges coordinateRanges = coordinateRanges(safeStore);
-        Ranges acceptRanges = txnId.epoch() == executeAt.epoch() ? coordinateRanges : safeStore.ranges().between(txnId.epoch(), executeAt.epoch());
-        Invariants.checkState(!acceptRanges.isEmpty());
-        ProgressShard shard = progressShard(safeStore, route, progressKey, coordinateRanges);
-
-        if (!validate(coordinateRanges, Ranges.EMPTY, shard, route, Ignore, null, Ignore, partialDeps, Set))
-            throw new AssertionError("Invalid response from validate function");
-
-        setExecuteAt(executeAt);
-        setPromised(ballot);
-        setAccepted(ballot);
-
-        // 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.
-        set(safeStore, coordinateRanges, acceptRanges, shard, route, null, Ignore, partialDeps, Set);
-
-        // set only registers by transaction keys, which we mightn't already have received
-        if (!known().isDefinitionKnown())
-            safeStore.register(keys, acceptRanges, this);
-
-        setStatus(Accepted);
-        safeStore.progressLog().accepted(this, shard);
-        notifyListeners(safeStore);
-
-        return AcceptOutcome.Success;
-    }
-
-    public AcceptOutcome acceptInvalidate(SafeCommandStore safeStore, Ballot ballot)
-    {
-        if (this.promised().compareTo(ballot) > 0)
-        {
-            logger.trace("{}: skipping accept invalidated - witnessed higher ballot ({} > {})", txnId(), promised(), ballot);
-            return AcceptOutcome.RejectedBallot;
-        }
-
-        if (hasBeen(PreCommitted))
-        {
-            logger.trace("{}: skipping accept invalidated - already committed ({})", txnId(), status());
-            return AcceptOutcome.Redundant;
-        }
-
-        setPromised(ballot);
-        setAccepted(ballot);
-        setStatus(AcceptedInvalidate);
-        setPartialDeps(null);
-        logger.trace("{}: accepted invalidated", txnId());
-
-        notifyListeners(safeStore);
-        return AcceptOutcome.Success;
-    }
-
-    public enum CommitOutcome { Success, Redundant, Insufficient }
-
-    // relies on mutual exclusion for each key
-    public CommitOutcome commit(SafeCommandStore safeStore, Route<?> route, @Nullable RoutingKey progressKey, @Nullable PartialTxn partialTxn, Timestamp executeAt, PartialDeps partialDeps)
-    {
-        if (hasBeen(PreCommitted))
-        {
-            logger.trace("{}: skipping commit - already committed ({})", txnId(), status());
-            if (!executeAt.equals(executeAt()) || status() == Invalidated)
-                safeStore.agent().onInconsistentTimestamp(this, (status() == Invalidated ? Timestamp.NONE : this.executeAt()), executeAt);
-
-            if (hasBeen(Committed))
-                return CommitOutcome.Redundant;
-        }
-
-        Ranges coordinateRanges = coordinateRanges(safeStore);
-        // TODO (expected, consider): consider ranges between coordinateRanges and executeRanges? Perhaps don't need them
-        Ranges executeRanges = executeRanges(safeStore, executeAt);
-        ProgressShard shard = progressShard(safeStore, route, progressKey, coordinateRanges);
-
-        if (!validate(coordinateRanges, executeRanges, shard, route, Check, partialTxn, Add, partialDeps, Set))
-            return CommitOutcome.Insufficient;
-
-        setExecuteAt(executeAt);
-        set(safeStore, coordinateRanges, executeRanges, shard, route, partialTxn, Add, partialDeps, Set);
-
-        setStatus(Committed);
-        logger.trace("{}: committed with executeAt: {}, deps: {}", txnId(), executeAt, partialDeps);
-        populateWaitingOn(safeStore);
-
-        safeStore.progressLog().committed(this, shard);
-
-        // TODO (expected, safety): introduce intermediate status to avoid reentry when notifying listeners (which might notify us)
-        maybeExecute(safeStore, shard, true, true);
-        return CommitOutcome.Success;
-    }
-
-    // relies on mutual exclusion for each key
-    public void precommit(SafeCommandStore safeStore, Timestamp executeAt)
-    {
-        if (hasBeen(PreCommitted))
-        {
-            logger.trace("{}: skipping precommit - already committed ({})", txnId(), status());
-            if (executeAt.equals(executeAt()) && status() != Invalidated)
-                return;
-
-            safeStore.agent().onInconsistentTimestamp(this, (status() == Invalidated ? Timestamp.NONE : this.executeAt()), executeAt);
-        }
-
-        setExecuteAt(executeAt);
-        setStatus(PreCommitted);
-        notifyListeners(safeStore);
-        logger.trace("{}: precommitted with executeAt: {}", txnId(), executeAt);
-    }
-
-    protected void populateWaitingOn(SafeCommandStore safeStore)
-    {
-        Ranges ranges = safeStore.ranges().since(executeAt().epoch());
-        if (ranges != null)
-        {
-            partialDeps().forEach(ranges, txnId -> {
-                Command command = safeStore.ifLoaded(txnId);
-                if (command == null)
-                {
-                    addWaitingOnCommit(txnId);
-                    safeStore.addAndInvokeListener(txnId, this);
-                }
-                else
-                {
-                    switch (command.status())
-                    {
-                        default:
-                            throw new IllegalStateException();
-                        case NotWitnessed:
-                        case PreAccepted:
-                        case Accepted:
-                        case AcceptedInvalidate:
-                            // we don't know when these dependencies will execute, and cannot execute until we do
-                            command.addListener(this);
-                            addWaitingOnCommit(command.txnId());
-                            break;
-                        case PreCommitted:
-                        case Committed:
-                            // TODO (desired, efficiency): split into ReadyToRead and ReadyToWrite;
-                            //                             the distributed read can be performed as soon as those keys are ready,
-                            //                             and in parallel with any other reads. the client can even ACK immediately after;
-                            //                             only the write needs to be postponed until other in-progress reads complete
-                        case ReadyToExecute:
-                        case PreApplied:
-                        case Applied:
-                            command.addListener(this);
-                            insertPredecessor(command);
-                        case Invalidated:
-                            break;
-                    }
-                }
-            });
-        }
-    }
-
-    // TODO (expected, ?): commitInvalidate may need to update cfks _if_ possible
-    public void commitInvalidate(SafeCommandStore safeStore)
-    {
-        if (hasBeen(PreCommitted))
-        {
-            logger.trace("{}: skipping commit invalidated - already committed ({})", txnId(), status());
-            if (!hasBeen(Invalidated))
-                safeStore.agent().onInconsistentTimestamp(this, Timestamp.NONE, executeAt());
-
-            return;
-        }
-
-        ProgressShard shard = progressShard(safeStore);
-        safeStore.progressLog().invalidated(this, shard);
-        setExecuteAt(txnId());
-        if (partialDeps() == null)
-            setPartialDeps(PartialDeps.NONE);
-        setStatus(Invalidated);
-        logger.trace("{}: committed invalidated", txnId());
-
-        notifyListeners(safeStore);
-    }
-
-    public enum ApplyOutcome { Success, Redundant, Insufficient }
-
-    public ApplyOutcome apply(SafeCommandStore safeStore, long untilEpoch, Route<?> route, Timestamp executeAt, @Nullable PartialDeps partialDeps, Writes writes, Result result)
-    {
-        if (hasBeen(PreApplied) && executeAt.equals(this.executeAt()))
-        {
-            logger.trace("{}: skipping apply - already executed ({})", txnId(), status());
-            return ApplyOutcome.Redundant;
-        }
-        else if (hasBeen(PreCommitted) && !executeAt.equals(this.executeAt()))
-        {
-            safeStore.agent().onInconsistentTimestamp(this, this.executeAt(), executeAt);
-        }
-
-        Ranges coordinateRanges = coordinateRanges(safeStore);
-        Ranges executeRanges = executeRanges(safeStore, executeAt);
-        if (untilEpoch < safeStore.latestEpoch())
-        {
-            Ranges expectedRanges = safeStore.ranges().between(executeAt.epoch(), untilEpoch);
-            Invariants.checkState(expectedRanges.containsAll(executeRanges));
-        }
-        ProgressShard shard = progressShard(safeStore, route, coordinateRanges);
-
-        if (!validate(coordinateRanges, executeRanges, shard, route, Check, null, Check, partialDeps, hasBeen(Committed) ? Add : TrySet))
-            return ApplyOutcome.Insufficient; // TODO (expected, consider): this should probably be an assertion failure if !TrySet
-
-        setWrites(writes);
-        setResult(result);
-        setExecuteAt(executeAt);
-        set(safeStore, coordinateRanges, executeRanges, shard, route, null, Check, partialDeps, hasBeen(Committed) ? Add : TrySet);
-
-        if (!hasBeen(Committed))
-            populateWaitingOn(safeStore);
-        setStatus(PreApplied);
-        logger.trace("{}: apply, status set to Executed with executeAt: {}, deps: {}", txnId(), executeAt, partialDeps);
-
-        safeStore.progressLog().executed(this, shard);
-
-        maybeExecute(safeStore, shard, true, true);
-        return ApplyOutcome.Success;
-    }
-
-    @Override
-    public PreLoadContext listenerPreLoadContext(TxnId caller)
-    {
-        return PreLoadContext.contextFor(listOf(txnId(), caller));
-    }
-
-    @Override
-    public void onChange(SafeCommandStore safeStore, Command command)
-    {
-        logger.trace("{}: updating as listener in response to change on {} with status {} ({})",
-                     txnId(), command.txnId(), command.status(), command);
-        switch (command.status())
-        {
-            default:
-                throw new IllegalStateException();
-            case NotWitnessed:
-            case PreAccepted:
-            case Accepted:
-            case AcceptedInvalidate:
-                break;
-
-            case PreCommitted:
-            case Committed:
-            case ReadyToExecute:
-            case PreApplied:
-            case Applied:
-            case Invalidated:
-                updatePredecessor(command);
-                maybeExecute(safeStore, progressShard(safeStore), false, true);
-                break;
-        }
-    }
-
-    protected void postApply(SafeCommandStore safeStore)
-    {
-        logger.trace("{} applied, setting status to Applied and notifying listeners", txnId());
-        setStatus(Applied);
-        notifyListeners(safeStore);
-    }
-
-    private static Function<SafeCommandStore, Void> callPostApply(TxnId txnId)
-    {
-        return safeStore -> {
-            safeStore.command(txnId).postApply(safeStore);
-            return null;
-        };
-    }
-
-    protected AsyncChain<Void> applyChain(SafeCommandStore safeStore)
-    {
-        // important: we can't include a reference to *this* in the lambda, since the C* implementation may evict
-        // the command instance from memory between now and the write completing (and post apply being called)
-        CommandStore unsafeStore = safeStore.commandStore();
-        return writes().apply(safeStore).flatMap(unused -> unsafeStore.submit(this, callPostApply(txnId())));
-    }
-
-    private void apply(SafeCommandStore safeStore)
-    {
-        applyChain(safeStore).begin(safeStore.agent());
-    }
-
-    public AsyncChain<Data> read(SafeCommandStore safeStore)
-    {
-        return partialTxn().read(safeStore, this);
-    }
-
-    // TODO (expected, API consistency): maybe split into maybeExecute and maybeApply?
-    private boolean maybeExecute(SafeCommandStore safeStore, ProgressShard shard, boolean alwaysNotifyListeners, boolean notifyWaitingOn)
-    {
-        if (logger.isTraceEnabled())
-            logger.trace("{}: Maybe executing with status {}. Will notify listeners on noop: {}", txnId(), status(), alwaysNotifyListeners);
-
-        if (status() != Committed && status() != PreApplied)
-        {
-            if (alwaysNotifyListeners)
-                notifyListeners(safeStore);
-            return false;
-        }
-
-        if (isWaitingOnDependency())
-        {
-            if (alwaysNotifyListeners)
-                notifyListeners(safeStore);
-
-            if (notifyWaitingOn)
-                new NotifyWaitingOn(this).accept(safeStore);
-            return false;
-        }
-
-        switch (status())
-        {
-            case Committed:
-                // TODO (desirable, efficiency): maintain distinct ReadyToRead and ReadyToWrite states
-                setStatus(ReadyToExecute);
-                logger.trace("{}: set to ReadyToExecute", txnId());
-                safeStore.progressLog().readyToExecute(this, shard);
-                notifyListeners(safeStore);
-                break;
-
-            case PreApplied:
-                Ranges executeRanges = executeRanges(safeStore, executeAt());
-                boolean intersects = writes().keys.intersects(executeRanges);
-
-                if (intersects)
-                {
-                    logger.trace("{}: applying", txnId());
-                    apply(safeStore);
-                }
-                else
-                {
-                    // 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", txnId());
-                    setStatus(Applied);
-                    notifyListeners(safeStore);
-                }
-        }
-        return true;
-    }
-
-    /**
-     * @param dependency is either committed or invalidated
-     * @return true iff {@code maybeExecute} might now have a different outcome
-     */
-    private boolean updatePredecessor(Command dependency)
-    {
-        Invariants.checkState(dependency.hasBeen(PreCommitted));
-        if (dependency.hasBeen(Invalidated))
-        {
-            logger.trace("{}: {} is invalidated. Stop listening and removing from waiting on commit set.", txnId(), dependency.txnId());
-            dependency.removeListener(this);
-            removeWaitingOnCommit(dependency.txnId());
-            return true;
-        }
-        else if (dependency.executeAt().compareTo(executeAt()) > 0)
-        {
-            // dependency cannot be a predecessor if it executes later
-            logger.trace("{}: {} executes after us. Stop listening and removing from waiting on apply set.", txnId(), dependency.txnId());
-            removeWaitingOn(dependency.txnId(), dependency.executeAt());
-            dependency.removeListener(this);
-            return true;
-        }
-        else if (dependency.hasBeen(Applied))
-        {
-            logger.trace("{}: {} has been applied. Stop listening and removing from waiting on apply set.", txnId(), dependency.txnId());
-            removeWaitingOn(dependency.txnId(), dependency.executeAt());
-            dependency.removeListener(this);
-            return true;
-        }
-        else if (isWaitingOnDependency())
-        {
-            logger.trace("{}: adding {} to waiting on apply set.", txnId(), dependency.txnId());
-            addWaitingOnApplyIfAbsent(dependency.txnId(), dependency.executeAt());
-            removeWaitingOnCommit(dependency.txnId());
-            return false;
-        }
-        else
-        {
-            throw new IllegalStateException();
-        }
-    }
-
-    private void insertPredecessor(Command dependency)
-    {
-        Invariants.checkState(dependency.hasBeen(PreCommitted));
-        if (dependency.hasBeen(Invalidated))
-        {
-            logger.trace("{}: {} is invalidated. Do not insert.", txnId(), dependency.txnId());
-        }
-        else if (dependency.executeAt().compareTo(executeAt()) > 0)
-        {
-            // dependency cannot be a predecessor if it executes later
-            logger.trace("{}: {} executes after us. Do not insert.", txnId(), dependency.txnId());
-        }
-        else if (dependency.hasBeen(Applied))
-        {
-            logger.trace("{}: {} has been applied. Do not insert.", txnId(), dependency.txnId());
-        }
-        else
-        {
-            logger.trace("{}: adding {} to waiting on apply set.", txnId(), dependency.txnId());
-            addWaitingOnApplyIfAbsent(dependency.txnId(), dependency.executeAt());
-        }
-    }
-
-    void updatePredecessorAndMaybeExecute(SafeCommandStore safeStore, Command predecessor, boolean notifyWaitingOn)
-    {
-        if (hasBeen(Applied))
-            return;
-
-        if (updatePredecessor(predecessor))
-            maybeExecute(safeStore, progressShard(safeStore), false, notifyWaitingOn);
-    }
-
-    static class NotifyWaitingOn implements PreLoadContext, Consumer<SafeCommandStore>
-    {
-        Known[] blockedUntil = new Known[4];
-        TxnId[] txnIds = new TxnId[4];
-        int depth;
-
-        public NotifyWaitingOn(Command command)
-        {
-            txnIds[0] = command.txnId();
-            blockedUntil[0] = Done;
-        }
-
-        @Override
-        public void accept(SafeCommandStore safeStore)
-        {
-            Command prev = get(safeStore, depth - 1);
-            while (depth >= 0)
-            {
-                Command cur = safeStore.ifLoaded(txnIds[depth]);
-                Known until = blockedUntil[depth];
-                if (cur == null)
-                {
-                    // need to load; schedule execution for later
-                    safeStore.execute(this, this).begin(safeStore.agent());
-                    return;
-                }
-
-                if (prev != null)
-                {
-                    if (cur.has(until) || (cur.hasBeen(PreCommitted) && cur.executeAt().compareTo(prev.executeAt()) > 0))
-                    {
-                        prev.updatePredecessorAndMaybeExecute(safeStore, cur, false);
-                        --depth;
-                        prev = get(safeStore, depth - 1);
-                        continue;
-                    }
-                }
-                else if (cur.has(until))
-                {
-                    // we're done; have already applied
-                    Invariants.checkState(depth == 0);
-                    break;
-                }
-
-                TxnId directlyBlockedOnCommit = cur.firstWaitingOnCommit();
-                TxnId directlyBlockedOnApply = cur.firstWaitingOnApply(directlyBlockedOnCommit);
-                if (directlyBlockedOnApply != null)
-                {
-                    push(directlyBlockedOnApply, Done);
-                }
-                else if (directlyBlockedOnCommit != null)
-                {
-                    push(directlyBlockedOnCommit, ExecuteAtOnly);
-                }
-                else
-                {
-                    if (cur.hasBeen(Committed) && !cur.hasBeen(ReadyToExecute) && !cur.isWaitingOnDependency())
-                    {
-                        if (!cur.maybeExecute(safeStore, cur.progressShard(safeStore), false, false))
-                            throw new AssertionError("Is able to Apply, but has not done so");
-                        // loop and re-test the command's status; we may still want to notify blocking, esp. if not homeShard
-                        continue;
-                    }
-
-                    Unseekables<?, ?> someKeys = cur.maxUnseekables();
-                    if (someKeys == null && prev != null) someKeys = prev.partialDeps().someUnseekables(cur.txnId());
-                    Invariants.checkState(someKeys != null);
-                    logger.trace("{} blocked on {} until {}", txnIds[0], cur.txnId(), until);
-                    safeStore.progressLog().waiting(cur.txnId(), until, someKeys);
-                    return;
-                }
-                prev = cur;
-            }
-        }
-
-        private Command get(SafeCommandStore safeStore, int i)
-        {
-            return i >= 0 ? safeStore.command(txnIds[i]) : null;
-        }
-
-        void push(TxnId by, Known until)
-        {
-            if (++depth == txnIds.length)
-            {
-                txnIds = Arrays.copyOf(txnIds, txnIds.length * 2);
-                blockedUntil = Arrays.copyOf(blockedUntil, txnIds.length);
-            }
-            txnIds[depth] = by;
-            blockedUntil[depth] = until;
-        }
-
-        @Override
-        public Iterable<TxnId> txnIds()
-        {
-            return Arrays.asList(txnIds).subList(0, depth + 1);
-        }
-
-        @Override
-        public Seekables<?, ?> keys()
-        {
-            return Keys.EMPTY;
-        }
-    }
-
-    /**
-     * A key nominated to represent the "home" shard - only members of the home shard may be nominated to recover
-     * a transaction, to reduce the cluster-wide overhead of ensuring progress. A transaction that has only been
-     * witnessed at PreAccept may however trigger a process of ensuring the home shard is durably informed of
-     * the transaction.
-     *
-     * Note that for ProgressLog purposes the "home shard" is the shard as of txnId.epoch.
-     * For recovery purposes the "home shard" is as of txnId.epoch until Committed, and executeAt.epoch once Executed
-     */
-    public final void homeKey(RoutingKey homeKey)
-    {
-        RoutingKey current = homeKey();
-        if (current == null) setHomeKey(homeKey);
-        else if (!current.equals(homeKey)) throw new AssertionError();
-    }
-
-    public void updateHomeKey(SafeCommandStore safeStore, RoutingKey homeKey)
-    {
-        if (homeKey() == null)
-        {
-            setHomeKey(homeKey);
-            // TODO (low priority, safety): if we're processed on a node that does not know the latest epoch,
-            //      do we guarantee the home key calculation is unchanged since the prior epoch?
-            if (progressKey() == null && owns(safeStore, txnId().epoch(), homeKey))
-                progressKey(homeKey);
-        }
-        else if (!this.homeKey().equals(homeKey))
-        {
-            throw new IllegalStateException();
-        }
-    }
-
-    private ProgressShard progressShard(SafeCommandStore safeStore, Route<?> route, @Nullable RoutingKey progressKey, Ranges coordinateRanges)
-    {
-        updateHomeKey(safeStore, route.homeKey());
-
-        if (progressKey == null || progressKey == NO_PROGRESS_KEY)
-        {
-            if (this.progressKey() == null)
-                setProgressKey(NO_PROGRESS_KEY);
-
-            return No;
-        }
-
-        if (this.progressKey() == null) setProgressKey(progressKey);
-        else if (!this.progressKey().equals(progressKey)) throw new AssertionError();
-
-        if (!coordinateRanges.contains(progressKey))
-            return No;
-
-        return progressKey.equals(homeKey()) ? Home : Local;
-    }
-
-    /**
-     * A key nominated to be the primary shard within this node for managing progress of the command.
-     * It is nominated only as of txnId.epoch, and may be null (indicating that this node does not monitor
-     * the progress of this command).
-     *
-     * Preferentially, this is homeKey on nodes that replicate it, and otherwise any key that is replicated, as of txnId.epoch
-     */
-
-    public final void progressKey(RoutingKey progressKey)
-    {
-        RoutingKey current = progressKey();
-        if (current == null) setProgressKey(progressKey);
-        else if (!current.equals(progressKey)) throw new AssertionError();
-    }
-
-    private ProgressShard progressShard(SafeCommandStore safeStore, Route<?> route, Ranges coordinateRanges)
-    {
-        if (progressKey() == null)
-            return Unsure;
-
-        return progressShard(safeStore, route, progressKey(), coordinateRanges);
-    }
-
-    private ProgressShard progressShard(SafeCommandStore safeStore)
-    {
-        RoutingKey progressKey = progressKey();
-        if (progressKey == null)
-            return Unsure;
-
-        if (progressKey == NO_PROGRESS_KEY)
-            return No;
-
-        Ranges coordinateRanges = safeStore.ranges().at(txnId().epoch());
-        if (!coordinateRanges.contains(progressKey))
-            return No;
-
-        return progressKey.equals(homeKey()) ? Home : Local;
-    }
-
-    private Ranges coordinateRanges(SafeCommandStore safeStore)
-    {
-        return safeStore.ranges().at(txnId().epoch());
-    }
-
-    private Ranges executeRanges(SafeCommandStore safeStore, Timestamp executeAt)
-    {
-        return safeStore.ranges().since(executeAt.epoch());
-    }
-
-    enum EnsureAction { Ignore, Check, Add, TrySet, Set }
-
-    /**
-     * Validate we have sufficient information for the route, partialTxn and partialDeps fields, and if so update them;
-     * otherwise return false (or throw an exception if an illegal state is encountered)
-     */
-    private boolean validate(Ranges existingRanges, Ranges additionalRanges, ProgressShard shard,
-                             Route<?> route, EnsureAction ensureRoute,
-                             @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn,
-                             @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps)
-    {
-        if (shard == Unsure)
-            return false;
-
-        // first validate route
-        if (shard.isHome())
-        {
-            switch (ensureRoute)
-            {
-                default: throw new AssertionError();
-                case Check:
-                    if (!isFullRoute(route()) && !isFullRoute(route))
-                        return false;
-                case Ignore:
-                    break;
-                case Add:
-                case Set:
-                    if (!isFullRoute(route))
-                        throw new IllegalArgumentException("Incomplete route (" + route + ") sent to home shard");
-                    break;
-                case TrySet:
-                    if (!isFullRoute(route))
-                        return false;
-            }
-        }
-        else
-        {
-            // failing any of these tests is always an illegal state
-            if (!route.covers(existingRanges))
-                return false;
-
-            if (existingRanges != additionalRanges && !route.covers(additionalRanges))
-                throw new IllegalArgumentException("Incomplete route (" + route + ") provided; does not cover " + additionalRanges);
-        }
-
-        // invalid to Add deps to Accepted or AcceptedInvalidate statuses, as Committed deps are not equivalent
-        // and we may erroneously believe we have covered a wider range than we have infact covered
-        if (ensurePartialDeps == Add)
-            Invariants.checkState(status() != Accepted && status() != AcceptedInvalidate);
-
-        // validate new partial txn
-        if (!validate(ensurePartialTxn, existingRanges, additionalRanges, covers(partialTxn()), covers(partialTxn), "txn", partialTxn))
-            return false;
-
-        if (partialTxn != null && txnId().rw() != partialTxn.kind())
-            throw new IllegalArgumentException("Transaction has different kind to its TxnId");
-
-        if (shard.isHome() && ensurePartialTxn != Ignore)
-        {
-            if (!hasQuery(partialTxn()) && !hasQuery(partialTxn))
-                throw new IllegalStateException();
-        }
-
-        return validate(ensurePartialDeps, existingRanges, additionalRanges, covers(partialDeps()), covers(partialDeps), "deps", partialDeps);
-    }
-
-    private void set(SafeCommandStore safeStore,
-                     Ranges existingRanges, Ranges additionalRanges, ProgressShard shard, Route<?> route,
-                     @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn,
-                     @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps)
-    {
-        Invariants.checkState(progressKey() != null);
-        Ranges allRanges = existingRanges.with(additionalRanges);
-
-        if (shard.isProgress()) setRoute(Route.merge(route(), (Route)route));
-        else setRoute(route.slice(allRanges));
-
-        switch (ensurePartialTxn)
-        {
-            case Add:
-                if (partialTxn == null)
-                    break;
-
-                if (partialTxn() != null)
-                {
-                    partialTxn = partialTxn.slice(allRanges, shard.isHome());
-                    Routables.foldlMissing((Seekables)partialTxn.keys(), partialTxn().keys(), (keyOrRange, p, v, i) -> {
-                        // TODO (expected, efficiency): we may register the same ranges more than once
-                        safeStore.register(keyOrRange, allRanges, this);
-                        return v;
-                    }, 0, 0, 1);
-                    this.setPartialTxn(partialTxn().with(partialTxn));
-                    break;
-                }
-
-            case Set:
-            case TrySet:
-                setPartialTxn(partialTxn = partialTxn.slice(allRanges, shard.isHome()));
-                // TODO (expected, efficiency): we may register the same ranges more than once
-                // TODO (desirable, efficiency): no need to register on PreAccept if already Accepted
-                safeStore.register(partialTxn.keys(), allRanges, this);
-                break;
-        }
-
-        switch (ensurePartialDeps)
-        {
-            case Add:
-                if (partialDeps == null)
-                    break;
-
-                if (partialDeps() != null)
-                {
-                    setPartialDeps(partialDeps().with(partialDeps.slice(allRanges)));
-                    break;
-                }
-
-            case Set:
-            case TrySet:
-                setPartialDeps(partialDeps.slice(allRanges));
-                break;
-        }
-    }
-
-    private static boolean validate(EnsureAction action, Ranges existingRanges, Ranges additionalRanges,
-                                    Ranges existing, Ranges adding, String kind, Object obj)
-    {
-        switch (action)
-        {
-            default: throw new IllegalStateException();
-            case Ignore:
-                break;
-
-            case TrySet:
-                if (adding != null)
-                {
-                    if (!adding.containsAll(existingRanges))
-                        return false;
-
-                    if (additionalRanges != existingRanges && !adding.containsAll(additionalRanges))
-                        return false;
-
-                    break;
-                }
-            case Set:
-                // failing any of these tests is always an illegal state
-                Invariants.checkState(adding != null);
-                if (!adding.containsAll(existingRanges))
-                    throw new IllegalArgumentException("Incomplete " + kind + " (" + obj + ") provided; does not cover " + existingRanges);
-
-                if (additionalRanges != existingRanges && !adding.containsAll(additionalRanges))
-                    throw new IllegalArgumentException("Incomplete " + kind + " (" + obj + ") provided; does not cover " + additionalRanges);
-                break;
-
-            case Check:
-            case Add:
-                if (adding == null)
-                {
-                    if (existing == null)
-                        return false;
-
-                    Invariants.checkState(existing.containsAll(existingRanges));
-                    if (existingRanges != additionalRanges && !existing.containsAll(additionalRanges))
-                    {
-                        if (action == Check)
-                            return false;
-
-                        throw new IllegalArgumentException("Missing additional " + kind + "; existing does not cover " + additionalRanges.difference(existingRanges));
-                    }
-                }
-                else if (existing != null)
-                {
-                    Ranges covering = adding.with(existing);
-                    Invariants.checkState(covering.containsAll(existingRanges));
-                    if (existingRanges != additionalRanges && !covering.containsAll(additionalRanges))
-                    {
-                        if (action == Check)
-                            return false;
-
-                        throw new IllegalArgumentException("Incomplete additional " + kind + " (" + obj + ") provided; does not cover " + additionalRanges.difference(existingRanges));
-                    }
-                }
-                else
-                {
-                    if (!adding.containsAll(existingRanges))
-                        return false;
-
-                    if (existingRanges != additionalRanges && !adding.containsAll(additionalRanges))
-                    {
-                        if (action == Check)
-                            return false;
-
-                        throw new IllegalArgumentException("Incomplete additional " + kind + " (" + obj + ") provided; does not cover " + additionalRanges.difference(existingRanges));
-                    }
-                }
-                break;
-        }
-
-        return true;
+        if (!isSameClass(command, klass))
+            throw new IllegalArgumentException(errorMsg + format(" expected %s got %s", klass.getSimpleName(), command.getClass().getSimpleName()));
     }
 
     // TODO (low priority, progress): callers should try to consult the local progress shard (if any) to obtain the full set of keys owned locally
-    public Route<?> someRoute()
+    public final Route<?> someRoute()
     {
         if (route() != null)
             return route();
@@ -1137,63 +362,657 @@
         return route.toMaximalUnseekables();
     }
 
-    /**
-     * true iff this commandStore owns the given key on the given epoch
-     */
-    public boolean owns(SafeCommandStore safeStore, long epoch, RoutingKey someKey)
+    public PreLoadContext contextForSelf()
     {
-        return safeStore.ranges().at(epoch).contains(someKey);
+        return contextForCommand(this);
     }
 
-    @Override
-    public void accept(SafeCommandStore safeStore, CommandListener listener)
+    public abstract Timestamp executeAt();
+    public abstract Ballot accepted();
+    public abstract PartialTxn partialTxn();
+    public abstract @Nullable PartialDeps partialDeps();
+
+    public final Status status()
     {
-        listener.onChange(safeStore, this);
+        return saveStatus().status;
     }
 
-    @Override
-    public String toString()
+    public final Status.Known known()
     {
-        return "Command{" +
-               "txnId=" + txnId() +
-               ", status=" + status() +
-               ", partialTxn=" + partialTxn() +
-               ", executeAt=" + executeAt() +
-               ", partialDeps=" + partialDeps() +
-               '}';
+        return saveStatus().known;
     }
 
-    private static Ranges covers(@Nullable PartialTxn txn)
+    public final boolean hasBeen(Status status)
     {
-        return txn == null ? null : txn.covering();
+        return status().compareTo(status) >= 0;
     }
 
-    private static Ranges covers(@Nullable PartialDeps deps)
+    public boolean has(Status.Known known)
     {
-        return deps == null ? null : deps.covering;
+        return known.isSatisfiedBy(saveStatus().known);
     }
 
-    private static boolean hasQuery(PartialTxn txn)
+    public boolean has(Status.Definition definition)
     {
-        return txn != null && txn.query() != null;
+        return known().definition.compareTo(definition) >= 0;
     }
 
-    // TODO (low priority, API): this is an ugly hack, need to encode progress/homeKey/Route state combinations much more clearly
-    //                           (perhaps introduce encapsulating class representing each possible arrangement)
-    static class NoProgressKey implements RoutingKey
+    public boolean has(Status.Outcome outcome)
     {
-        @Override
-        public int compareTo(@Nonnull RoutableKey that)
+        return known().outcome.compareTo(outcome) >= 0;
+    }
+
+    public boolean is(Status status)
+    {
+        return status() == status;
+    }
+
+    public final CommandListener asListener()
+    {
+        return new Listener(txnId());
+    }
+
+    public final boolean isWitnessed()
+    {
+        boolean result = status().hasBeen(Status.PreAccepted);
+        Invariants.checkState(result == (this instanceof PreAccepted));
+        return result;
+    }
+
+    public final PreAccepted asWitnessed()
+    {
+        return Invariants.cast(this, PreAccepted.class);
+    }
+
+    public final boolean isAccepted()
+    {
+        boolean result = status().hasBeen(Status.AcceptedInvalidate);
+        Invariants.checkState(result == (this instanceof Accepted));
+        return result;
+    }
+
+    public final Accepted asAccepted()
+    {
+        return Invariants.cast(this, Accepted.class);
+    }
+
+    public final boolean isCommitted()
+    {
+        boolean result = status().hasBeen(Status.Committed);
+        Invariants.checkState(result == (this instanceof Committed));
+        return result;
+    }
+
+    public final Committed asCommitted()
+    {
+        return Invariants.cast(this, Committed.class);
+    }
+
+    public final boolean isExecuted()
+    {
+        boolean result = status().hasBeen(Status.PreApplied);
+        Invariants.checkState(result == (this instanceof Executed));
+        return result;
+    }
+
+    public final Executed asExecuted()
+    {
+        return Invariants.cast(this, Executed.class);
+    }
+
+    public abstract Command updateAttributes(CommonAttributes attrs, Ballot promised);
+
+    public final Command updateAttributes(CommonAttributes attrs)
+    {
+        return updateAttributes(attrs, promised());
+    }
+
+    public final Command updatePromised(Ballot promised)
+    {
+        return updateAttributes(this, promised);
+    }
+
+    public static final class NotWitnessed extends AbstractCommand
+    {
+        NotWitnessed(TxnId txnId, SaveStatus status, Status.Durability durability, RoutingKey homeKey, RoutingKey progressKey, Route<?> route, Ballot promised, Listeners.Immutable listeners)
         {
-            throw new UnsupportedOperationException();
+            super(txnId, status, durability, homeKey, progressKey, route, promised, listeners);
+        }
+
+        NotWitnessed(CommonAttributes common, SaveStatus status, Ballot promised)
+        {
+            super(common, status, promised);
         }
 
         @Override
-        public Range asRange()
+        public Command updateAttributes(CommonAttributes attrs, Ballot promised)
         {
-            throw new UnsupportedOperationException();
+            return new NotWitnessed(attrs, saveStatus(), promised);
+        }
+
+        public static NotWitnessed notWitnessed(CommonAttributes common, Ballot promised)
+        {
+            return new NotWitnessed(common, SaveStatus.NotWitnessed, promised);
+        }
+
+        public static NotWitnessed notWitnessed(TxnId txnId)
+        {
+            return new NotWitnessed(txnId, SaveStatus.NotWitnessed, NotDurable, null, null, null, Ballot.ZERO, null);
+        }
+
+        public static NotWitnessed notWitnessed(NotWitnessed command, CommonAttributes common, Ballot promised)
+        {
+            checkSameClass(command, NotWitnessed.class, "Cannot update");
+            Invariants.checkArgument(command.txnId().equals(common.txnId()));
+            return new NotWitnessed(common, command.saveStatus(), promised);
+        }
+
+        @Override
+        public Timestamp executeAt()
+        {
+            return null;
+        }
+
+        @Override
+        public Ballot promised()
+        {
+            return Ballot.ZERO;
+        }
+
+        @Override
+        public Ballot accepted()
+        {
+            return Ballot.ZERO;
+        }
+
+        @Override
+        public PartialTxn partialTxn()
+        {
+            return null;
+        }
+
+        @Override
+        public @Nullable PartialDeps partialDeps()
+        {
+            return null;
         }
     }
 
-    private static final NoProgressKey NO_PROGRESS_KEY = new NoProgressKey();
+    public static class PreAccepted extends AbstractCommand
+    {
+        private final Timestamp executeAt;
+        private final PartialTxn partialTxn;
+        private final @Nullable PartialDeps partialDeps;
+
+        private PreAccepted(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised)
+        {
+            super(common, status, promised);
+            this.executeAt = executeAt;
+            this.partialTxn = common.partialTxn();
+            this.partialDeps = common.partialDeps();
+        }
+
+        @Override
+        public Command updateAttributes(CommonAttributes attrs, Ballot promised)
+        {
+            return new PreAccepted(attrs, saveStatus(), executeAt(), promised);
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            if (!super.equals(o)) return false;
+            PreAccepted that = (PreAccepted) o;
+            return executeAt.equals(that.executeAt)
+                    && Objects.equals(partialTxn, that.partialTxn)
+                    && Objects.equals(partialDeps, that.partialDeps);
+        }
+
+        public static PreAccepted preAccepted(CommonAttributes common, Timestamp executeAt, Ballot promised)
+        {
+            return new PreAccepted(common, SaveStatus.PreAccepted, executeAt, promised);
+        }
+        public static PreAccepted preAccepted(PreAccepted command, CommonAttributes common, Ballot promised)
+        {
+            checkPromised(command, promised);
+            checkSameClass(command, PreAccepted.class, "Cannot update");
+            Invariants.checkArgument(command.getClass() == PreAccepted.class);
+            return preAccepted(common, command.executeAt(), promised);
+        }
+
+        @Override
+        public Timestamp executeAt()
+        {
+            return executeAt;
+        }
+
+        @Override
+        public Ballot accepted()
+        {
+            return Ballot.ZERO;
+        }
+
+        @Override
+        public PartialTxn partialTxn()
+        {
+            return partialTxn;
+        }
+
+        @Override
+        public @Nullable PartialDeps partialDeps()
+        {
+            return partialDeps;
+        }
+    }
+
+    public static class Accepted extends PreAccepted
+    {
+        private final Ballot accepted;
+
+        Accepted(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted)
+        {
+            super(common, status, executeAt, promised);
+            this.accepted = accepted;
+        }
+
+        @Override
+        public Command updateAttributes(CommonAttributes attrs, Ballot promised)
+        {
+            return new Accepted(attrs, saveStatus(), executeAt(), promised, accepted());
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            if (!super.equals(o)) return false;
+            Accepted that = (Accepted) o;
+            return Objects.equals(accepted, that.accepted);
+        }
+
+        static Accepted accepted(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted)
+        {
+            return new Accepted(common, status, executeAt, promised, accepted);
+        }
+        static Accepted accepted(Accepted command, CommonAttributes common, SaveStatus status, Ballot promised)
+        {
+            checkPromised(command, promised);
+            checkSameClass(command, Accepted.class, "Cannot update");
+            return new Accepted(common, status, command.executeAt(), promised, command.accepted());
+        }
+        static Accepted accepted(Accepted command, CommonAttributes common, Ballot promised)
+        {
+            return accepted(command, common, command.saveStatus(), promised);
+        }
+
+        @Override
+        public Ballot accepted()
+        {
+            return accepted;
+        }
+    }
+
+    public static class Committed extends Accepted
+    {
+        private final ImmutableSortedSet<TxnId> waitingOnCommit;
+        private final ImmutableSortedMap<Timestamp, TxnId> waitingOnApply;
+
+        private Committed(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted, ImmutableSortedSet<TxnId> waitingOnCommit, ImmutableSortedMap<Timestamp, TxnId> waitingOnApply)
+        {
+            super(common, status, executeAt, promised, accepted);
+            this.waitingOnCommit = waitingOnCommit;
+            this.waitingOnApply = waitingOnApply;
+        }
+
+        private Committed(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted, WaitingOn waitingOn)
+        {
+            this(common, status, executeAt, promised, accepted, waitingOn.waitingOnCommit, waitingOn.waitingOnApply);
+        }
+
+        @Override
+        public Command updateAttributes(CommonAttributes attrs, Ballot promised)
+        {
+            return new Committed(attrs, saveStatus(), executeAt(), promised, accepted(), waitingOnCommit(), waitingOnApply());
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            if (!super.equals(o)) return false;
+            Committed committed = (Committed) o;
+            return Objects.equals(waitingOnCommit, committed.waitingOnCommit)
+                    && Objects.equals(waitingOnApply, committed.waitingOnApply);
+        }
+
+        private static Committed committed(Committed command, CommonAttributes common, Ballot promised, SaveStatus status, ImmutableSortedSet<TxnId> waitingOnCommit, ImmutableSortedMap<Timestamp, TxnId> waitingOnApply)
+        {
+            checkPromised(command, promised);
+            checkSameClass(command, Committed.class, "Cannot update");
+            return new Committed(common, status, command.executeAt(), promised, command.accepted(), waitingOnCommit, waitingOnApply);
+        }
+
+        static Committed committed(Committed command, CommonAttributes common, Ballot promised)
+        {
+            return committed(command, common, promised, command.saveStatus(), command.waitingOnCommit(), command.waitingOnApply());
+        }
+
+        static Committed committed(Committed command, CommonAttributes common, SaveStatus status)
+        {
+            return committed(command, common, command.promised(), status, command.waitingOnCommit(), command.waitingOnApply());
+        }
+
+        static Committed committed(Committed command, CommonAttributes common, WaitingOn waitingOn)
+        {
+            return committed(command, common, command.promised(), command.saveStatus(), waitingOn.waitingOnCommit, waitingOn.waitingOnApply);
+        }
+
+        static Committed committed(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted, ImmutableSortedSet<TxnId> waitingOnCommit, ImmutableSortedMap<Timestamp, TxnId> waitingOnApply)
+        {
+            return new Committed(common, status, executeAt, promised, accepted, waitingOnCommit, waitingOnApply);
+        }
+
+        public AsyncChain<Data> read(SafeCommandStore safeStore)
+        {
+            return partialTxn().read(safeStore, this);
+        }
+
+        public WaitingOn waitingOn()
+        {
+            return new WaitingOn(waitingOnCommit, waitingOnApply);
+        }
+
+        public ImmutableSortedSet<TxnId> waitingOnCommit()
+        {
+            return waitingOnCommit;
+        }
+
+        public boolean isWaitingOnCommit()
+        {
+            return waitingOnCommit != null && !waitingOnCommit.isEmpty();
+        }
+
+        public TxnId firstWaitingOnCommit()
+        {
+            return isWaitingOnCommit() ? waitingOnCommit.first() : null;
+        }
+
+        public ImmutableSortedMap<Timestamp, TxnId> waitingOnApply()
+        {
+            return waitingOnApply;
+        }
+
+        public boolean isWaitingOnApply()
+        {
+            return waitingOnApply != null && !waitingOnApply.isEmpty();
+        }
+
+        public TxnId firstWaitingOnApply()
+        {
+            return isWaitingOnApply() ? waitingOnApply.firstEntry().getValue() : null;
+        }
+
+        public boolean isWaitingOnDependency()
+        {
+            return isWaitingOnCommit() || isWaitingOnApply();
+        }
+    }
+
+    public static class Executed extends Committed
+    {
+        private final Writes writes;
+        private final Result result;
+
+        public Executed(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted, ImmutableSortedSet<TxnId> waitingOnCommit, ImmutableSortedMap<Timestamp, TxnId> waitingOnApply, Writes writes, Result result)
+        {
+            super(common, status, executeAt, promised, accepted, waitingOnCommit, waitingOnApply);
+            this.writes = writes;
+            this.result = result;
+        }
+
+        public Executed(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted, WaitingOn waitingOn, Writes writes, Result result)
+        {
+            super(common, status, executeAt, promised, accepted, waitingOn);
+            this.writes = writes;
+            this.result = result;
+        }
+
+        @Override
+        public Command updateAttributes(CommonAttributes attrs, Ballot promised)
+        {
+            return new Executed(attrs, saveStatus(), executeAt(), promised, accepted(), waitingOnCommit(), waitingOnApply(), writes, result);
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            if (!super.equals(o)) return false;
+            Executed executed = (Executed) o;
+            return Objects.equals(writes, executed.writes)
+                    && Objects.equals(result, executed.result);
+        }
+
+        public static Executed executed(Executed command, CommonAttributes common, SaveStatus status, Ballot promised, ImmutableSortedSet<TxnId> waitingOnCommit, ImmutableSortedMap<Timestamp, TxnId> waitingOnApply)
+        {
+            checkSameClass(command, Executed.class, "Cannot update");
+            return new Executed(common, status, command.executeAt(), promised, command.accepted(), waitingOnCommit, waitingOnApply, command.writes(), command.result());
+        }
+
+        public static Executed executed(Executed command, CommonAttributes common, SaveStatus status)
+        {
+            return executed(command, common, status, command.promised(), command.waitingOnCommit(), command.waitingOnApply());
+        }
+
+        public static Executed executed(Executed command, CommonAttributes common, WaitingOn waitingOn)
+        {
+            return executed(command, common, command.saveStatus(), command.promised(), waitingOn.waitingOnCommit, waitingOn.waitingOnApply);
+        }
+
+        public static Executed executed(Executed command, CommonAttributes common, Ballot promised)
+        {
+            return executed(command, common, command.saveStatus(), promised, command.waitingOnCommit(), command.waitingOnApply());
+        }
+
+        public static Executed executed(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted, ImmutableSortedSet<TxnId> waitingOnCommit, ImmutableSortedMap<Timestamp, TxnId> waitingOnApply, Writes writes, Result result)
+        {
+            return new Executed(common, status, executeAt, promised, accepted, waitingOnCommit, waitingOnApply, writes, result);
+        }
+
+        public static Executed executed(CommonAttributes common, SaveStatus status, Timestamp executeAt, Ballot promised, Ballot accepted, WaitingOn waitingOn, Writes writes, Result result)
+        {
+            return new Executed(common, status, executeAt, promised, accepted, waitingOn.waitingOnCommit, waitingOn.waitingOnApply, writes, result);
+        }
+
+        public Writes writes()
+        {
+            return writes;
+        }
+
+        public Result result()
+        {
+            return result;
+        }
+    }
+
+    public static class WaitingOn
+    {
+        public static final WaitingOn EMPTY = new WaitingOn(ImmutableSortedSet.of(), ImmutableSortedMap.of());
+        public final ImmutableSortedSet<TxnId> waitingOnCommit;
+        public final ImmutableSortedMap<Timestamp, TxnId> waitingOnApply;
+
+        public WaitingOn(ImmutableSortedSet<TxnId> waitingOnCommit, ImmutableSortedMap<Timestamp, TxnId> waitingOnApply)
+        {
+            this.waitingOnCommit = waitingOnCommit;
+            this.waitingOnApply = waitingOnApply;
+        }
+
+        public static class Update
+        {
+            private boolean hasChanges = false;
+            private NavigableSet<TxnId> waitingOnCommit;
+            private NavigableMap<Timestamp, TxnId> waitingOnApply;
+
+            public Update()
+            {
+
+            }
+
+            public Update(WaitingOn waitingOn)
+            {
+                this.waitingOnCommit = waitingOn.waitingOnCommit;
+                this.waitingOnApply = waitingOn.waitingOnApply;
+            }
+
+            public Update(Committed committed)
+            {
+                this.waitingOnCommit = committed.waitingOnCommit();
+                this.waitingOnApply = committed.waitingOnApply();
+            }
+
+            public boolean hasChanges()
+            {
+                return hasChanges;
+            }
+
+            public void addWaitingOnCommit(TxnId txnId)
+            {
+                waitingOnCommit = ensureSortedMutable(waitingOnCommit);
+                waitingOnCommit.add(txnId);
+                hasChanges = true;
+            }
+
+            public void removeWaitingOnCommit(TxnId txnId)
+            {
+                if (waitingOnApply == null)
+                    return;
+                waitingOnCommit = ensureSortedMutable(waitingOnCommit);
+                waitingOnCommit.remove(txnId);
+                hasChanges = true;
+            }
+
+            public void addWaitingOnApply(TxnId txnId, Timestamp executeAt)
+            {
+                waitingOnApply = ensureSortedMutable(waitingOnApply);
+                waitingOnApply.put(executeAt, txnId);
+                hasChanges = true;
+            }
+
+            public void removeWaitingOnApply(TxnId txnId, Timestamp executeAt)
+            {
+                if (waitingOnApply == null)
+                    return;
+                waitingOnApply = ensureSortedMutable(waitingOnApply);
+                waitingOnApply.remove(executeAt);
+                hasChanges = true;
+            }
+
+            public void removeWaitingOn(TxnId txnId, Timestamp executeAt)
+            {
+                removeWaitingOnCommit(txnId);
+                removeWaitingOnApply(txnId, executeAt);
+                hasChanges = true;
+            }
+
+            public WaitingOn build()
+            {
+                if ((waitingOnCommit == null || waitingOnCommit.isEmpty()) && (waitingOnApply == null || waitingOnApply.isEmpty()))
+                    return EMPTY;
+                return new WaitingOn(ensureSortedImmutable(waitingOnCommit), ensureSortedImmutable(waitingOnApply));
+            }
+        }
+    }
+
+    static Command addListener(Command command, CommandListener listener)
+    {
+        CommonAttributes attrs = command.mutable().addListener(listener);
+        return command.updateAttributes(attrs);
+    }
+
+    static Command removeListener(Command command, CommandListener listener)
+    {
+        CommonAttributes attrs = command.mutable().removeListener(listener);
+        return command.updateAttributes(attrs);
+    }
+
+    static Command.Committed updateWaitingOn(Committed command, WaitingOn.Update waitingOn)
+    {
+        if (!waitingOn.hasChanges())
+            return command;
+
+        return command instanceof Command.Executed ?
+                Command.Executed.executed(command.asExecuted(), command, waitingOn.build()) :
+                Command.Committed.committed(command, command, waitingOn.build());
+    }
+
+    static Command.PreAccepted preaccept(Command command, CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
+    {
+        if (command.status() == Status.NotWitnessed)
+        {
+            return Command.PreAccepted.preAccepted(attrs, executeAt, ballot);
+        }
+        else if (command.status() == Status.AcceptedInvalidate && command.executeAt() == null)
+        {
+            Command.Accepted accepted = command.asAccepted();
+            return Command.Accepted.accepted(attrs, accepted.saveStatus(), executeAt, ballot, accepted.accepted());
+        }
+        else
+        {
+            Invariants.checkState(command.status() == Status.Accepted);
+            return (Command.PreAccepted) command.updateAttributes(attrs, ballot);
+        }
+    }
+
+    static Command.Accepted markDefined(Command command, CommonAttributes attributes, Ballot promised)
+    {
+        if (Command.isSameClass(command, Command.Accepted.class))
+            return Command.Accepted.accepted(command.asAccepted(), attributes, SaveStatus.enrich(command.saveStatus(), DefinitionOnly), promised);
+        return (Command.Accepted) command.updateAttributes(attributes, promised);
+    }
+
+    static Command.Accepted accept(Command command, CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
+    {
+        return new Command.Accepted(attrs, SaveStatus.get(Status.Accepted, command.known()), executeAt, ballot, ballot);
+    }
+
+    static Command.Accepted acceptInvalidated(Command command, Ballot ballot)
+    {
+        Timestamp executeAt = command.isWitnessed() ? command.asWitnessed().executeAt() : null;
+        return new Command.Accepted(command, SaveStatus.get(Status.AcceptedInvalidate, command.known()), executeAt, ballot, ballot);
+    }
+
+    static Command.Committed commit(Command command, CommonAttributes attrs, Timestamp executeAt, Command.WaitingOn waitingOn)
+    {
+        return Command.Committed.committed(attrs, SaveStatus.get(Status.Committed, command.known()), executeAt, command.promised(), command.accepted(), waitingOn.waitingOnCommit, waitingOn.waitingOnApply);
+    }
+
+    static Command precommit(Command command, Timestamp executeAt)
+    {
+        return new Command.Accepted(command, SaveStatus.get(Status.PreCommitted, command.known()), executeAt, command.promised(), command.accepted());
+    }
+
+    static Command.Committed commitInvalidated(Command command, CommonAttributes attrs, Timestamp executeAt)
+    {
+        return Command.Executed.executed(attrs, SaveStatus.get(Status.Invalidated, command.known()), executeAt, command.promised(), command.accepted(), Command.WaitingOn.EMPTY, null, null);
+    }
+
+    static Command.Committed readyToExecute(Command.Committed command)
+    {
+        return Command.Committed.committed(command, command, SaveStatus.ReadyToExecute);
+    }
+
+    static Command.Executed preapplied(Command command, CommonAttributes attrs, Timestamp executeAt, Command.WaitingOn waitingOn, Writes writes, Result result)
+    {
+        return Command.Executed.executed(attrs, SaveStatus.get(Status.PreApplied, command.known()), executeAt, command.promised(), command.accepted(), waitingOn, writes, result);
+    }
+
+    static Command.Executed applied(Command.Executed command)
+    {
+        return Command.Executed.executed(command, command, SaveStatus.get(Status.Applied, command.known()));
+    }
 }
diff --git a/accord-core/src/main/java/accord/local/CommandListener.java b/accord-core/src/main/java/accord/local/CommandListener.java
index 1cd21f4..cb7a782 100644
--- a/accord-core/src/main/java/accord/local/CommandListener.java
+++ b/accord-core/src/main/java/accord/local/CommandListener.java
@@ -22,7 +22,7 @@
 
 public interface CommandListener
 {
-    void onChange(SafeCommandStore safeStore, Command command);
+    void onChange(SafeCommandStore safeStore, SafeCommand safeCommand);
 
     /**
      * Scope needed to run onChange
diff --git a/accord-core/src/main/java/accord/local/CommandStore.java b/accord-core/src/main/java/accord/local/CommandStore.java
index 4436a0a..65f8949 100644
--- a/accord-core/src/main/java/accord/local/CommandStore.java
+++ b/accord-core/src/main/java/accord/local/CommandStore.java
@@ -30,9 +30,9 @@
 /**
  * Single threaded internal shard of accord transaction metadata
  */
-public abstract class CommandStore
+public interface CommandStore
 {
-    public interface Factory
+    interface Factory
     {
         CommandStore create(int id,
                             NodeTimeService time,
@@ -42,20 +42,9 @@
                             RangesForEpochHolder rangesForEpoch);
     }
 
-    private final int id; // unique id
-
-    public CommandStore(int id)
-    {
-        this.id = id;
-    }
-
-    public int id()
-    {
-        return id;
-    }
-
-    public abstract Agent agent();
-    public abstract AsyncChain<Void> execute(PreLoadContext context, Consumer<? super SafeCommandStore> consumer);
-    public abstract <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> apply);
-    public abstract void shutdown();
+    int id();
+    Agent agent();
+    AsyncChain<Void> execute(PreLoadContext context, Consumer<? super SafeCommandStore> consumer);
+    <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> apply);
+    void shutdown();
 }
diff --git a/accord-core/src/main/java/accord/local/CommandStores.java b/accord-core/src/main/java/accord/local/CommandStores.java
index 46471fb..67b4f05 100644
--- a/accord-core/src/main/java/accord/local/CommandStores.java
+++ b/accord-core/src/main/java/accord/local/CommandStores.java
@@ -35,7 +35,10 @@
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiFunction;
 import java.util.function.Consumer;
+import java.util.function.Function;
 import java.util.stream.IntStream;
 
 import static accord.local.PreLoadContext.empty;
@@ -287,15 +290,6 @@
         return new Snapshot(result.toArray(new ShardHolder[0]), newLocalTopology, newTopology);
     }
 
-    interface MapReduceAdapter<S extends CommandStore, Intermediate, Accumulator, O>
-    {
-        Accumulator allocate();
-        Intermediate apply(MapReduce<? super SafeCommandStore, O> map, S commandStore, PreLoadContext context);
-        Accumulator reduce(MapReduce<? super SafeCommandStore, O> reduce, Accumulator accumulator, Intermediate next);
-        void consume(MapReduceConsume<?, O> consume, Intermediate reduced);
-        Intermediate reduce(MapReduce<?, O> reduce, Accumulator accumulator);
-    }
-
     public AsyncChain<Void> forEach(Consumer<SafeCommandStore> forEach)
     {
         List<AsyncChain<Void>> list = new ArrayList<>();
@@ -341,7 +335,7 @@
 
                 return null;
             }
-        }, AsyncCommandStores.AsyncMapReduceAdapter.instance());
+        });
     }
 
     /**
@@ -359,29 +353,24 @@
      * Note that {@code reduce} and {@code accept} are invoked by only one thread, and never concurrently with {@code apply},
      * so they do not require mutual exclusion.
      *
-     * Implementations are expected to invoke {@link #mapReduceConsume(PreLoadContext, Routables, long, long, MapReduceConsume, MapReduceAdapter)}
+     * Implementations are expected to invoke {@link #mapReduceConsume(PreLoadContext, Routables, long, long, MapReduceConsume)}
      */
-    public abstract <O> void mapReduceConsume(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume);
-    public abstract <O> void mapReduceConsume(PreLoadContext context, IntStream commandStoreIds, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume);
-
-    protected <T1, T2, O> void mapReduceConsume(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume,
-                                                MapReduceAdapter<? super S, T1, T2, O> adapter)
+    protected <O> void mapReduceConsume(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
     {
-        T1 reduced = mapReduce(context, keys, minEpoch, maxEpoch, mapReduceConsume, adapter);
-        adapter.consume(mapReduceConsume, reduced);
+        AsyncChain<O> reduced = mapReduce(context, keys, minEpoch, maxEpoch, mapReduceConsume);
+        reduced.begin(mapReduceConsume);
     }
 
-    protected <T1, T2, O> void mapReduceConsume(PreLoadContext context, IntStream commandStoreIds, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume,
-                                                   MapReduceAdapter<? super S, T1, T2, O> adapter)
+    public  <O> void mapReduceConsume(PreLoadContext context, IntStream commandStoreIds, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
     {
-        T1 reduced = mapReduce(context, commandStoreIds, mapReduceConsume, adapter);
-        adapter.consume(mapReduceConsume, reduced);
+        AsyncChain<O> reduced = mapReduce(context, commandStoreIds, mapReduceConsume);
+        reduced.begin(mapReduceConsume);
     }
 
-    protected <T1, T2, O> T1 mapReduce(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduce<? super SafeCommandStore, O> mapReduce,
-                                       MapReduceAdapter<? super S, T1, T2, O> adapter)
+    public <O> AsyncChain<O> mapReduce(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduce<? super SafeCommandStore, O> mapReduce)
     {
-        T2 accumulator = adapter.allocate();
+        AsyncChain<O> chain = null;
+        BiFunction<O, O, O> reducer = mapReduce::reduce;
         Snapshot snapshot = current;
         ShardHolder[] shards = snapshot.shards;
         for (ShardHolder shard : shards)
@@ -391,24 +380,29 @@
             if (!shardRanges.intersects(keys))
                 continue;
 
-            T1 next = adapter.apply(mapReduce, (S)shard.store, context);
-            accumulator = adapter.reduce(mapReduce, accumulator, next);
+            AsyncChain<O> next = shard.store.submit(context, mapReduce);
+            chain = chain != null ? AsyncChains.reduce(chain, next, reducer) : next;
         }
-        return adapter.reduce(mapReduce, accumulator);
+        if (chain == null)
+            return AsyncChains.success(null);
+        return chain;
     }
 
-    protected <T1, T2, O> T1 mapReduce(PreLoadContext context, IntStream commandStoreIds, MapReduce<? super SafeCommandStore, O> mapReduce,
-                                       MapReduceAdapter<? super S, T1, T2, O> adapter)
+    protected <O> AsyncChain<O> mapReduce(PreLoadContext context, IntStream commandStoreIds, MapReduce<? super SafeCommandStore, O> mapReduce)
     {
         // TODO (low priority, efficiency): avoid using an array, or use a scratch buffer
         int[] ids = commandStoreIds.toArray();
-        T2 accumulator = adapter.allocate();
+        AsyncChain<O> chain = null;
+        BiFunction<O, O, O> reducer = mapReduce::reduce;
         for (int id : ids)
         {
-            T1 next = adapter.apply(mapReduce, (S)forId(id), context);
-            accumulator = adapter.reduce(mapReduce, accumulator, next);
+            CommandStore commandStore = forId(id);
+            AsyncChain<O> next = commandStore.submit(context, mapReduce);
+            chain = chain != null ? AsyncChains.reduce(chain, next, reducer) : next;
         }
-        return adapter.reduce(mapReduce, accumulator);
+        if (chain == null)
+            return AsyncChains.success(null);
+        return chain;
     }
 
     public synchronized void updateTopology(Topology newTopology)
diff --git a/accord-core/src/main/java/accord/local/Commands.java b/accord-core/src/main/java/accord/local/Commands.java
new file mode 100644
index 0000000..e124462
--- /dev/null
+++ b/accord-core/src/main/java/accord/local/Commands.java
@@ -0,0 +1,1094 @@
+/*
+ * 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 accord.api.ProgressLog.ProgressShard;
+import accord.api.Result;
+import accord.api.RoutingKey;
+import accord.local.Command.WaitingOn;
+import accord.primitives.*;
+import accord.utils.Invariants;
+import accord.utils.async.AsyncChain;
+import accord.utils.async.AsyncChains;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.function.Consumer;
+
+import static accord.api.ProgressLog.ProgressShard.*;
+import static accord.api.ProgressLog.ProgressShard.Local;
+import static accord.local.Commands.EnsureAction.*;
+import static accord.local.Status.*;
+import static accord.local.Status.Known.ExecuteAtOnly;
+import static accord.primitives.Route.isFullRoute;
+
+public class Commands
+{
+    private static final Logger logger = LoggerFactory.getLogger(Commands.class);
+
+    private Commands()
+    {
+    }
+
+    private static Ranges covers(@Nullable PartialTxn txn)
+    {
+        return txn == null ? null : txn.covering();
+    }
+
+    private static Ranges covers(@Nullable PartialDeps deps)
+    {
+        return deps == null ? null : deps.covering;
+    }
+
+    private static boolean hasQuery(PartialTxn txn)
+    {
+        return txn != null && txn.query() != null;
+    }
+
+    /**
+     * true iff this commandStore owns the given key on the given epoch
+     */
+    public static boolean owns(SafeCommandStore safeStore, long epoch, RoutingKey someKey)
+    {
+        return safeStore.ranges().at(epoch).contains(someKey);
+    }
+
+    public static RoutingKey noProgressKey()
+    {
+        return NO_PROGRESS_KEY;
+    }
+
+    public enum AcceptOutcome {Success, Redundant, RejectedBallot}
+
+    public static AcceptOutcome preaccept(SafeCommandStore safeStore, TxnId txnId, PartialTxn partialTxn, Route<?> route, @Nullable RoutingKey progressKey)
+    {
+        return preacceptOrRecover(safeStore, txnId, partialTxn, route, progressKey, Ballot.ZERO);
+    }
+
+    public static AcceptOutcome recover(SafeCommandStore safeStore, TxnId txnId, PartialTxn partialTxn, Route<?> route, @Nullable RoutingKey progressKey, Ballot ballot)
+    {
+        return preacceptOrRecover(safeStore, txnId, partialTxn, route, progressKey, ballot);
+    }
+
+    private static AcceptOutcome preacceptOrRecover(SafeCommandStore safeStore, TxnId txnId, PartialTxn partialTxn, Route<?> route, @Nullable RoutingKey progressKey, Ballot ballot)
+    {
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Command command = safeCommand.current();
+
+        int compareBallots = command.promised().compareTo(ballot);
+        if (compareBallots > 0)
+        {
+            logger.trace("{}: skipping preaccept - higher ballot witnessed ({})", txnId, command.promised());
+            return AcceptOutcome.RejectedBallot;
+        }
+
+        if (command.known().definition.isKnown())
+        {
+            Invariants.checkState(command.status() == Invalidated || command.executeAt() != null);
+            logger.trace("{}: skipping preaccept - already known ({})", txnId, command.status());
+            // in case of Ballot.ZERO, we must either have a competing recovery coordinator or have late delivery of the
+            // preaccept; in the former case we should abandon coordination, and in the latter we have already completed
+            safeCommand.updatePromised(ballot);
+            return ballot.equals(Ballot.ZERO) ? AcceptOutcome.Redundant : AcceptOutcome.Success;
+        }
+
+        Ranges coordinateRanges = coordinateRanges(safeStore, command);
+        Invariants.checkState(!coordinateRanges.isEmpty());
+        CommonAttributes attrs = updateHomeAndProgressKeys(safeStore, command.txnId(), command, route, progressKey, coordinateRanges);
+        ProgressShard shard = progressShard(attrs, progressKey, coordinateRanges);
+        if (!validate(command.status(), attrs, Ranges.EMPTY, coordinateRanges, shard, route, Set, partialTxn, Set, null, Ignore))
+            throw new IllegalStateException();
+
+        // FIXME: this should go into a consumer method
+        attrs = set(safeStore, command, attrs, Ranges.EMPTY, coordinateRanges, 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:
+            //  - use a global logical clock to issue new timestamps; or
+            //  - assign each shard _and_ process a unique id, and use both as components of the timestamp
+            // 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
+            Timestamp executeAt = ballot.equals(Ballot.ZERO)
+                    ? safeStore.preaccept(txnId, partialTxn.keys())
+                    : safeStore.time().uniqueNow(txnId);
+            command = safeCommand.preaccept(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);
+        }
+
+        safeStore.notifyListeners(safeCommand);
+        return AcceptOutcome.Success;
+    }
+
+    public static boolean preacceptInvalidate(SafeCommandStore safeStore, TxnId txnId, Ballot ballot)
+    {
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Command command = safeCommand.current();
+        if (command.promised().compareTo(ballot) > 0)
+        {
+            logger.trace("{}: skipping preacceptInvalidate - witnessed higher ballot ({})", command.txnId(), command.promised());
+            return false;
+        }
+        safeCommand.updatePromised(ballot);
+        return true;
+    }
+
+    public static AcceptOutcome accept(SafeCommandStore safeStore, TxnId txnId, Ballot ballot, PartialRoute<?> route, Seekables<?, ?> keys, @Nullable RoutingKey progressKey, Timestamp executeAt, PartialDeps partialDeps)
+    {
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Command command = safeCommand.current();
+        if (command.promised().compareTo(ballot) > 0)
+        {
+            logger.trace("{}: skipping accept - witnessed higher ballot ({} > {})", txnId, command.promised(), ballot);
+            return AcceptOutcome.RejectedBallot;
+        }
+
+        if (command.hasBeen(PreCommitted))
+        {
+            logger.trace("{}: skipping accept - already committed ({})", txnId, command.status());
+            return AcceptOutcome.Redundant;
+        }
+
+        Ranges coordinateRanges = coordinateRanges(safeStore, command);
+        Ranges acceptRanges = txnId.epoch() == executeAt.epoch() ? coordinateRanges : safeStore.ranges().between(txnId.epoch(), executeAt.epoch());
+        Invariants.checkState(!acceptRanges.isEmpty());
+
+        CommonAttributes attrs = updateHomeAndProgressKeys(safeStore, command.txnId(), command, route, progressKey, coordinateRanges);
+        ProgressShard shard = progressShard(attrs, progressKey, coordinateRanges);
+        if (!validate(command.status(), attrs, coordinateRanges, Ranges.EMPTY, shard, route, Ignore, null, Ignore, partialDeps, Set))
+        {
+            throw new AssertionError("Invalid response from validate function");
+        }
+
+        // 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.
+        attrs = set(safeStore, command, attrs, 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);
+        safeStore.progressLog().accepted(command, shard);
+        safeStore.notifyListeners(safeCommand);
+
+        return AcceptOutcome.Success;
+    }
+
+    public static AcceptOutcome acceptInvalidate(SafeCommandStore safeStore, SafeCommand safeCommand, Ballot ballot)
+    {
+        Command command = safeCommand.current();
+        if (command.promised().compareTo(ballot) > 0)
+        {
+            logger.trace("{}: skipping accept invalidated - witnessed higher ballot ({} > {})", command.txnId(), command.promised(), ballot);
+            return AcceptOutcome.RejectedBallot;
+        }
+
+        if (command.hasBeen(PreCommitted))
+        {
+            logger.trace("{}: skipping accept invalidated - already committed ({})", command.txnId(), command.status());
+            return AcceptOutcome.Redundant;
+        }
+
+        logger.trace("{}: accepted invalidated", command.txnId());
+
+        safeCommand.acceptInvalidated(ballot);
+        safeStore.notifyListeners(safeCommand);
+        return AcceptOutcome.Success;
+    }
+
+    public enum CommitOutcome {Success, Redundant, Insufficient;}
+
+
+    // relies on mutual exclusion for each key
+    public static CommitOutcome commit(SafeCommandStore safeStore, TxnId txnId, Route<?> route, @Nullable RoutingKey progressKey, @Nullable PartialTxn partialTxn, Timestamp executeAt, PartialDeps partialDeps)
+    {
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Command command = safeCommand.current();
+
+        if (command.hasBeen(PreCommitted))
+        {
+            logger.trace("{}: skipping commit - already committed ({})", txnId, command.status());
+            if (!executeAt.equals(command.executeAt()) || command.status() == Invalidated)
+                safeStore.agent().onInconsistentTimestamp(command, (command.status() == Invalidated ? Timestamp.NONE : command.executeAt()), executeAt);
+
+            if (command.hasBeen(Committed))
+                return CommitOutcome.Redundant;
+        }
+
+        Ranges coordinateRanges = coordinateRanges(safeStore, command);
+        // TODO (expected, consider): consider ranges between coordinateRanges and executeRanges? Perhaps don't need them
+        Ranges executeRanges = executeRanges(safeStore, executeAt);
+
+        CommonAttributes attrs = updateHomeAndProgressKeys(safeStore, command.txnId(), command, route, progressKey, coordinateRanges);
+        ProgressShard shard = progressShard(attrs, progressKey, coordinateRanges);
+
+        if (!validate(command.status(), attrs, coordinateRanges, executeRanges, shard, route, Check, partialTxn, Add, partialDeps, Set))
+        {
+            safeCommand.updateAttributes(attrs);
+            return CommitOutcome.Insufficient;
+        }
+
+        // FIXME: split up set
+        attrs = set(safeStore, command, attrs, coordinateRanges, executeRanges, shard, route, partialTxn, Add, partialDeps, Set);
+
+        logger.trace("{}: committed with executeAt: {}, deps: {}", txnId, executeAt, partialDeps);
+        WaitingOn waitingOn = populateWaitingOn(safeStore, txnId, executeAt, partialDeps);
+        command = safeCommand.commit(attrs, executeAt, waitingOn);
+
+        safeStore.progressLog().committed(command, shard);
+
+        // TODO (expected, safety): introduce intermediate status to avoid reentry when notifying listeners (which might notify us)
+        maybeExecute(safeStore, safeCommand, shard, true, true);
+        return CommitOutcome.Success;
+    }
+
+    // relies on mutual exclusion for each key
+    public static void precommit(SafeCommandStore safeStore, TxnId txnId, Timestamp executeAt)
+    {
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Command command = safeCommand.current();
+        if (command.hasBeen(PreCommitted))
+        {
+            logger.trace("{}: skipping precommit - already committed ({})", txnId, command.status());
+            if (executeAt.equals(command.executeAt()) && command.status() != Invalidated)
+                return;
+
+            safeStore.agent().onInconsistentTimestamp(command, (command.status() == Invalidated ? Timestamp.NONE : command.executeAt()), executeAt);
+        }
+
+        safeCommand.precommit(executeAt);
+        safeStore.notifyListeners(safeCommand);
+        logger.trace("{}: precommitted with executeAt: {}", txnId, executeAt);
+    }
+
+    protected static WaitingOn populateWaitingOn(SafeCommandStore safeStore, TxnId txnId, Timestamp executeAt, PartialDeps partialDeps)
+    {
+        Ranges ranges = safeStore.ranges().since(executeAt.epoch());
+        if (ranges == null)
+            return WaitingOn.EMPTY;
+
+        WaitingOn.Update update = new WaitingOn.Update();
+        partialDeps.forEach(ranges, depId -> {
+            SafeCommand safeCommand = safeStore.ifLoaded(depId);
+            if (safeCommand == null)
+            {
+                update.addWaitingOnCommit(depId);
+                safeStore.addAndInvokeListener(depId, txnId);
+            }
+            else
+            {
+                Command command = safeCommand.current();
+                switch (command.status())
+                {
+                    default:
+                        throw new IllegalStateException();
+                    case NotWitnessed:
+                    case PreAccepted:
+                    case Accepted:
+                    case AcceptedInvalidate:
+                    case PreCommitted:
+                        // we don't know when these dependencies will execute, and cannot execute until we do
+
+                        command = safeCommand.addListener(new Command.Listener(txnId));
+                        update.addWaitingOnCommit(command.txnId());
+                        break;
+                    case Committed:
+                        // TODO (desired, efficiency): split into ReadyToRead and ReadyToWrite;
+                        //                             the distributed read can be performed as soon as those keys are ready,
+                        //                             and in parallel with any other reads. the client can even ACK immediately after;
+                        //                             only the write needs to be postponed until other in-progress reads complete
+                    case ReadyToExecute:
+                    case PreApplied:
+                    case Applied:
+                        command = safeCommand.addListener(new Command.Listener(txnId));
+                        insertPredecessor(txnId, executeAt, update, command);
+                    case Invalidated:
+                        break;
+                }
+            }
+        });
+        return update.build();
+    }
+
+    // TODO (expected, ?): commitInvalidate may need to update cfks _if_ possible
+    public static void commitInvalidate(SafeCommandStore safeStore, TxnId txnId)
+    {
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Command command = safeCommand.current();
+        if (command.hasBeen(PreCommitted))
+        {
+            logger.trace("{}: skipping commit invalidated - already committed ({})", txnId, command.status());
+            if (!command.hasBeen(Invalidated))
+                safeStore.agent().onInconsistentTimestamp(command, Timestamp.NONE, command.executeAt());
+
+            return;
+        }
+
+        ProgressShard shard = progressShard(safeStore, command);
+        safeStore.progressLog().invalidated(command, shard);
+
+        CommonAttributes attrs = command;
+        if (command.partialDeps() == null)
+            attrs = attrs.mutable().partialDeps(PartialDeps.NONE);
+        safeCommand.commitInvalidated(attrs, txnId);
+        logger.trace("{}: committed invalidated", txnId);
+
+        safeStore.notifyListeners(safeCommand);
+    }
+
+    public enum ApplyOutcome {Success, Redundant, Insufficient}
+
+
+    public static ApplyOutcome apply(SafeCommandStore safeStore, TxnId txnId, long untilEpoch, Route<?> route, Timestamp executeAt, @Nullable PartialDeps partialDeps, Writes writes, Result result)
+    {
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Command command = safeCommand.current();
+        if (command.hasBeen(PreApplied) && executeAt.equals(command.executeAt()))
+        {
+            logger.trace("{}: skipping apply - already executed ({})", txnId, command.status());
+            return ApplyOutcome.Redundant;
+        }
+        else if (command.hasBeen(PreCommitted) && !executeAt.equals(command.executeAt()))
+        {
+            safeStore.agent().onInconsistentTimestamp(command, command.executeAt(), executeAt);
+        }
+
+        Ranges coordinateRanges = coordinateRanges(safeStore, command);
+        Ranges executeRanges = executeRanges(safeStore, executeAt);
+        if (untilEpoch < safeStore.latestEpoch())
+        {
+            Ranges expectedRanges = safeStore.ranges().between(executeAt.epoch(), untilEpoch);
+            Invariants.checkState(expectedRanges.containsAll(executeRanges));
+        }
+
+        CommonAttributes attrs = updateHomeAndProgressKeys(safeStore, command.txnId(), command, route, coordinateRanges);
+        ProgressShard shard = progressShard(attrs, coordinateRanges);
+
+        if (!validate(command.status(), attrs, coordinateRanges, executeRanges, shard, route, Check, null, Check, partialDeps, command.hasBeen(Committed) ? Add : TrySet))
+        {
+            safeCommand.updateAttributes(attrs);
+            return ApplyOutcome.Insufficient; // TODO (expected, consider): this should probably be an assertion failure if !TrySet
+        }
+
+        WaitingOn waitingOn = !command.hasBeen(Committed) ? populateWaitingOn(safeStore, txnId, executeAt, partialDeps) : command.asCommitted().waitingOn();
+        attrs = set(safeStore, command, attrs, coordinateRanges, executeRanges, shard, route, null, Check, partialDeps, command.hasBeen(Committed) ? Add : TrySet);
+
+        safeCommand.preapplied(attrs, executeAt, waitingOn, writes, result);
+        logger.trace("{}: apply, status set to Executed with executeAt: {}, deps: {}", txnId, executeAt, partialDeps);
+
+        maybeExecute(safeStore, safeCommand, shard, true, true);
+        safeStore.progressLog().executed(safeCommand.current(), shard);
+
+        return ApplyOutcome.Success;
+    }
+
+    public static void listenerUpdate(SafeCommandStore safeStore, SafeCommand safeListener, SafeCommand safeUpdated)
+    {
+        Command listener = safeListener.current();
+        Command updated = safeUpdated.current();
+        logger.trace("{}: updating as listener in response to change on {} with status {} ({})",
+                     listener.txnId(), updated.txnId(), updated.status(), updated);
+        switch (updated.status())
+        {
+            default:
+                throw new IllegalStateException("Unexpected status: " + updated.status());
+            case NotWitnessed:
+            case PreAccepted:
+            case Accepted:
+            case AcceptedInvalidate:
+                break;
+
+            case PreCommitted:
+            case Committed:
+            case ReadyToExecute:
+            case PreApplied:
+            case Applied:
+            case Invalidated:
+                updatePredecessorAndMaybeExecute(safeStore, safeListener, safeUpdated, true);
+                break;
+        }
+    }
+
+    protected static void postApply(SafeCommandStore safeStore, TxnId txnId)
+    {
+        logger.trace("{} applied, setting status to Applied and notifying listeners", txnId);
+        SafeCommand safeCommand = safeStore.command(txnId);
+        safeCommand.applied();
+        safeStore.notifyListeners(safeCommand);
+    }
+
+    private static AsyncChain<Void> applyChain(SafeCommandStore safeStore, PreLoadContext context, TxnId txnId)
+    {
+        Command.Executed command = safeStore.command(txnId).current().asExecuted();
+        if (command.hasBeen(Applied))
+            return AsyncChains.success(null);
+        CommandStore unsafeStore = safeStore.commandStore();
+        return command.writes().apply(safeStore)
+               .flatMap(unused -> unsafeStore.submit(context, ss -> {
+                   postApply(ss, txnId);
+                   return null;
+               }));
+    }
+
+    private static void apply(SafeCommandStore safeStore, Command.Executed command)
+    {
+        CommandStore unsafeStore = safeStore.commandStore();
+        TxnId txnId = command.txnId();
+        PreLoadContext context = command.contextForSelf();
+        // this is sometimes called from a listener update, which will not have the keys in context
+        if (safeStore.canExecuteWith(context))
+        {
+            applyChain(safeStore, context, txnId).begin(safeStore.agent());
+        }
+        else
+        {
+            unsafeStore.submit(context, ss -> {
+                applyChain(ss, context, txnId).begin(ss.agent());
+                return null;
+            }).begin(safeStore.agent());
+        }
+    }
+
+    // TODO (expected, API consistency): maybe split into maybeExecute and maybeApply?
+    private static boolean maybeExecute(SafeCommandStore safeStore, SafeCommand safeCommand, ProgressShard shard, boolean alwaysNotifyListeners, boolean notifyWaitingOn)
+    {
+        Command command = safeCommand.current();
+        if (logger.isTraceEnabled())
+            logger.trace("{}: Maybe executing with status {}. Will notify listeners on noop: {}", command.txnId(), command.status(), alwaysNotifyListeners);
+
+        if (command.status() != Committed && command.status() != PreApplied)
+        {
+            if (alwaysNotifyListeners)
+                safeStore.notifyListeners(safeCommand);
+            return false;
+        }
+
+        if (command.asCommitted().isWaitingOnDependency())
+        {
+            if (alwaysNotifyListeners)
+                safeStore.notifyListeners(safeCommand);
+
+            if (notifyWaitingOn)
+                new NotifyWaitingOn(command.txnId()).accept(safeStore);
+            return false;
+        }
+
+        // FIXME: need to communicate to caller that we didn't execute if we take one of the above paths
+
+        switch (command.status())
+        {
+            case Committed:
+                // TODO (desirable, efficiency): maintain distinct ReadyToRead and ReadyToWrite states
+                command = safeCommand.readyToExecute();
+                logger.trace("{}: set to ReadyToExecute", command.txnId());
+                safeStore.progressLog().readyToExecute(command, shard);
+                safeStore.notifyListeners(safeCommand);
+                return true;
+
+            case PreApplied:
+                Ranges executeRanges = executeRanges(safeStore, command.executeAt());
+                Command.Executed executed = command.asExecuted();
+                boolean intersects = executed.writes().keys.intersects(executeRanges);
+
+                if (intersects)
+                {
+                    logger.trace("{}: applying", command.txnId());
+                    apply(safeStore, executed);
+                    return true;
+                }
+                else
+                {
+                    // 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();
+                    safeStore.notifyListeners(safeCommand);
+                    return true;
+                }
+            default:
+                throw new IllegalStateException();
+        }
+    }
+
+    /**
+     * @param safeDependency is either committed or invalidated
+     * @return true iff {@code maybeExecute} might now have a different outcome
+     */
+    private static boolean updatePredecessor(SafeCommand safeCommand, WaitingOn.Update waitingOn, SafeCommand safeDependency)
+    {
+        Command.Committed command = safeCommand.current().asCommitted();
+        Command dependency = safeDependency.current();
+        Invariants.checkState(dependency.hasBeen(PreCommitted));
+        if (dependency.hasBeen(Invalidated))
+        {
+            logger.trace("{}: {} is invalidated. Stop listening and removing from waiting on commit set.", command.txnId(), dependency.txnId());
+            safeDependency.removeListener(command.asListener());
+            waitingOn.removeWaitingOnCommit(dependency.txnId());
+            return true;
+        }
+        else if (dependency.executeAt().compareTo(command.executeAt()) > 0)
+        {
+            // dependency cannot be a predecessor if it executes later
+            logger.trace("{}: {} executes after us. Stop listening and removing from waiting on apply set.", command.txnId(), dependency.txnId());
+            waitingOn.removeWaitingOn(dependency.txnId(), dependency.executeAt());
+            safeDependency.removeListener(command.asListener());
+            return true;
+        }
+        else if (dependency.hasBeen(Applied))
+        {
+            logger.trace("{}: {} has been applied. Stop listening and removing from waiting on apply set.", command.txnId(), dependency.txnId());
+            waitingOn.removeWaitingOn(dependency.txnId(), dependency.executeAt());
+            safeDependency.removeListener(command.asListener());
+            return true;
+        }
+        else if (command.isWaitingOnDependency())
+        {
+            logger.trace("{}: adding {} to waiting on apply set.", command.txnId(), dependency.txnId());
+            waitingOn.addWaitingOnApply(dependency.txnId(), dependency.executeAt());
+            waitingOn.removeWaitingOnCommit(dependency.txnId());
+            return false;
+        }
+        else
+        {
+            throw new IllegalStateException();
+        }
+    }
+
+    private static void insertPredecessor(TxnId txnId, Timestamp executeAt, WaitingOn.Update update, Command dependency)
+    {
+        Invariants.checkState(dependency.hasBeen(Committed));
+        if (dependency.hasBeen(Invalidated))
+        {
+            logger.trace("{}: {} is invalidated. Do not insert.", txnId, dependency.txnId());
+        }
+        else if (dependency.executeAt().compareTo(executeAt) > 0)
+        {
+            // dependency cannot be a predecessor if it executes later
+            logger.trace("{}: {} executes after us. Do not insert.", txnId, dependency.txnId());
+        }
+        else if (dependency.hasBeen(Applied))
+        {
+            logger.trace("{}: {} has been applied. Do not insert.", txnId, dependency.txnId());
+        }
+        else
+        {
+            logger.trace("{}: adding {} to waiting on apply set.", txnId, dependency.txnId());
+            update.addWaitingOnApply(dependency.txnId(), dependency.executeAt());
+        }
+    }
+
+    static void updatePredecessorAndMaybeExecute(SafeCommandStore safeStore, SafeCommand safeCommand, SafeCommand livePredecessor, boolean notifyWaitingOn)
+    {
+        Command.Committed command = safeCommand.current().asCommitted();
+        if (command.hasBeen(Applied))
+            return;
+
+        WaitingOn.Update waitingOn = new WaitingOn.Update(command);
+        boolean attemptExecution = updatePredecessor(safeCommand, waitingOn, livePredecessor);
+        command = safeCommand.updateWaitingOn(waitingOn);
+
+        if (attemptExecution)
+            maybeExecute(safeStore, safeCommand, progressShard(safeStore, command), false, notifyWaitingOn);
+    }
+
+    // TODO (now): check/move methods below
+    private static Command setDurability(SafeCommandStore safeStore, SafeCommand safeCommand, Durability durability, RoutingKey homeKey, @Nullable Timestamp executeAt)
+    {
+        Command command = safeCommand.current();
+        CommonAttributes attrs = updateHomeKey(safeStore, command.txnId(), command, homeKey);
+        if (executeAt != null && command.status().hasBeen(Committed) && !command.asCommitted().executeAt().equals(executeAt))
+            safeStore.agent().onInconsistentTimestamp(command, command.asCommitted().executeAt(), executeAt);
+        attrs = attrs.mutable().durability(durability);
+        return safeCommand.updateAttributes(attrs);
+    }
+
+    public static Command setDurability(SafeCommandStore safeStore, TxnId txnId, Durability durability, RoutingKey homeKey, @Nullable Timestamp executeAt)
+    {
+        return setDurability(safeStore, safeStore.command(txnId), durability, homeKey, executeAt);
+    }
+
+    private static TxnId firstWaitingOnCommit(Command command)
+    {
+        if (!command.hasBeen(Committed))
+            return null;
+
+        Command.Committed committed = command.asCommitted();
+        return committed.isWaitingOnCommit() ? committed.waitingOnCommit().first() : null;
+    }
+
+    private static TxnId firstWaitingOnApply(Command command, @Nullable TxnId ifExecutesBefore)
+    {
+        if (!command.hasBeen(Committed))
+            return null;
+
+        Command.Committed committed = command.asCommitted();
+        if (!committed.isWaitingOnApply())
+            return null;
+
+        Map.Entry<Timestamp, TxnId> first = committed.waitingOnApply().firstEntry();
+        if (ifExecutesBefore == null || first.getKey().compareTo(ifExecutesBefore) < 0)
+            return first.getValue();
+
+        return null;
+    }
+
+    static class NotifyWaitingOn implements PreLoadContext, Consumer<SafeCommandStore>
+    {
+        Known[] blockedUntil = new Known[4];
+        TxnId[] txnIds = new TxnId[4];
+        int depth;
+
+        public NotifyWaitingOn(TxnId txnId)
+        {
+            txnIds[0] = txnId;
+            blockedUntil[0] = Known.Done;
+        }
+
+        @Override
+        public void accept(SafeCommandStore safeStore)
+        {
+            SafeCommand prevSafe = get(safeStore, depth - 1);
+            while (depth >= 0)
+            {
+                Command prev = prevSafe != null ? prevSafe.current() : null;
+                SafeCommand curSafe = safeStore.ifLoaded(txnIds[depth]);
+                Command cur = curSafe != null ? curSafe.current() : null;
+                Known until = blockedUntil[depth];
+                if (cur == null)
+                {
+                    // need to load; schedule execution for later
+                    safeStore.commandStore().execute(this, this).begin(safeStore.agent());
+                    return;
+                }
+
+                if (prev != null)
+                {
+                    if (cur.has(until) || (cur.hasBeen(PreCommitted) && cur.executeAt().compareTo(prev.executeAt()) > 0))
+                    {
+                        updatePredecessorAndMaybeExecute(safeStore, prevSafe, curSafe, false);
+                        --depth;
+                        prevSafe = get(safeStore, depth - 1);
+                        continue;
+                    }
+                }
+                else if (cur.has(until))
+                {
+                    // we're done; have already applied
+                    Invariants.checkState(depth == 0);
+                    break;
+                }
+
+                TxnId directlyBlockedOnCommit = firstWaitingOnCommit(cur);
+                TxnId directlyBlockedOnApply = firstWaitingOnApply(cur, directlyBlockedOnCommit);
+                if (directlyBlockedOnApply != null)
+                {
+                    push(directlyBlockedOnApply, Known.Done);
+                }
+                else if (directlyBlockedOnCommit != null)
+                {
+                    push(directlyBlockedOnCommit, ExecuteAtOnly);
+                }
+                else
+                {
+                    if (cur.hasBeen(Committed) && !cur.hasBeen(ReadyToExecute) && !cur.asCommitted().isWaitingOnDependency())
+                    {
+                        if (!maybeExecute(safeStore, curSafe, progressShard(safeStore, cur), false, false))
+                            throw new AssertionError("Is able to Apply, but has not done so");
+                        // loop and re-test the command's status; we may still want to notify blocking, esp. if not homeShard
+                        continue;
+                    }
+
+                    Unseekables<?, ?> someKeys = cur.maxUnseekables();
+                    if (someKeys == null && prev != null) someKeys = prev.partialDeps().someUnseekables(cur.txnId());
+                    Invariants.checkState(someKeys != null);
+                    logger.trace("{} blocked on {} until {}", txnIds[0], cur.txnId(), until);
+                    safeStore.progressLog().waiting(cur.txnId(), until, someKeys);
+                    return;
+                }
+                prevSafe = curSafe;
+            }
+        }
+
+        private SafeCommand get(SafeCommandStore safeStore, int i)
+        {
+            return i >= 0 ? safeStore.command(txnIds[i]) : null;
+        }
+
+        void push(TxnId by, Known until)
+        {
+            if (++depth == txnIds.length)
+            {
+                txnIds = Arrays.copyOf(txnIds, txnIds.length * 2);
+                blockedUntil = Arrays.copyOf(blockedUntil, txnIds.length);
+            }
+            txnIds[depth] = by;
+            blockedUntil[depth] = until;
+        }
+
+        @Override
+        public Iterable<TxnId> txnIds()
+        {
+            return Arrays.asList(txnIds).subList(0, depth + 1);
+        }
+
+        @Override
+        public Seekables<?, ?> keys()
+        {
+            return Keys.EMPTY;
+        }
+    }
+
+    public static Command updateHomeKey(SafeCommandStore safeStore, SafeCommand safeCommand, RoutingKey homeKey)
+    {
+        Command command = safeCommand.current();
+        CommonAttributes attrs = updateHomeKey(safeStore, command.txnId(), command, homeKey);
+        return safeCommand.updateAttributes(attrs);
+    }
+
+    /**
+     * A key nominated to represent the "home" shard - only members of the home shard may be nominated to recover
+     * a transaction, to reduce the cluster-wide overhead of ensuring progress. A transaction that has only been
+     * witnessed at PreAccept may however trigger a process of ensuring the home shard is durably informed of
+     * the transaction.
+     *
+     * Note that for ProgressLog purposes the "home shard" is the shard as of txnId.epoch.
+     * For recovery purposes the "home shard" is as of txnId.epoch until Committed, and executeAt.epoch once Executed
+     */
+    public static CommonAttributes updateHomeKey(SafeCommandStore safeStore, TxnId txnId, CommonAttributes attrs, RoutingKey homeKey)
+    {
+        if (attrs.homeKey() == null)
+        {
+            attrs = attrs.mutable().homeKey(homeKey);
+            // TODO (low priority, safety): if we're processed on a node that does not know the latest epoch,
+            //      do we guarantee the home key calculation is unchanged since the prior epoch?
+            if (attrs.progressKey() == null && owns(safeStore, txnId.epoch(), homeKey))
+                attrs = attrs.mutable().progressKey(homeKey);
+        }
+        else if (!attrs.homeKey().equals(homeKey))
+        {
+            throw new IllegalStateException();
+        }
+        return attrs;
+    }
+
+    private static CommonAttributes updateHomeAndProgressKeys(SafeCommandStore safeStore, TxnId txnId, CommonAttributes attrs, Route<?> route, @Nullable RoutingKey progressKey, Ranges coordinateRanges)
+    {
+        attrs = updateHomeKey(safeStore, txnId, attrs, route.homeKey());
+        if (progressKey == null || progressKey == NO_PROGRESS_KEY)
+        {
+            if (attrs.progressKey() == null)
+                attrs = attrs.mutable().progressKey(NO_PROGRESS_KEY);
+            return attrs;
+        }
+        if (attrs.progressKey() == null) attrs = attrs.mutable().progressKey(progressKey);
+        else if (!attrs.progressKey().equals(progressKey))
+            throw new AssertionError();
+        return attrs;
+    }
+
+    private static CommonAttributes updateHomeAndProgressKeys(SafeCommandStore safeStore, TxnId txnId, CommonAttributes attrs, Route<?> route, Ranges coordinateRanges)
+    {
+        if (attrs.progressKey() == null)
+            return attrs;
+
+        return updateHomeAndProgressKeys(safeStore, txnId, attrs, route, attrs.progressKey(), coordinateRanges);
+    }
+
+    private static ProgressShard progressShard(CommonAttributes attrs, @Nullable RoutingKey progressKey, Ranges coordinateRanges)
+    {
+        if (progressKey == null || progressKey == NO_PROGRESS_KEY)
+            return No;
+
+        if (!coordinateRanges.contains(progressKey))
+            return No;
+
+        return progressKey.equals(attrs.homeKey()) ? Home : Local;
+    }
+
+    private static ProgressShard progressShard(CommonAttributes attrs, Ranges coordinateRanges)
+    {
+        if (attrs.progressKey() == null)
+            return Unsure;
+
+        return progressShard(attrs, attrs.progressKey(), coordinateRanges);
+    }
+
+    private static ProgressShard progressShard(SafeCommandStore safeStore, Command command)
+    {
+        RoutingKey progressKey = command.progressKey();
+        if (progressKey == null)
+            return Unsure;
+
+        if (progressKey == noProgressKey())
+            return No;
+
+        Ranges coordinateRanges = safeStore.ranges().at(command.txnId().epoch());
+        if (!coordinateRanges.contains(progressKey))
+            return No;
+
+        return progressKey.equals(command.homeKey()) ? Home : Local;
+    }
+
+
+    private static Ranges coordinateRanges(SafeCommandStore safeStore, Command command)
+    {
+        return safeStore.ranges().at(command.txnId().epoch());
+    }
+
+    private static Ranges executeRanges(SafeCommandStore safeStore, Timestamp executeAt)
+    {
+        return safeStore.ranges().since(executeAt.epoch());
+    }
+
+    enum EnsureAction {Ignore, Check, Add, TrySet, Set}
+
+    private static CommonAttributes set(SafeCommandStore safeStore, Command command, CommonAttributes attrs,
+                                        Ranges existingRanges, Ranges additionalRanges, ProgressShard shard, Route<?> route,
+                                        @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn,
+                                        @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps)
+    {
+        Invariants.checkState(attrs.progressKey() != null);
+        Ranges allRanges = existingRanges.with(additionalRanges);
+
+        if (shard.isProgress())
+            attrs = attrs.mutable().route(Route.merge(attrs.route(), (Route)route));
+        else
+            attrs = attrs.mutable().route(route.slice(allRanges));
+
+        // TODO (soon): stop round-robin hashing; partition only on ranges
+        switch (ensurePartialTxn)
+        {
+            case Add:
+                if (partialTxn == null)
+                    break;
+
+                if (attrs.partialTxn() != null)
+                {
+                    partialTxn = partialTxn.slice(allRanges, shard.isHome());
+                    Routables.foldlMissing((Seekables)partialTxn.keys(), attrs.partialTxn().keys(), (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;
+                }
+
+            case Set:
+            case TrySet:
+                attrs = attrs.mutable().partialTxn(partialTxn = partialTxn.slice(allRanges, shard.isHome()));
+                // TODO (expected, efficiency): we may register the same ranges more than once
+                // TODO (desirable, efficiency): no need to register on PreAccept if already Accepted
+                safeStore.register(partialTxn.keys(), allRanges, command);
+                break;
+        }
+
+        switch (ensurePartialDeps)
+        {
+            case Add:
+                if (partialDeps == null)
+                    break;
+
+                if (attrs.partialDeps() != null)
+                {
+                    attrs = attrs.mutable().partialDeps(attrs.partialDeps().with(partialDeps.slice(allRanges)));
+                    break;
+                }
+
+            case Set:
+            case TrySet:
+                attrs = attrs.mutable().partialDeps(partialDeps.slice(allRanges));
+                break;
+        }
+        return attrs;
+    }
+
+    /**
+     * Validate we have sufficient information for the route, partialTxn and partialDeps fields, and if so update them;
+     * otherwise return false (or throw an exception if an illegal state is encountered)
+     */
+    private static boolean validate(Status status, CommonAttributes attrs, Ranges existingRanges, Ranges additionalRanges, ProgressShard shard,
+                                    Route<?> route, EnsureAction ensureRoute,
+                                    @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn,
+                                    @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps)
+    {
+        if (shard == Unsure)
+            return false;
+
+        // first validate route
+        if (shard.isHome())
+        {
+            switch (ensureRoute)
+            {
+                default: throw new AssertionError();
+                case Check:
+                    if (!isFullRoute(attrs.route()) && !isFullRoute(route))
+                        return false;
+                case Ignore:
+                    break;
+                case Add:
+                case Set:
+                    if (!isFullRoute(route))
+                        throw new IllegalArgumentException("Incomplete route (" + route + ") sent to home shard");
+                    break;
+                case TrySet:
+                    if (!isFullRoute(route))
+                        return false;
+            }
+        }
+        else
+        {
+            // failing any of these tests is always an illegal state
+            if (!route.covers(existingRanges))
+                return false;
+
+            if (existingRanges != additionalRanges && !route.covers(additionalRanges))
+                throw new IllegalArgumentException("Incomplete route (" + route + ") provided; does not cover " + additionalRanges);
+        }
+
+        // invalid to Add deps to Accepted or AcceptedInvalidate statuses, as Committed deps are not equivalent
+        // and we may erroneously believe we have covered a wider range than we have infact covered
+        if (ensurePartialDeps == Add)
+            Invariants.checkState(status != Accepted && status != AcceptedInvalidate);
+
+        // validate new partial txn
+        if (!validate(ensurePartialTxn, existingRanges, additionalRanges, covers(attrs.partialTxn()), covers(partialTxn), "txn", partialTxn))
+            return false;
+
+        if (partialTxn != null && attrs.txnId().rw() != null && !attrs.txnId().rw().equals(partialTxn.kind()))
+            throw new IllegalArgumentException("Transaction has different kind to its TxnId");
+
+        if (shard.isHome() && ensurePartialTxn != Ignore)
+        {
+            if (!hasQuery(attrs.partialTxn()) && !hasQuery(partialTxn))
+                throw new IllegalStateException();
+        }
+
+        return validate(ensurePartialDeps, existingRanges, additionalRanges, covers(attrs.partialDeps()), covers(partialDeps), "deps", partialDeps);
+    }
+
+    // FIXME (immutable-state): has this been removed?
+    private static boolean validate(EnsureAction action, Ranges existingRanges, Ranges additionalRanges,
+                                    Ranges existing, Ranges adding, String kind, Object obj)
+    {
+        switch (action)
+        {
+            default: throw new IllegalStateException();
+            case Ignore:
+                break;
+
+            case TrySet:
+                if (adding != null)
+                {
+                    if (!adding.containsAll(existingRanges))
+                        return false;
+
+                    if (additionalRanges != existingRanges && !adding.containsAll(additionalRanges))
+                        return false;
+
+                    break;
+                }
+            case Set:
+                // failing any of these tests is always an illegal state
+                Invariants.checkState(adding != null);
+                if (!adding.containsAll(existingRanges))
+                    throw new IllegalArgumentException("Incomplete " + kind + " (" + obj + ") provided; does not cover " + existingRanges);
+
+                if (additionalRanges != existingRanges && !adding.containsAll(additionalRanges))
+                    throw new IllegalArgumentException("Incomplete " + kind + " (" + obj + ") provided; does not cover " + additionalRanges);
+                break;
+
+            case Check:
+            case Add:
+                if (adding == null)
+                {
+                    if (existing == null)
+                        return false;
+
+                    Invariants.checkState(existing.containsAll(existingRanges));
+                    if (existingRanges != additionalRanges && !existing.containsAll(additionalRanges))
+                    {
+                        if (action == Check)
+                            return false;
+
+                        throw new IllegalArgumentException("Missing additional " + kind + "; existing does not cover " + additionalRanges.difference(existingRanges));
+                    }
+                }
+                else if (existing != null)
+                {
+                    Ranges covering = adding.with(existing);
+                    Invariants.checkState(covering.containsAll(existingRanges));
+                    if (existingRanges != additionalRanges && !covering.containsAll(additionalRanges))
+                    {
+                        if (action == Check)
+                            return false;
+
+                        throw new IllegalArgumentException("Incomplete additional " + kind + " (" + obj + ") provided; does not cover " + additionalRanges.difference(existingRanges));
+                    }
+                }
+                else
+                {
+                    if (!adding.containsAll(existingRanges))
+                        return false;
+
+                    if (existingRanges != additionalRanges && !adding.containsAll(additionalRanges))
+                    {
+                        if (action == Check)
+                            return false;
+
+                        throw new IllegalArgumentException("Incomplete additional " + kind + " (" + obj + ") provided; does not cover " + additionalRanges.difference(existingRanges));
+                    }
+                }
+                break;
+        }
+
+        return true;
+    }
+
+    // TODO (low priority, API): this is an ugly hack, need to encode progress/homeKey/Route state combinations much more clearly
+    //                           (perhaps introduce encapsulating class representing each possible arrangement)
+    static class NoProgressKey implements RoutingKey
+    {
+        @Override
+        public int compareTo(@Nonnull RoutableKey that)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public Range asRange()
+        {
+            throw new UnsupportedOperationException();
+        }
+    }
+
+    private static final NoProgressKey NO_PROGRESS_KEY = new NoProgressKey();
+}
diff --git a/accord-core/src/main/java/accord/local/CommonAttributes.java b/accord-core/src/main/java/accord/local/CommonAttributes.java
new file mode 100644
index 0000000..033b526
--- /dev/null
+++ b/accord-core/src/main/java/accord/local/CommonAttributes.java
@@ -0,0 +1,201 @@
+/*
+ * 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 accord.api.RoutingKey;
+import accord.api.VisibleForImplementation;
+import accord.primitives.PartialDeps;
+import accord.primitives.PartialTxn;
+import accord.primitives.Route;
+import accord.primitives.TxnId;
+import accord.utils.Invariants;
+
+public interface CommonAttributes
+{
+    TxnId txnId();
+    Status.Durability durability();
+    RoutingKey homeKey();
+    RoutingKey progressKey();
+    Route<?> route();
+    PartialTxn partialTxn();
+    PartialDeps partialDeps();
+    Listeners.Immutable listeners();
+
+    default Mutable mutable()
+    {
+        return new Mutable(this);
+    }
+
+    class Mutable implements CommonAttributes
+    {
+        private TxnId txnId;
+        private Status.Durability durability;
+        private RoutingKey homeKey;
+        private RoutingKey progressKey;
+        private Route<?> route;
+        private PartialTxn partialTxn;
+        private PartialDeps partialDeps;
+        private Listeners listeners;
+
+        public Mutable(TxnId txnId)
+        {
+            this.txnId = txnId;
+        }
+
+        public Mutable(CommonAttributes attributes)
+        {
+            this.txnId = attributes.txnId();
+            this.durability = attributes.durability();
+            this.homeKey = attributes.homeKey();
+            this.progressKey = attributes.progressKey();
+            this.route = attributes.route();
+            this.partialTxn = attributes.partialTxn();
+            this.partialDeps = attributes.partialDeps();
+            this.listeners = attributes.listeners();
+        }
+
+        @Override
+        public Mutable mutable()
+        {
+            return this;
+        }
+
+        @Override
+        public TxnId txnId()
+        {
+            return txnId;
+        }
+
+        public Mutable txnId(TxnId txnId)
+        {
+            this.txnId = txnId;
+            return this;
+        }
+
+        @Override
+        public Status.Durability durability()
+        {
+            return durability;
+        }
+
+        public Mutable durability(Status.Durability durability)
+        {
+            this.durability = durability;
+            return this;
+        }
+
+        @Override
+        public RoutingKey homeKey()
+        {
+            return homeKey;
+        }
+
+        public Mutable homeKey(RoutingKey homeKey)
+        {
+            this.homeKey = homeKey;
+            return this;
+        }
+
+        @Override
+        public RoutingKey progressKey()
+        {
+            return progressKey;
+        }
+
+        public Mutable progressKey(RoutingKey progressKey)
+        {
+            Invariants.checkArgument(progressKey == null || progressKey.equals(progressKey));
+            this.progressKey = progressKey;
+            return this;
+        }
+
+        @Override
+        public Route<?> route()
+        {
+            return route;
+        }
+
+        public Mutable route(Route<?> route)
+        {
+            this.route = route;
+            return this;
+        }
+
+        @Override
+        public PartialTxn partialTxn()
+        {
+            return partialTxn;
+        }
+
+        public Mutable partialTxn(PartialTxn partialTxn)
+        {
+            this.partialTxn = partialTxn;
+            return this;
+        }
+
+        @Override
+        public PartialDeps partialDeps()
+        {
+            return partialDeps;
+        }
+
+        public Mutable partialDeps(PartialDeps partialDeps)
+        {
+            this.partialDeps = partialDeps;
+            return this;
+        }
+
+        @Override
+        public Listeners.Immutable listeners()
+        {
+            if (listeners == null || listeners.isEmpty())
+                return Listeners.Immutable.EMPTY;
+            if (listeners instanceof Listeners.Immutable)
+                return (Listeners.Immutable) listeners;
+            return new Listeners.Immutable(listeners);
+        }
+
+        public Mutable addListener(CommandListener listener)
+        {
+            if (listeners == null)
+                listeners = new Listeners();
+            else if (listeners instanceof Listeners.Immutable)
+                listeners = new Listeners(listeners);
+            listeners.add(listener);
+            return this;
+        }
+
+        public Mutable removeListener(CommandListener listener)
+        {
+            if (listener == null || listeners.isEmpty())
+                return this;
+            if (listeners instanceof Listeners.Immutable)
+                listeners = new Listeners(listeners);
+            listeners.remove(listener);
+            return this;
+        }
+
+        @VisibleForImplementation
+        public Mutable setListeners(Listeners.Immutable listeners)
+        {
+            this.listeners = listeners;
+            return this;
+        }
+    }
+}
diff --git a/accord-core/src/main/java/accord/local/Listeners.java b/accord-core/src/main/java/accord/local/Listeners.java
index 70faff6..45967fd 100644
--- a/accord-core/src/main/java/accord/local/Listeners.java
+++ b/accord-core/src/main/java/accord/local/Listeners.java
@@ -20,6 +20,73 @@
 
 import accord.utils.DeterministicIdentitySet;
 
+import java.util.Collection;
+import java.util.function.Predicate;
+
 public class Listeners extends DeterministicIdentitySet<CommandListener>
 {
+    public Listeners()
+    {
+    }
+
+    public Listeners(Listeners copy)
+    {
+        super(copy);
+    }
+
+    public static class Immutable extends Listeners
+    {
+        public static final Immutable EMPTY = new Immutable();
+
+        private Immutable()
+        {
+            super();
+        }
+
+        public Immutable(Listeners listeners)
+        {
+            super(listeners);
+        }
+
+        Listeners mutable()
+        {
+            return new Listeners(this);
+        }
+
+        @Override
+        public boolean add(CommandListener item)
+        {
+            throw new UnsupportedOperationException("Cannot modify immutable set");
+        }
+
+        @Override
+        public boolean remove(Object item)
+        {
+            throw new UnsupportedOperationException("Cannot modify immutable set");
+        }
+
+        @Override
+        public boolean removeAll(Collection<?> c)
+        {
+            throw new UnsupportedOperationException("Cannot modify immutable set");
+        }
+
+        @Override
+        public boolean addAll(Collection<? extends CommandListener> c)
+        {
+            throw new UnsupportedOperationException("Cannot modify immutable set");
+        }
+
+        @Override
+        public boolean retainAll(Collection<?> c)
+        {
+            throw new UnsupportedOperationException("Cannot modify immutable set");
+        }
+
+        @Override
+        public boolean removeIf(Predicate<? super CommandListener> filter)
+        {
+            throw new UnsupportedOperationException("Cannot modify immutable set");
+        }
+    }
 }
diff --git a/accord-core/src/main/java/accord/local/Node.java b/accord-core/src/main/java/accord/local/Node.java
index 235faf5..344e10b 100644
--- a/accord-core/src/main/java/accord/local/Node.java
+++ b/accord-core/src/main/java/accord/local/Node.java
@@ -204,7 +204,7 @@
     }
 
     @Inline
-    public <T> AsyncResult<T> withEpoch(long epoch, Supplier<AsyncResult<T>> supplier)
+    public <T> AsyncChain<T> withEpoch(long epoch, Supplier<? extends AsyncChain<T>> supplier)
     {
         if (topology.hasEpoch(epoch))
         {
@@ -213,7 +213,7 @@
         else
         {
             configService.fetchTopologyForEpoch(epoch);
-            return topology.awaitEpoch(epoch).flatMap(ignore -> supplier.get()).beginAsResult();
+            return topology.awaitEpoch(epoch).flatMap(ignore -> supplier.get());
         }
     }
 
@@ -273,11 +273,6 @@
         return commandStores.ifLocal(context, key, epoch, epoch, ifLocal);
     }
 
-    public AsyncChain<Void> ifLocalSince(PreLoadContext context, RoutingKey key, Timestamp since, Consumer<SafeCommandStore> ifLocal)
-    {
-        return commandStores.ifLocal(context, key, since.epoch(), Long.MAX_VALUE, ifLocal);
-    }
-
     public <T> void mapReduceConsumeLocal(TxnRequest<?> request, long minEpoch, long maxEpoch, MapReduceConsume<SafeCommandStore, T> mapReduceConsume)
     {
         commandStores.mapReduceConsume(request, request.scope(), minEpoch, maxEpoch, mapReduceConsume);
@@ -377,7 +372,7 @@
         // TODO (desirable, consider): The combination of updating the epoch of the next timestamp with epochs we don't have topologies for,
         //  and requiring preaccept to talk to its topology epoch means that learning of a new epoch via timestamp
         //  (ie not via config service) will halt any new txns from a node until it receives this topology
-        AsyncResult<Result> result = withEpoch(txnId.epoch(), () -> initiateCoordination(txnId, txn));
+        AsyncResult<Result> result = withEpoch(txnId.epoch(), () -> initiateCoordination(txnId, txn)).beginAsResult();
         coordinating.putIfAbsent(txnId, result);
         result.addCallback((success, fail) -> coordinating.remove(txnId, result));
         return result;
@@ -450,7 +445,7 @@
         return range.someIntersectingRoutingKey(null);
     }
 
-    static class RecoverFuture<T> extends AsyncResults.Settable<T> implements BiConsumer<T, Throwable>
+    static class RecoverFuture<T> extends AsyncResults.SettableResult<T> implements BiConsumer<T, Throwable>
     {
         @Override
         public void accept(T success, Throwable fail)
@@ -472,7 +467,7 @@
             RecoverFuture<Outcome> future = new RecoverFuture<>();
             RecoverWithRoute.recover(this, txnId, route, null, future);
             return future;
-        });
+        }).beginAsResult();
         coordinating.putIfAbsent(txnId, result);
         result.addCallback((success, fail) -> coordinating.remove(txnId, result));
         return result;
@@ -496,7 +491,7 @@
         if (unknownEpoch > 0)
         {
             configService.fetchTopologyForEpoch(unknownEpoch);
-            topology().awaitEpoch(unknownEpoch).addListener(() -> receive(request, from, replyContext));
+            topology().awaitEpoch(unknownEpoch).addCallback(() -> receive(request, from, replyContext));
             return;
         }
         scheduler.now(() -> request.process(this, from, replyContext));
diff --git a/accord-core/src/main/java/accord/local/PreLoadContext.java b/accord-core/src/main/java/accord/local/PreLoadContext.java
index 8a56c9f..6c9ef44 100644
--- a/accord-core/src/main/java/accord/local/PreLoadContext.java
+++ b/accord-core/src/main/java/accord/local/PreLoadContext.java
@@ -21,10 +21,15 @@
 import accord.api.Key;
 import accord.impl.CommandsForKey;
 import accord.primitives.Keys;
+import accord.primitives.Seekable;
 import accord.primitives.Seekables;
 import accord.primitives.TxnId;
+import accord.utils.Utils;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
 
 import java.util.Collections;
+import java.util.Set;
 
 /**
  * Lists txnids and keys of commands and commands for key that will be needed for an operation. Used
@@ -52,6 +57,13 @@
      */
     Seekables<?, ?> keys();
 
+    default boolean isSubsetOf(PreLoadContext superset)
+    {
+        Set<TxnId> superIds = superset.txnIds() instanceof Set ? (Set<TxnId>) superset.txnIds() : Sets.newHashSet(superset.txnIds());
+        Set<Seekable> superKeys = Sets.newHashSet(superset.keys());
+        return Iterables.all(txnIds(), superIds::contains) && Iterables.all(keys(), superKeys::contains);
+    }
+
     static PreLoadContext contextFor(Iterable<TxnId> txnIds, Seekables<?, ?> keys)
     {
         return new PreLoadContext()
@@ -74,6 +86,11 @@
         }
     }
 
+    static PreLoadContext contextFor(TxnId... txnIds)
+    {
+        return contextFor(Utils.listOf(txnIds), Keys.EMPTY);
+    }
+
     static PreLoadContext contextFor(TxnId txnId)
     {
         return contextFor(Collections.singleton(txnId), Keys.EMPTY);
diff --git a/accord-core/src/main/java/accord/local/SafeCommand.java b/accord-core/src/main/java/accord/local/SafeCommand.java
new file mode 100644
index 0000000..5648860
--- /dev/null
+++ b/accord-core/src/main/java/accord/local/SafeCommand.java
@@ -0,0 +1,139 @@
+/*
+ * 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 accord.api.Result;
+import accord.primitives.Ballot;
+import accord.primitives.Timestamp;
+import accord.primitives.TxnId;
+import accord.primitives.Writes;
+import accord.utils.Invariants;
+
+public abstract class SafeCommand
+{
+    private final TxnId txnId;
+
+    public SafeCommand(TxnId txnId)
+    {
+        this.txnId = txnId;
+    }
+
+    public abstract Command current();
+    public abstract void invalidate();
+    public abstract boolean invalidated();
+
+    public boolean isEmpty()
+    {
+        return current() == null;
+    }
+
+    protected abstract void set(Command command);
+
+    public TxnId txnId()
+    {
+        return txnId;
+    }
+
+    private <C extends Command> C update(C update)
+    {
+        set(update);
+        return update;
+    }
+
+    public Command addListener(CommandListener listener)
+    {
+        return update(Command.addListener(current(), listener));
+    }
+
+    public Command removeListener(CommandListener listener)
+    {
+        return update(Command.removeListener(current(), listener));
+    }
+
+    public Command.Committed updateWaitingOn(Command.WaitingOn.Update waitingOn)
+    {
+        return update(Command.updateWaitingOn(current().asCommitted(), waitingOn));
+    }
+
+    public Command updateAttributes(CommonAttributes attrs)
+    {
+        return update(current().updateAttributes(attrs));
+    }
+
+    public Command.PreAccepted preaccept(CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
+    {
+        return update(Command.preaccept(current(), attrs, executeAt, ballot));
+    }
+
+    public Command.Accepted markDefined(CommonAttributes attributes, Ballot promised)
+    {
+        return update(Command.markDefined(current(), attributes, promised));
+    }
+
+    public Command updatePromised(Ballot promised)
+    {
+        return update(current().updatePromised(promised));
+    }
+
+    public Command.Accepted accept(CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
+    {
+        return update(Command.accept(current(), attrs, executeAt, ballot));
+    }
+
+    public Command.Accepted acceptInvalidated(Ballot ballot)
+    {
+        return update(Command.acceptInvalidated(current(), ballot));
+    }
+
+    public Command.Committed commit(CommonAttributes attrs, Timestamp executeAt, Command.WaitingOn waitingOn)
+    {
+        return update(Command.commit(current(), attrs, executeAt, waitingOn));
+    }
+
+    public Command precommit(Timestamp executeAt)
+    {
+        return update(Command.precommit(current(), executeAt));
+    }
+
+    public Command.Committed commitInvalidated(CommonAttributes attrs, Timestamp executeAt)
+    {
+        return update(Command.commitInvalidated(current(), attrs, executeAt));
+    }
+
+    public Command.Committed readyToExecute()
+    {
+        return update(Command.readyToExecute(current().asCommitted()));
+    }
+
+    public Command.Executed preapplied(CommonAttributes attrs, Timestamp executeAt, Command.WaitingOn waitingOn, Writes writes, Result result)
+    {
+        return update(Command.preapplied(current(), attrs, executeAt, waitingOn, writes, result));
+    }
+
+    public Command.Executed applied()
+    {
+        return update(Command.applied(current().asExecuted()));
+    }
+
+    public Command.NotWitnessed notWitnessed()
+    {
+        Invariants.checkArgument(current() == null);
+        return update(Command.NotWitnessed.notWitnessed(txnId));
+    }
+}
diff --git a/accord-core/src/main/java/accord/local/SafeCommandStore.java b/accord-core/src/main/java/accord/local/SafeCommandStore.java
index eb89a3d..5b2f5ec 100644
--- a/accord-core/src/main/java/accord/local/SafeCommandStore.java
+++ b/accord-core/src/main/java/accord/local/SafeCommandStore.java
@@ -22,13 +22,10 @@
 import accord.api.DataStore;
 import accord.api.ProgressLog;
 import accord.primitives.*;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
-import accord.utils.async.AsyncChain;
 
 import javax.annotation.Nullable;
-import java.util.function.Consumer;
-import java.util.function.Function;
+
+import static accord.utils.Utils.listOf;
 
 /**
  * A CommandStore with exclusive access; a reference to this should not be retained outside of the scope of the method
@@ -38,7 +35,7 @@
  */
 public interface SafeCommandStore
 {
-    Command ifPresent(TxnId txnId);
+    SafeCommand ifPresent(TxnId txnId);
 
     /**
      * If the transaction is in memory, return it (and make it visible to future invocations of {@code command}, {@code ifPresent} etc).
@@ -46,14 +43,25 @@
      *
      * This permits efficient operation when a transaction involved in processing another transaction happens to be in memory.
      */
-    Command ifLoaded(TxnId txnId);
-    Command command(TxnId txnId);
+    SafeCommand ifLoaded(TxnId txnId);
+    SafeCommand command(TxnId txnId);
+
+    boolean canExecuteWith(PreLoadContext context);
 
     /**
      * Register a listener against the given TxnId, then load the associated transaction and invoke the listener
      * with its current state.
      */
-    void addAndInvokeListener(TxnId txnId, CommandListener listener);
+    default void addAndInvokeListener(TxnId txnId, TxnId listenerId)
+    {
+        PreLoadContext context = PreLoadContext.contextFor(listOf(txnId, listenerId), Keys.EMPTY);
+        commandStore().execute(context, safeStore -> {
+            SafeCommand safeCommand = safeStore.command(txnId);
+            CommandListener listener = new Command.Listener(listenerId);
+            safeCommand.addListener(listener);
+            listener.onChange(safeStore, safeCommand);
+        }).begin(agent());
+    }
 
     interface CommandFunction<I, O>
     {
@@ -75,10 +83,9 @@
      * Within each key or range visits TxnId in ascending order of queried timestamp.
      */
     <T> T mapReduce(Seekables<?, ?> keys, Ranges slice,
-                       TestKind testKind, TestTimestamp testTimestamp, Timestamp timestamp,
-                       TestDep testDep, @Nullable TxnId depId,
-                       @Nullable Status minStatus, @Nullable Status maxStatus,
-                       CommandFunction<T, T> map, T initialValue, T terminalValue);
+                    TestKind testKind, TestTimestamp testTimestamp, Timestamp timestamp,
+                    TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus,
+                    CommandFunction<T, T> map, T initialValue, T terminalValue);
 
     void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command);
     void register(Seekable keyOrRange, Ranges slice, Command command);
@@ -92,6 +99,21 @@
     long latestEpoch();
     Timestamp preaccept(TxnId txnId, Seekables<?, ?> keys);
 
-    AsyncChain<Void> execute(PreLoadContext context, Consumer<? super SafeCommandStore> consumer);
-    <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> function);
+    default void notifyListeners(SafeCommand safeCommand)
+    {
+        TxnId txnId = safeCommand.txnId();
+        Command command = safeCommand.current();
+        for (CommandListener listener : command.listeners())
+        {
+            PreLoadContext context = listener.listenerPreLoadContext(command.txnId());
+            if (canExecuteWith(context))
+            {
+                listener.onChange(this, safeCommand);
+            }
+            else
+            {
+                commandStore().execute(context, safeStore -> listener.onChange(safeStore, safeStore.command(txnId))).begin(agent());
+            }
+        }
+    }
 }
diff --git a/accord-core/src/main/java/accord/local/Status.java b/accord-core/src/main/java/accord/local/Status.java
index 98e9e23..88fec01 100644
--- a/accord-core/src/main/java/accord/local/Status.java
+++ b/accord-core/src/main/java/accord/local/Status.java
@@ -20,7 +20,7 @@
 
 import accord.messages.BeginRecovery;
 import accord.primitives.Ballot;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import java.util.List;
 import java.util.function.Function;
@@ -199,7 +199,7 @@
         {
             if (!deps.hasDecidedDeps())
                 return false;
-            Preconditions.checkState(executeAt.hasDecidedExecuteAt());
+            Invariants.checkState(executeAt.hasDecidedExecuteAt());
             return true;
         }
     }
diff --git a/accord-core/src/main/java/accord/local/SyncCommandStores.java b/accord-core/src/main/java/accord/local/SyncCommandStores.java
deleted file mode 100644
index f52fdbf..0000000
--- a/accord-core/src/main/java/accord/local/SyncCommandStores.java
+++ /dev/null
@@ -1,114 +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.local;
-
-import accord.api.Agent;
-import accord.api.DataStore;
-import accord.api.ProgressLog;
-import accord.primitives.Routables;
-import accord.utils.MapReduce;
-import accord.utils.MapReduceConsume;
-
-import java.util.function.Function;
-import java.util.stream.IntStream;
-
-// TODO (desired, testing): introduce new CommandStores that mimics asynchrony by integrating with Cluster scheduling for List workload
-public class SyncCommandStores extends CommandStores<SyncCommandStores.SyncCommandStore>
-{
-    public interface SafeSyncCommandStore extends SafeCommandStore
-    {
-    }
-
-    public static abstract class SyncCommandStore extends CommandStore
-    {
-        public SyncCommandStore(int id)
-        {
-            super(id);
-        }
-        protected abstract <T> T executeSync(PreLoadContext context, Function<? super SafeCommandStore, T> function);
-    }
-
-    public SyncCommandStores(NodeTimeService time, Agent agent, DataStore store, ShardDistributor shardDistributor, ProgressLog.Factory progressLogFactory, CommandStore.Factory shardFactory)
-    {
-        super(time, agent, store, shardDistributor, progressLogFactory, shardFactory);
-    }
-
-    protected static class SyncMapReduceAdapter<O> implements MapReduceAdapter<SyncCommandStore, O, O, O>
-    {
-        private static final SyncMapReduceAdapter INSTANCE = new SyncMapReduceAdapter<>();
-        public static <O> SyncMapReduceAdapter<O> instance() { return INSTANCE; }
-        private static final Object SENTINEL = new Object();
-
-        @Override
-        public O allocate()
-        {
-            return (O)SENTINEL;
-        }
-
-        @Override
-        public O apply(MapReduce<? super SafeCommandStore, O> map, SyncCommandStore commandStore, PreLoadContext context)
-        {
-            return commandStore.executeSync(context, map);
-        }
-
-        @Override
-        public O reduce(MapReduce<? super SafeCommandStore, O> reduce, O prev, O next)
-        {
-            return prev == SENTINEL ? next : reduce.reduce(prev, next);
-        }
-
-        @Override
-        public void consume(MapReduceConsume<?, O> reduceAndConsume, O result)
-        {
-            reduceAndConsume.accept(result, null);
-        }
-
-        @Override
-        public O reduce(MapReduce<?, O> reduce, O result)
-        {
-            return result == SENTINEL ? null : result;
-        }
-    }
-
-    @Override
-    public <O> void mapReduceConsume(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
-    {
-        try
-        {
-            mapReduceConsume(context, keys, minEpoch, maxEpoch, mapReduceConsume, SyncMapReduceAdapter.INSTANCE);
-        }
-        catch (Throwable t)
-        {
-            mapReduceConsume.accept(null, t);
-        }
-    }
-
-    @Override
-    public <O> void mapReduceConsume(PreLoadContext context, IntStream commandStoreIds, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
-    {
-        try
-        {
-            mapReduceConsume(context, commandStoreIds, mapReduceConsume, SyncMapReduceAdapter.INSTANCE);
-        }
-        catch (Throwable t)
-        {
-            mapReduceConsume.accept(null, t);
-        }
-    }
-}
diff --git a/accord-core/src/main/java/accord/messages/Accept.java b/accord-core/src/main/java/accord/messages/Accept.java
index f155ffe..be54d7d 100644
--- a/accord-core/src/main/java/accord/messages/Accept.java
+++ b/accord-core/src/main/java/accord/messages/Accept.java
@@ -18,26 +18,28 @@
 
 package accord.messages;
 
+import accord.local.Commands;
+import accord.local.Commands.AcceptOutcome;
+import accord.local.SafeCommand;
 import accord.local.SafeCommandStore;
 import accord.primitives.*;
 import accord.local.Node.Id;
 import accord.topology.Topologies;
 
 import accord.api.RoutingKey;
-import accord.local.Command.AcceptOutcome;
 import accord.primitives.PartialDeps;
 import accord.primitives.FullRoute;
 import accord.primitives.Ballot;
-import accord.local.Command;
 
 import java.util.Collections;
 import accord.primitives.Deps;
 import accord.primitives.TxnId;
 
 import javax.annotation.Nonnull;
+
 import javax.annotation.Nullable;
 
-import static accord.local.Command.AcceptOutcome.*;
+import static accord.local.Commands.AcceptOutcome.*;
 
 // TODO (low priority, efficiency): use different objects for send and receive, so can be more efficient
 //                                  (e.g. serialize without slicing, and without unnecessary fields)
@@ -87,14 +89,13 @@
         }
 
         // only accept if we actually participate in the ranges - otherwise we're just looking
-        Command command = safeStore.command(txnId);
-        switch (command.accept(safeStore, ballot, scope, keys, progressKey, executeAt, partialDeps))
+        switch (Commands.accept(safeStore, txnId, ballot, scope, keys, progressKey, executeAt, partialDeps))
         {
             default: throw new IllegalStateException();
             case Redundant:
                 return AcceptReply.REDUNDANT;
             case RejectedBallot:
-                return new AcceptReply(command.promised());
+                return new AcceptReply(safeStore.command(txnId).current().promised());
             case Success:
                 // TODO (desirable, efficiency): we don't need to calculate deps if executeAt == txnId
                 return new AcceptReply(calculatePartialDeps(safeStore));
@@ -240,8 +241,8 @@
         @Override
         public AcceptReply apply(SafeCommandStore safeStore)
         {
-            Command command = safeStore.command(txnId);
-            switch (command.acceptInvalidate(safeStore, ballot))
+            SafeCommand safeCommand = safeStore.command(txnId);
+            switch (Commands.acceptInvalidate(safeStore, safeCommand, ballot))
             {
                 default:
                 case Redundant:
@@ -249,7 +250,7 @@
                 case Success:
                     return AcceptReply.ACCEPT_INVALIDATE;
                 case RejectedBallot:
-                    return new AcceptReply(command.promised());
+                    return new AcceptReply(safeCommand.current().promised());
             }
         }
 
diff --git a/accord-core/src/main/java/accord/messages/Apply.java b/accord-core/src/main/java/accord/messages/Apply.java
index b45452d..ffc4693 100644
--- a/accord-core/src/main/java/accord/messages/Apply.java
+++ b/accord-core/src/main/java/accord/messages/Apply.java
@@ -19,8 +19,8 @@
 package accord.messages;
 
 import accord.local.SafeCommandStore;
+import accord.local.*;
 import accord.primitives.*;
-import accord.local.Command;
 import accord.local.Node.Id;
 import accord.api.Result;
 import accord.topology.Topologies;
@@ -37,35 +37,39 @@
 {
     public static class SerializationSupport
     {
-        public static Apply create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, long untilEpoch, Timestamp executeAt, PartialDeps deps, Writes writes, Result result)
+        public static Apply create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, long untilEpoch, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, Writes writes, Result result)
         {
-            return new Apply(txnId, scope, waitForEpoch, untilEpoch, executeAt, deps, writes, result);
+            return new Apply(txnId, scope, waitForEpoch, untilEpoch, keys, executeAt, deps, writes, result);
         }
     }
 
     public final long untilEpoch;
     public final Timestamp executeAt;
     public final PartialDeps deps;
+    public final Seekables<?, ?> keys;
     public final Writes writes;
     public final Result result;
 
-    public Apply(Id to, Topologies sendTo, Topologies applyTo, long untilEpoch, TxnId txnId, Route<?> route, Timestamp executeAt, Deps deps, Writes writes, Result result)
+    public Apply(Id to, Topologies sendTo, Topologies applyTo, long untilEpoch, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
     {
         super(to, sendTo, route, txnId);
         this.untilEpoch = untilEpoch;
         Ranges slice = applyTo == sendTo ? scope.covering() : applyTo.computeRangesForNode(to);
+
         this.deps = deps.slice(slice);
+        this.keys = txn.keys().slice(slice);
         this.executeAt = executeAt;
         this.writes = writes;
         this.result = result;
     }
 
-    private Apply(TxnId txnId, PartialRoute<?> route, long waitForEpoch, long untilEpoch, Timestamp executeAt, PartialDeps deps, Writes writes, Result result)
+    private Apply(TxnId txnId, PartialRoute<?> route, long waitForEpoch, long untilEpoch, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, Writes writes, Result result)
     {
         super(txnId, route, waitForEpoch);
         this.untilEpoch = untilEpoch;
         this.executeAt = executeAt;
         this.deps = deps;
+        this.keys = keys;
         this.writes = writes;
         this.result = result;
     }
@@ -80,8 +84,7 @@
     @Override
     public ApplyReply apply(SafeCommandStore safeStore)
     {
-        Command command = safeStore.command(txnId);
-        switch (command.apply(safeStore, untilEpoch, scope, executeAt, deps, writes, result))
+        switch (Commands.apply(safeStore, txnId, untilEpoch, scope, executeAt, deps, writes, result))
         {
             default:
             case Insufficient:
@@ -120,7 +123,7 @@
     @Override
     public Seekables<?, ?> keys()
     {
-        return Keys.EMPTY;
+        return keys;
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/messages/BeginInvalidation.java b/accord-core/src/main/java/accord/messages/BeginInvalidation.java
index ecc79c7..1bb3ba2 100644
--- a/accord-core/src/main/java/accord/messages/BeginInvalidation.java
+++ b/accord-core/src/main/java/accord/messages/BeginInvalidation.java
@@ -23,6 +23,9 @@
 import accord.local.Node.Id;
 import accord.primitives.*;
 import accord.topology.Topologies;
+import accord.primitives.Ballot;
+import accord.primitives.Route;
+import accord.primitives.TxnId;
 
 import javax.annotation.Nullable;
 import java.util.Collections;
@@ -60,8 +63,9 @@
     @Override
     public InvalidateReply apply(SafeCommandStore instance)
     {
-        Command command = instance.command(txnId);
-        boolean isOk = command.preacceptInvalidate(ballot);
+        boolean isOk = Commands.preacceptInvalidate(instance, txnId, ballot);
+        SafeCommand safeCommand = instance.command(txnId);
+        Command command = safeCommand.current();
         Ballot supersededBy = isOk ? null : command.promised();
         boolean acceptedFastPath = command.executeAt() != null && command.executeAt().equals(command.txnId());
         return new InvalidateReply(supersededBy, command.accepted(), command.status(), acceptedFastPath, command.route(), command.homeKey());
diff --git a/accord-core/src/main/java/accord/messages/BeginRecovery.java b/accord-core/src/main/java/accord/messages/BeginRecovery.java
index 3d699c8..cb93435 100644
--- a/accord-core/src/main/java/accord/messages/BeginRecovery.java
+++ b/accord-core/src/main/java/accord/messages/BeginRecovery.java
@@ -20,6 +20,7 @@
 
 import accord.api.Result;
 import accord.local.SafeCommandStore;
+import accord.local.*;
 import accord.local.Status.Phase;
 import accord.primitives.*;
 import accord.topology.Topologies;
@@ -32,8 +33,6 @@
 import accord.utils.Invariants;
 
 import accord.local.Node.Id;
-import accord.local.Command;
-import accord.local.Status;
 
 import java.util.Collections;
 
@@ -84,9 +83,8 @@
 
     public RecoverReply apply(SafeCommandStore safeStore)
     {
-        Command command = safeStore.command(txnId);
-
-        switch (command.recover(safeStore, partialTxn, route != null ? route : scope, progressKey, ballot))
+        SafeCommand safeCommand = safeStore.command(txnId);
+        switch (Commands.recover(safeStore, txnId, partialTxn, route != null ? route : scope, progressKey, ballot))
         {
             default:
                 throw new IllegalStateException("Unhandled Outcome");
@@ -95,11 +93,12 @@
                 throw new IllegalStateException("Invalid Outcome");
 
             case RejectedBallot:
-                return new RecoverNack(command.promised());
+                return new RecoverNack(safeCommand.current().promised());
 
             case Success:
         }
 
+        Command command = safeCommand.current();
         PartialDeps deps = command.partialDeps();
         if (!command.known().deps.hasProposedOrDecidedDeps())
         {
@@ -128,7 +127,11 @@
             // accepted txns with an earlier txnid that don't have our txnid as a dependency
             earlierAcceptedNoWitness = acceptedStartedBeforeWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
         }
-        return new RecoverOk(txnId, command.status(), command.accepted(), command.executeAt(), deps, earlierCommittedWitness, earlierAcceptedNoWitness, rejectsFastPath, command.writes(), command.result());
+
+        Writes writes = command.isExecuted() ? command.asExecuted().writes() : null;
+        Result result = command.isExecuted() ? command.asExecuted().result() : null;
+        return new RecoverOk(txnId, command.status(), command.accepted(), command.executeAt(), deps,
+                             earlierCommittedWitness, earlierAcceptedNoWitness, rejectsFastPath, writes, result);
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/messages/CheckStatus.java b/accord-core/src/main/java/accord/messages/CheckStatus.java
index 5f03522..63add8c 100644
--- a/accord-core/src/main/java/accord/messages/CheckStatus.java
+++ b/accord-core/src/main/java/accord/messages/CheckStatus.java
@@ -107,9 +107,10 @@
     }
 
     @Override
-    public CheckStatusOk apply(SafeCommandStore instance)
+    public CheckStatusOk apply(SafeCommandStore safeStore)
     {
-        Command command = instance.command(txnId);
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Command command = safeCommand.current();
         switch (includeInfo)
         {
             default: throw new IllegalStateException();
@@ -248,8 +249,8 @@
             super(node, command);
             this.partialTxn = command.partialTxn();
             this.committedDeps = command.status().compareTo(Committed) >= 0 ? command.partialDeps() : null;
-            this.writes = command.writes();
-            this.result = command.result();
+            this.writes = command.isExecuted() ? command.asExecuted().writes() : null;
+            this.result = command.isExecuted() ? command.asExecuted().result() : null;
         }
 
         protected CheckStatusOkFull(SaveStatus status, Ballot promised, Ballot accepted, Timestamp executeAt,
diff --git a/accord-core/src/main/java/accord/messages/Commit.java b/accord-core/src/main/java/accord/messages/Commit.java
index 25040c0..ce010ae 100644
--- a/accord-core/src/main/java/accord/messages/Commit.java
+++ b/accord-core/src/main/java/accord/messages/Commit.java
@@ -33,12 +33,16 @@
 import accord.utils.Invariants;
 
 import accord.topology.Topology;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static accord.local.Status.Committed;
 import static accord.local.Status.Known.DefinitionOnly;
 
 public class Commit extends TxnRequest<ReadNack>
 {
+    private static final Logger logger = LoggerFactory.getLogger(Commit.class);
+
     public static class SerializerSupport
     {
         public static Commit create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Timestamp executeAt, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData read)
@@ -135,7 +139,7 @@
     @Override
     public Seekables<?, ?> keys()
     {
-        return Keys.EMPTY;
+        return partialTxn != null ? partialTxn.keys() : Keys.EMPTY;
     }
 
     @Override
@@ -148,8 +152,7 @@
     @Override
     public synchronized ReadNack apply(SafeCommandStore safeStore)
     {
-        Command command = safeStore.command(txnId);
-        switch (command.commit(safeStore, route != null ? route : scope, progressKey, partialTxn, executeAt, partialDeps))
+        switch (Commands.commit(safeStore, txnId, route != null ? route : scope, progressKey, partialTxn, executeAt, partialDeps))
         {
             default:
             case Success:
@@ -157,10 +160,11 @@
                 return null;
 
             case Insufficient:
-                Invariants.checkState(!command.hasBeenWitnessed());
+                SafeCommand safeCommand = safeStore.command(txnId);
+                Invariants.checkState(!safeCommand.current().known().isDefinitionKnown());
                 if (defer == null)
                     defer = new Defer(DefinitionOnly, Committed.minKnown, Commit.this);
-                defer.add(command, safeStore.commandStore());
+                defer.add(safeStore, safeCommand, safeStore.commandStore());
                 return ReadNack.NotCommitted;
         }
     }
@@ -174,6 +178,12 @@
     @Override
     public void accept(ReadNack reply, Throwable failure)
     {
+        if (failure != null)
+        {
+            logger.error("Unhandled exception during commit", failure);
+            node.agent().onUncaughtException(failure);
+            return;
+        }
         if (reply != null)
             node.reply(replyTo, replyContext, reply);
         else if (read != null)
@@ -273,7 +283,7 @@
         public void process(Node node, Id from, ReplyContext replyContext)
         {
             node.forEachLocal(this, scope, txnId.epoch(), invalidateUntilEpoch,
-                            safeStore -> safeStore.command(txnId).commitInvalidate(safeStore))
+                            safeStore -> Commands.commitInvalidate(safeStore, txnId))
                     .begin(node.agent());
         }
 
diff --git a/accord-core/src/main/java/accord/messages/Defer.java b/accord-core/src/main/java/accord/messages/Defer.java
index d28810a..1e93b03 100644
--- a/accord-core/src/main/java/accord/messages/Defer.java
+++ b/accord-core/src/main/java/accord/messages/Defer.java
@@ -57,22 +57,25 @@
         this.request = request;
     }
 
-    void add(Command command, CommandStore commandStore)
+    void add(SafeCommandStore safeStore, SafeCommand safeCommand, CommandStore commandStore)
     {
         if (isDone)
             throw new IllegalStateException("Recurrent retry of " + request);
 
         waitingOn.add(commandStore.id());
         ++waitingOnCount;
-        command.addListener(this);
+        safeCommand.addListener(this);
     }
 
     @Override
-    public void onChange(SafeCommandStore safeStore, Command command)
+    public void onChange(SafeCommandStore safeStore, SafeCommand safeCommand)
     {
+        Command command = safeCommand.current();
         Ready ready = waitUntil.apply(command);
         if (ready == No) return;
-        command.removeListener(this);
+
+        safeCommand.removeListener(this);
+
         if (ready == Expired) return;
 
         int id = safeStore.commandStore().id();
@@ -93,5 +96,11 @@
         Invariants.checkState(caller.equals(request.txnId));
         return request;
     }
+
+    @Override
+    public boolean isTransient()
+    {
+        return true;
+    }
 }
 
diff --git a/accord-core/src/main/java/accord/messages/InformDurable.java b/accord-core/src/main/java/accord/messages/InformDurable.java
index 3ff7ae9..be2f984 100644
--- a/accord-core/src/main/java/accord/messages/InformDurable.java
+++ b/accord-core/src/main/java/accord/messages/InformDurable.java
@@ -19,10 +19,8 @@
 package accord.messages;
 
 import accord.api.ProgressLog.ProgressShard;
-import accord.local.Command;
+import accord.local.*;
 import accord.local.Node.Id;
-import accord.local.PreLoadContext;
-import accord.local.SafeCommandStore;
 import accord.local.Status.Durability;
 import accord.primitives.*;
 import accord.topology.Topologies;
@@ -89,8 +87,7 @@
     @Override
     public Reply apply(SafeCommandStore safeStore)
     {
-        Command command = safeStore.command(txnId);
-        command.setDurability(safeStore, durability, scope.homeKey(), executeAt);
+        Commands.setDurability(safeStore, txnId, durability, scope.homeKey(), executeAt);
         safeStore.progressLog().durable(txnId, scope, shard);
         return Ok;
     }
diff --git a/accord-core/src/main/java/accord/messages/InformHomeDurable.java b/accord-core/src/main/java/accord/messages/InformHomeDurable.java
index 7f62212..c5c6a05 100644
--- a/accord-core/src/main/java/accord/messages/InformHomeDurable.java
+++ b/accord-core/src/main/java/accord/messages/InformHomeDurable.java
@@ -21,7 +21,8 @@
 import java.util.Set;
 
 import accord.api.RoutingKey;
-import accord.local.Command;
+import accord.local.Commands;
+import accord.local.SafeCommand;
 import accord.local.Node;
 import accord.local.Node.Id;
 import accord.local.Status.Durability;
@@ -52,9 +53,9 @@
     {
         // TODO (expected, efficiency): do not load txnId first
         node.ifLocal(contextFor(txnId), homeKey, txnId.epoch(), safeStore -> {
-            Command command = safeStore.command(txnId);
-            command.setDurability(safeStore, durability, homeKey, executeAt);
-            safeStore.progressLog().durable(command, persistedOn);
+            SafeCommand safeCommand = safeStore.command(txnId);
+            Commands.setDurability(safeStore, txnId, durability, homeKey, executeAt);
+            safeStore.progressLog().durable(safeCommand.current(), persistedOn);
         }).begin(node.agent());
     }
 
diff --git a/accord-core/src/main/java/accord/messages/InformOfTxnId.java b/accord-core/src/main/java/accord/messages/InformOfTxnId.java
index b03d22f..22e2cf7 100644
--- a/accord-core/src/main/java/accord/messages/InformOfTxnId.java
+++ b/accord-core/src/main/java/accord/messages/InformOfTxnId.java
@@ -48,10 +48,10 @@
     @Override
     public Reply apply(SafeCommandStore safeStore)
     {
-        Command command = safeStore.command(txnId);
-        if (!command.hasBeen(Status.PreAccepted))
+        SafeCommand safeCommand = safeStore.command(txnId);
+        if (!safeCommand.current().hasBeen(Status.PreAccepted))
         {
-            command.updateHomeKey(safeStore, homeKey);
+            Commands.updateHomeKey(safeStore, safeCommand, homeKey);
             safeStore.progressLog().unwitnessed(txnId, homeKey, Home);
         }
         return Ok;
diff --git a/accord-core/src/main/java/accord/messages/PreAccept.java b/accord-core/src/main/java/accord/messages/PreAccept.java
index 8426cb7..5f37417 100644
--- a/accord-core/src/main/java/accord/messages/PreAccept.java
+++ b/accord-core/src/main/java/accord/messages/PreAccept.java
@@ -100,12 +100,12 @@
                     calculatePartialDeps(safeStore, txnId, partialTxn.keys(), txnId, safeStore.ranges().between(minUnsyncedEpoch, txnId.epoch())));
         }
 
-        Command command = safeStore.command(txnId);
-        switch (command.preaccept(safeStore, partialTxn, route != null ? route : scope, progressKey))
+        switch (Commands.preaccept(safeStore, txnId, partialTxn, route != null ? route : scope, progressKey))
         {
             default:
             case Success:
             case Redundant:
+                Command command = safeStore.command(txnId).current();
                 return new PreAcceptOk(txnId, command.executeAt(),
                         calculatePartialDeps(safeStore, txnId, partialTxn.keys(), txnId, safeStore.ranges().between(minUnsyncedEpoch, txnId.epoch())));
 
diff --git a/accord-core/src/main/java/accord/messages/ReadData.java b/accord-core/src/main/java/accord/messages/ReadData.java
index 479ce0c..8d4e275 100644
--- a/accord-core/src/main/java/accord/messages/ReadData.java
+++ b/accord-core/src/main/java/accord/messages/ReadData.java
@@ -28,7 +28,7 @@
 import accord.local.*;
 import accord.api.Data;
 import accord.topology.Topologies;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -108,8 +108,9 @@
     }
 
     @Override
-    public synchronized void onChange(SafeCommandStore safeStore, Command command)
+    public synchronized void onChange(SafeCommandStore safeStore, SafeCommand safeCommand)
     {
+        Command command = safeCommand.current();
         logger.trace("{}: updating as listener in response to change on {} with status {} ({})",
                 this, command.txnId(), command.status(), command);
         switch (command.status())
@@ -130,18 +131,19 @@
             case ReadyToExecute:
         }
 
-        command.removeListener(this);
+        command = safeCommand.removeListener(this);
+
         if (!isObsolete)
-            read(safeStore, command);
+            read(safeStore, command.asCommitted());
     }
 
     @Override
     public synchronized ReadNack apply(SafeCommandStore safeStore)
     {
-        Command command = safeStore.command(txnId);
-        Status status = command.status();
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Status status = safeCommand.current().status();
         logger.trace("{}: setting up read with status {} on {}", txnId, status, safeStore);
-        switch (command.status()) {
+        switch (status) {
             default:
                 throw new AssertionError();
             case Committed:
@@ -152,7 +154,8 @@
             case PreCommitted:
                 waitingOn.set(safeStore.commandStore().id());
                 ++waitingOnCount;
-                command.addListener(this);
+
+                safeCommand.addListener(this);
 
                 if (status == Committed)
                     return null;
@@ -164,7 +167,7 @@
                 waitingOn.set(safeStore.commandStore().id());
                 ++waitingOnCount;
                 if (!isObsolete)
-                    read(safeStore, command);
+                    read(safeStore, safeCommand.current().asCommitted());
                 return null;
 
             case PreApplied:
@@ -183,6 +186,11 @@
                 : r1.compareTo(r2) >= 0 ? r1 : r2;
     }
 
+    private void removeListener(SafeCommandStore safeStore, TxnId txnId)
+    {
+        safeStore.command(txnId).removeListener(this);
+    }
+
     @Override
     public synchronized void accept(ReadNack reply, Throwable failure)
     {
@@ -197,7 +205,7 @@
             data = null;
             // TODO (expected, exceptions): probably a better way to handle this, as might not be uncaught
             node.agent().onUncaughtException(failure);
-            node.commandStores().mapReduceConsume(this, waitingOn.stream(), forEach(in -> in.command(txnId).removeListener(this), node.agent()));
+            node.commandStores().mapReduceConsume(this, waitingOn.stream(), forEach(in -> removeListener(in, txnId), node.agent()));
         }
         else
         {
@@ -207,13 +215,16 @@
 
     private void ack()
     {
+        // wait for -1 to ensure the setup phase has also completed. Setup calls ack in its callback
+        // and prevents races where we respond before dispatching all the required reads (if the reads are
+        // completing faster than the reads can be setup on all required shards)
         if (-1 == --waitingOnCount)
             node.reply(replyTo, replyContext, new ReadOk(data));
     }
 
     private synchronized void readComplete(CommandStore commandStore, Data result)
     {
-        Preconditions.checkState(waitingOn.get(commandStore.id()));
+        Invariants.checkState(waitingOn.get(commandStore.id()));
         logger.trace("{}: read completed on {}", txnId, commandStore);
         if (result != null)
             data = data == null ? result : data.merge(result);
@@ -222,10 +233,10 @@
         ack();
     }
 
-    private void read(SafeCommandStore safeStore, Command command)
+    private void read(SafeCommandStore safeStore, Command.Committed command)
     {
-        logger.trace("{}: executing read", command.txnId());
         CommandStore unsafeStore = safeStore.commandStore();
+        logger.trace("{}: executing read", command.txnId());
         command.read(safeStore).begin((next, throwable) -> {
             if (throwable != null)
             {
diff --git a/accord-core/src/main/java/accord/messages/WaitOnCommit.java b/accord-core/src/main/java/accord/messages/WaitOnCommit.java
index b7c20f9..f27816f 100644
--- a/accord-core/src/main/java/accord/messages/WaitOnCommit.java
+++ b/accord-core/src/main/java/accord/messages/WaitOnCommit.java
@@ -75,9 +75,10 @@
     }
 
     @Override
-    public Void apply(SafeCommandStore instance)
+    public Void apply(SafeCommandStore safeStore)
     {
-        Command command = instance.command(txnId);
+        SafeCommand safeCommand = safeStore.command(txnId);
+        Command command = safeCommand.current();
         switch (command.status())
         {
             default: throw new AssertionError();
@@ -87,8 +88,8 @@
             case AcceptedInvalidate:
             case PreCommitted:
                 waitingOnUpdater.incrementAndGet(this);
-                command.addListener(this);
-                instance.progressLog().waiting(txnId, Committed.minKnown, scope);
+                safeCommand.addListener(this);
+                safeStore.progressLog().waiting(txnId, Committed.minKnown, scope);
                 break;
 
             case Committed:
@@ -101,8 +102,9 @@
     }
 
     @Override
-    public void onChange(SafeCommandStore safeStore, Command command)
+    public void onChange(SafeCommandStore safeStore, SafeCommand safeCommand)
     {
+        Command command = safeCommand.current();
         logger.trace("{}: updating as listener in response to change on {} with status {} ({})",
                 this, command.txnId(), command.status(), command);
         switch (command.status())
@@ -122,7 +124,7 @@
             case Invalidated:
         }
 
-        command.removeListener(this);
+        safeCommand.removeListener(this);
         ack();
     }
 
diff --git a/accord-core/src/main/java/accord/primitives/Timestamp.java b/accord-core/src/main/java/accord/primitives/Timestamp.java
index 5ac5e15..6ec2057 100644
--- a/accord-core/src/main/java/accord/primitives/Timestamp.java
+++ b/accord-core/src/main/java/accord/primitives/Timestamp.java
@@ -50,7 +50,7 @@
         return new Timestamp(epochMsb(epoch), 0, Id.NONE);
     }
 
-    private static final long MAX_EPOCH = (1L << 48) - 1;
+    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;
     public static final Timestamp NONE = new Timestamp(0, 0, 0, Id.NONE);
diff --git a/accord-core/src/main/java/accord/primitives/Txn.java b/accord-core/src/main/java/accord/primitives/Txn.java
index 73878e5..f1d2655 100644
--- a/accord-core/src/main/java/accord/primitives/Txn.java
+++ b/accord-core/src/main/java/accord/primitives/Txn.java
@@ -22,9 +22,8 @@
 import java.util.List;
 import java.util.Objects;
 
-import accord.local.Command;
-
 import accord.api.*;
+import accord.local.Command;
 import accord.local.SafeCommandStore;
 import accord.utils.async.AsyncChain;
 import accord.utils.async.AsyncChains;
@@ -186,7 +185,7 @@
         return new Writes(executeAt, update.keys(), update.apply(data));
     }
 
-    default AsyncChain<Data> read(SafeCommandStore safeStore, Command command)
+    default AsyncChain<Data> read(SafeCommandStore safeStore, Command.Committed command)
     {
         Ranges ranges = safeStore.ranges().at(command.executeAt().epoch());
         List<AsyncChain<Data>> futures = Routables.foldlMinimal(keys(), ranges, (key, accumulate, index) -> {
diff --git a/accord-core/src/main/java/accord/primitives/TxnId.java b/accord-core/src/main/java/accord/primitives/TxnId.java
index 5f51c20..e4842ab 100644
--- a/accord-core/src/main/java/accord/primitives/TxnId.java
+++ b/accord-core/src/main/java/accord/primitives/TxnId.java
@@ -42,6 +42,11 @@
         return new TxnId(epoch, hlc, flags, node);
     }
 
+    public static TxnId fromValues(long epoch, long hlc, int flags, int node)
+    {
+        return new TxnId(epoch, hlc, flags, new Id(node));
+    }
+
     public TxnId(Timestamp timestamp, Kind rw, Domain domain)
     {
         super(timestamp, flags(rw, domain));
diff --git a/accord-core/src/main/java/accord/utils/DeterministicIdentitySet.java b/accord-core/src/main/java/accord/utils/DeterministicIdentitySet.java
index 1bb9410..743f288 100644
--- a/accord-core/src/main/java/accord/utils/DeterministicIdentitySet.java
+++ b/accord-core/src/main/java/accord/utils/DeterministicIdentitySet.java
@@ -53,6 +53,12 @@
         lookup = new IdentityHashMap<>(size);
     }
 
+    public DeterministicIdentitySet(DeterministicIdentitySet<T> copy)
+    {
+        this(copy.size());
+        copy.forEach(this::addInternal);
+    }
+
     @Override
     public Iterator<T> iterator()
     {
@@ -104,9 +110,7 @@
         return lookup.size();
     }
 
-    // we add to the front, and iterate in reverse order, so that we can add and remove while iterating without modifying the set we iterate over
-    @Override
-    public boolean add(T item)
+    private boolean addInternal(T item)
     {
         Entry<T> entry = lookup.computeIfAbsent(item, Entry::new);
         if (entry.prev != null)
@@ -117,6 +121,12 @@
         entry.next.prev = entry;
         return true;
     }
+    // we add to the front, and iterate in reverse order, so that we can add and remove while iterating without modifying the set we iterate over
+    @Override
+    public boolean add(T item)
+    {
+        return addInternal(item);
+    }
 
     @Override
     public boolean remove(Object item)
diff --git a/accord-core/src/main/java/accord/utils/Invariants.java b/accord-core/src/main/java/accord/utils/Invariants.java
index 1fb4e8e..7248742 100644
--- a/accord-core/src/main/java/accord/utils/Invariants.java
+++ b/accord-core/src/main/java/accord/utils/Invariants.java
@@ -20,8 +20,11 @@
 
 import net.nicoulaj.compilecommand.annotations.Inline;
 
+import javax.annotation.Nullable;
 import java.util.function.Predicate;
 
+import static java.lang.String.format;
+
 public class Invariants
 {
     private static final boolean PARANOID = true;
@@ -36,6 +39,27 @@
         return DEBUG;
     }
 
+    private static void illegalState(String msg)
+    {
+        throw new IllegalStateException(msg);
+    }
+
+    private static void illegalState()
+    {
+        illegalState(null);
+    }
+
+    private static void illegalArgument(String msg)
+    {
+        throw new IllegalArgumentException(msg);
+    }
+
+
+    private static void illegalArgument()
+    {
+        illegalArgument(null);
+    }
+
     public static <T1, T2 extends T1> T2 checkType(T1 cast)
     {
         return (T2)cast;
@@ -44,33 +68,75 @@
     public static <T1, T2 extends T1> T2 checkType(Class<T2> to, T1 cast)
     {
         if (cast != null && !to.isInstance(cast))
-            throw new IllegalStateException();
+            illegalState();
         return (T2)cast;
     }
 
     public static <T1, T2 extends T1> T2 checkType(Class<T2> to, T1 cast, String msg)
     {
         if (cast != null && !to.isInstance(cast))
-            throw new IllegalStateException(msg);
+            illegalState(msg);
         return (T2)cast;
     }
 
     public static void paranoid(boolean condition)
     {
         if (PARANOID && !condition)
-            throw new IllegalStateException();
+            illegalState();
     }
 
     public static void checkState(boolean condition)
     {
         if (!condition)
-            throw new IllegalStateException();
+            illegalState();
     }
 
     public static void checkState(boolean condition, String msg)
     {
         if (!condition)
-            throw new IllegalStateException(msg);
+            illegalState(msg);
+    }
+
+    public static void checkState(boolean condition, String fmt, int p1)
+    {
+        if (!condition)
+            illegalState(format(fmt, p1));
+    }
+
+    public static void checkState(boolean condition, String fmt, int p1, int p2)
+    {
+        if (!condition)
+            illegalState(format(fmt, p1, p2));
+    }
+
+    public static void checkState(boolean condition, String fmt, long p1)
+    {
+        if (!condition)
+            illegalState(format(fmt, p1));
+    }
+
+    public static void checkState(boolean condition, String fmt, long p1, long p2)
+    {
+        if (!condition)
+            illegalState(format(fmt, p1, p2));
+    }
+
+    public static void checkState(boolean condition, String fmt, @Nullable Object p1)
+    {
+        if (!condition)
+            illegalState(format(fmt, p1));
+    }
+
+    public static void checkState(boolean condition, String fmt, @Nullable Object p1, @Nullable Object p2)
+    {
+        if (!condition)
+            illegalState(format(fmt, p1, p2));
+    }
+
+    public static void checkState(boolean condition, String fmt, Object... args)
+    {
+        if (!condition)
+            illegalState(format(fmt, args));
     }
 
     public static <T> T nonNull(T param)
@@ -83,40 +149,131 @@
     public static int isNatural(int input)
     {
         if (input < 0)
-            throw new IllegalStateException();
+            illegalState();
         return input;
     }
 
     public static long isNatural(long input)
     {
         if (input < 0)
-            throw new IllegalStateException();
+            illegalState();
         return input;
     }
 
     public static void checkArgument(boolean condition)
     {
         if (!condition)
-            throw new IllegalArgumentException();
+            illegalArgument();
     }
 
     public static void checkArgument(boolean condition, String msg)
     {
         if (!condition)
-            throw new IllegalArgumentException(msg);
+            illegalArgument(msg);
+    }
+
+    public static void checkArgument(boolean condition, String fmt, int p1)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1));
+    }
+
+    public static void checkArgument(boolean condition, String fmt, int p1, int p2)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1, p2));
+    }
+
+    public static void checkArgument(boolean condition, String fmt, long p1)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1));
+    }
+
+    public static void checkArgument(boolean condition, String fmt, long p1, long p2)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1, p2));
+    }
+
+    public static void checkArgument(boolean condition, String fmt, @Nullable Object p1)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1));
+    }
+
+    public static void checkArgument(boolean condition, String fmt, @Nullable Object p1, @Nullable Object p2)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1, p2));
+    }
+
+    public static void checkArgument(boolean condition, String fmt, Object... args)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, args));
     }
 
     public static <T> T checkArgument(T param, boolean condition)
     {
         if (!condition)
-            throw new IllegalArgumentException();
+            illegalArgument();
         return param;
     }
 
     public static <T> T checkArgument(T param, boolean condition, String msg)
     {
         if (!condition)
-            throw new IllegalArgumentException(msg);
+            illegalArgument(msg);
+        return param;
+    }
+
+    public static <T> T checkArgument(T param, boolean condition, String fmt, int p1)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1));
+        return param;
+    }
+
+    public static <T> T checkArgument(T param, boolean condition, String fmt, int p1, int p2)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1, p2));
+        return param;
+    }
+
+    public static <T> T checkArgument(T param, boolean condition, String fmt, long p1)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1));
+        return param;
+    }
+
+    public static <T> T checkArgument(T param, boolean condition, String fmt, long p1, long p2)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1, p2));
+        return param;
+    }
+
+    public static <T> T checkArgument(T param, boolean condition, String fmt, @Nullable Object p1)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1));
+        return param;
+    }
+
+    public static <T> T checkArgument(T param, boolean condition, String fmt, @Nullable Object p1, @Nullable Object p2)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, p1, p2));
+        return param;
+    }
+
+    public static <T> T checkArgument(T param, boolean condition, String fmt, Object... args)
+    {
+        if (!condition)
+            illegalArgument(format(fmt, args));
         return param;
     }
 
@@ -124,7 +281,7 @@
     public static <T> T checkArgument(T param, Predicate<T> condition)
     {
         if (!condition.test(param))
-            throw new IllegalArgumentException();
+            illegalArgument();
         return param;
     }
 
@@ -132,7 +289,19 @@
     public static <T> T checkArgument(T param, Predicate<T> condition, String msg)
     {
         if (!condition.test(param))
-            throw new IllegalArgumentException(msg);
+            illegalArgument(msg);
         return param;
     }
+
+    public static <O> O cast(Object o, Class<O> klass)
+    {
+        try
+        {
+            return klass.cast(o);
+        }
+        catch (ClassCastException e)
+        {
+            throw new IllegalArgumentException(format("Unable to cast %s to %s", o, klass.getName()));
+        }
+    }
 }
diff --git a/accord-core/src/main/java/accord/utils/MapReduce.java b/accord-core/src/main/java/accord/utils/MapReduce.java
index 98c058c..57a9214 100644
--- a/accord-core/src/main/java/accord/utils/MapReduce.java
+++ b/accord-core/src/main/java/accord/utils/MapReduce.java
@@ -18,6 +18,7 @@
 
 package accord.utils;
 
+import java.util.function.BiFunction;
 import java.util.function.Function;
 
 public interface MapReduce<I, O> extends Function<I, O>
@@ -26,4 +27,22 @@
     @Override
     O apply(I in);
     O reduce(O o1, O o2);
+
+    static <I, O> MapReduce<I, O> of(Function<I, O> map, BiFunction<O, O, O> reduce)
+    {
+        return new MapReduce<I, O>()
+        {
+            @Override
+            public O apply(I in)
+            {
+                return map.apply(in);
+            }
+
+            @Override
+            public O reduce(O o1, O o2)
+            {
+                return reduce.apply(o1, o2);
+            }
+        };
+    }
 }
diff --git a/accord-core/src/main/java/accord/utils/Utils.java b/accord-core/src/main/java/accord/utils/Utils.java
index 0c5ff5a..c31da64 100644
--- a/accord-core/src/main/java/accord/utils/Utils.java
+++ b/accord-core/src/main/java/accord/utils/Utils.java
@@ -18,6 +18,10 @@
 
 package accord.utils;
 
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedSet;
+
 import java.util.*;
 import java.util.function.IntFunction;
 
@@ -51,4 +55,46 @@
             dst.add(src[i]);
         return dst;
     }
+
+    public static <T> ImmutableSortedSet<T> ensureSortedImmutable(NavigableSet<T> set)
+    {
+        if (set == null || set.isEmpty())
+            return ImmutableSortedSet.of();
+        return set instanceof ImmutableSortedSet ? (ImmutableSortedSet<T>) set : ImmutableSortedSet.copyOf(set);
+    }
+
+    public static <K, V> ImmutableSortedMap<K, V> ensureSortedImmutable(NavigableMap<K, V> map)
+    {
+        if (map == null || map.isEmpty())
+            return ImmutableSortedMap.of();
+        return map instanceof ImmutableSortedMap ? (ImmutableSortedMap<K, V>) map : ImmutableSortedMap.copyOf(map);
+    }
+
+    public static <T> ImmutableSet<T> ensureImmutable(java.util.Set<T> set)
+    {
+        if (set == null || set.isEmpty())
+            return ImmutableSet.of();
+        return set instanceof ImmutableSet ? (ImmutableSet<T>) set : ImmutableSet.copyOf(set);
+    }
+
+    public static <T extends Comparable<? super T>> NavigableSet<T> ensureSortedMutable(NavigableSet<T> set)
+    {
+        if (set == null)
+            return new TreeSet<>();
+        return set instanceof ImmutableSortedSet ? new TreeSet<>(set) : set;
+    }
+
+    public static <K extends Comparable<K>, V> NavigableMap<K, V> ensureSortedMutable(NavigableMap<K, V> map)
+    {
+        if (map == null)
+            return new TreeMap<>();
+        return map instanceof ImmutableSortedMap ? new TreeMap<>(map) : map;
+    }
+
+    public static <T> Set<T> ensureMutable(Set<T> set)
+    {
+        if (set == null)
+            return new HashSet<>();
+        return set instanceof ImmutableSet ? new HashSet<>(set) : set;
+    }
 }
diff --git a/accord-core/src/main/java/accord/utils/async/AsyncCallbacks.java b/accord-core/src/main/java/accord/utils/async/AsyncCallbacks.java
index c338ea3..0c54622 100644
--- a/accord-core/src/main/java/accord/utils/async/AsyncCallbacks.java
+++ b/accord-core/src/main/java/accord/utils/async/AsyncCallbacks.java
@@ -40,9 +40,12 @@
 
     public static <T> BiConsumer<? super T, Throwable> inExecutor(Runnable runnable, Executor executor)
     {
-        return (result, throwable) -> {
-            if (throwable == null) executor.execute(runnable);
-            else throw new RuntimeException(throwable);
+        return inExecutor(toCallback(runnable), executor);
+    }
+
+    public static <T> BiConsumer<T, Throwable> toCallback(Runnable runnable) {
+        return (unused, failure) -> {
+            if (failure == null) runnable.run();
         };
     }
 }
diff --git a/accord-core/src/main/java/accord/utils/async/AsyncChain.java b/accord-core/src/main/java/accord/utils/async/AsyncChain.java
index 8bb5163..fa88aed 100644
--- a/accord-core/src/main/java/accord/utils/async/AsyncChain.java
+++ b/accord-core/src/main/java/accord/utils/async/AsyncChain.java
@@ -20,6 +20,7 @@
 
 import java.util.concurrent.Executor;
 import java.util.function.BiConsumer;
+import java.util.function.Consumer;
 import java.util.function.Function;
 
 import com.google.common.util.concurrent.ListenableFuture;
@@ -36,21 +37,43 @@
      */
     <T> AsyncChain<T> flatMap(Function<? super V, ? extends AsyncChain<T>> mapper);
 
+    default AsyncChain<Void> accept(Consumer<? super V> action)
+    {
+        return map(r -> {
+            action.accept(r);
+            return null;
+        });
+    }
+
     /**
      * Support {@link com.google.common.util.concurrent.Futures#addCallback} natively
      */
     AsyncChain<V> addCallback(BiConsumer<? super V, Throwable> callback);
 
-    void begin(BiConsumer<? super V, Throwable> callback);
-
-    default void begin(Runnable runnable)
+    /**
+     * Adds a callback that only listens to the successful case, a failed chain will not trigger the callback
+     */
+    default AsyncChain<V> addCallback(Runnable runnable)
     {
-        begin((unused, failure) -> {
-            if (failure == null) runnable.run();
-            else throw new RuntimeException(failure);
-        });
+        return addCallback(AsyncCallbacks.toCallback(runnable));
     }
 
+    default AsyncChain<V> addCallback(BiConsumer<? super V, Throwable> callback, Executor executor)
+    {
+        return addCallback(AsyncCallbacks.inExecutor(callback, executor));
+    }
+
+    default AsyncChain<V> addCallback(Runnable runnable, Executor executor)
+    {
+        return addCallback(AsyncCallbacks.inExecutor(runnable, executor));
+    }
+
+    /**
+     * Causes the chain to begin, starting all work required.  This method must be called exactly once, not calling will
+     * not cause any work to start, and calling multiple times will be rejected.
+     */
+    void begin(BiConsumer<? super V, Throwable> callback);
+
     default AsyncResult<V> beginAsResult()
     {
         return AsyncResults.forChain(this);
diff --git a/accord-core/src/main/java/accord/utils/async/AsyncChainCombiner.java b/accord-core/src/main/java/accord/utils/async/AsyncChainCombiner.java
index d489243..f4a9e63 100644
--- a/accord-core/src/main/java/accord/utils/async/AsyncChainCombiner.java
+++ b/accord-core/src/main/java/accord/utils/async/AsyncChainCombiner.java
@@ -18,7 +18,7 @@
 
 package accord.utils.async;
 
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 import com.google.common.collect.Lists;
 
 import java.util.List;
@@ -33,32 +33,32 @@
     private volatile BiConsumer<? super O, Throwable> callback;
     private volatile int remaining;
 
-    protected AsyncChainCombiner(List<AsyncChain<I>> inputs)
+    protected AsyncChainCombiner(List<? extends AsyncChain<? extends I>> inputs)
     {
-        Preconditions.checkArgument(!inputs.isEmpty());
+        Invariants.checkArgument(!inputs.isEmpty(), "No inputs defined");
         this.state = inputs;
     }
 
     private List<AsyncChain<? extends I>> inputs()
     {
         Object current = state;
-        Preconditions.checkState(current instanceof List);
+        Invariants.checkState(current instanceof List, "Expected state to be List but was %s", (current == null ? null : current.getClass()));
         return (List<AsyncChain<? extends I>>) current;
     }
 
     private I[] results()
     {
         Object current = state;
-        Preconditions.checkState(current instanceof Object[]);
+        Invariants.checkState(current instanceof Object[], "Expected state to be Object[] but was %s", (current == null ? null : current.getClass()));
         return (I[]) current;
     }
 
-    void add(AsyncChain<I> chain)
+    void add(AsyncChain<? extends I> chain)
     {
         inputs().add(chain);
     }
 
-    void addAll(List<AsyncChain<I>> chains)
+    void addAll(List<? extends AsyncChain<? extends I>> chains)
     {
         inputs().addAll(chains);
     }
@@ -70,7 +70,7 @@
             return ((List) current).size();
         if (current instanceof Object[])
             return ((Object[]) current).length;
-        throw new IllegalStateException();
+        throw new IllegalStateException("Unexpected type: " + (current == null ? "null" : current.getClass()));
     }
 
     abstract void complete(I[] results, BiConsumer<? super O, Throwable> callback);
@@ -81,7 +81,7 @@
         if (current == 0)
             return;
 
-        if (throwable != null && REMAINING.compareAndSet(this, current, 0))
+        if (throwable != null && REMAINING.getAndSet(this, 0) != 0)
         {
             callback.accept(null, throwable);
             return;
@@ -107,7 +107,7 @@
     }
 
     @Override
-    public void begin(BiConsumer<? super O, Throwable> callback)
+    protected void start(BiConsumer<? super O, Throwable> callback)
     {
         List<? extends AsyncChain<? extends I>> chains = inputs();
         state = new Object[chains.size()];
@@ -121,7 +121,7 @@
 
     static class All<V> extends AsyncChainCombiner<V, List<V>>
     {
-        All(List<AsyncChain<V>> asyncChains)
+        All(List<? extends AsyncChain<? extends V>> asyncChains)
         {
             super(asyncChains);
         }
@@ -137,7 +137,7 @@
     static class Reduce<V> extends AsyncChainCombiner<V, V>
     {
         private final BiFunction<V, V, V> reducer;
-        Reduce(List<AsyncChain<V>> asyncChains, BiFunction<V, V, V> reducer)
+        Reduce(List<? extends AsyncChain<? extends V>> asyncChains, BiFunction<V, V, V> reducer)
         {
             super(asyncChains);
             this.reducer = reducer;
diff --git a/accord-core/src/main/java/accord/utils/async/AsyncChains.java b/accord-core/src/main/java/accord/utils/async/AsyncChains.java
index a470eda..e697d23 100644
--- a/accord-core/src/main/java/accord/utils/async/AsyncChains.java
+++ b/accord-core/src/main/java/accord/utils/async/AsyncChains.java
@@ -20,19 +20,26 @@
 
 import java.util.List;
 import java.util.concurrent.*;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executor;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 
+import accord.api.VisibleForImplementation;
 import accord.utils.Invariants;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static accord.utils.async.AsyncChainCombiner.Reduce;
 
 public abstract class AsyncChains<V> implements AsyncChain<V>
 {
+    private static final Logger logger = LoggerFactory.getLogger(AsyncChains.class);
+
     static class Immediate<V> implements AsyncChain<V>
     {
         static class FailureHolder
@@ -89,12 +96,22 @@
             next = this;
         }
 
+        protected abstract void start(BiConsumer<? super V, Throwable> callback);
+
+        @Override
+        public final void begin(BiConsumer<? super V, Throwable> callback)
+        {
+            Invariants.checkArgument(next != null);
+            next = null;
+            start(callback);
+        }
+
         void begin()
         {
             Invariants.checkArgument(next != null);
             BiConsumer<? super V, Throwable> next = this.next;
             this.next = null;
-            begin(next);
+            start(next);
         }
 
         @Override
@@ -115,8 +132,8 @@
         @Override
         public void begin(BiConsumer<? super O, Throwable> callback)
         {
-            Preconditions.checkArgument(!(callback instanceof AsyncChains.Head));
-            Preconditions.checkState(next instanceof AsyncChains.Head);
+            Invariants.checkArgument(!(callback instanceof AsyncChains.Head));
+            checkNextIsHead();
             Head<?> head = (Head<?>) next;
             next = callback;
             head.begin();
@@ -249,7 +266,7 @@
     // (or perhaps some additional helper implementations that permit us to simply implement apply for Map and FlatMap)
     <O, T extends AsyncChain<O> & BiConsumer<? super V, Throwable>> AsyncChain<O> add(Function<Head<?>, T> factory)
     {
-        Preconditions.checkState(next instanceof Head<?>);
+        checkNextIsHead();
         Head<?> head = (Head<?>) next;
         T result = factory.apply(head);
         next = result;
@@ -258,13 +275,19 @@
 
     <P, O, T extends AsyncChain<O> & BiConsumer<? super V, Throwable>> AsyncChain<O> add(BiFunction<Head<?>, P, T> factory, P param)
     {
-        Preconditions.checkState(next instanceof Head<?>);
+        checkNextIsHead();
         Head<?> head = (Head<?>) next;
         T result = factory.apply(head, param);
         next = result;
         return result;
     }
 
+    protected void checkNextIsHead()
+    {
+        Invariants.checkState(next != null, "Begin was called multiple times");
+        Invariants.checkState(next instanceof Head<?>, "Next is not an instance of AsyncChains.Head (it is %s); was map/flatMap called on the same object multiple times?", next.getClass());
+    }
+
     private static <V> Runnable encapsulate(Callable<V> callable, BiConsumer<? super V, Throwable> receiver)
     {
         return () -> {
@@ -275,6 +298,7 @@
             }
             catch (Throwable t)
             {
+                logger.debug("AsyncChain Callable threw an Exception", t);
                 receiver.accept(null, t);
             }
         };
@@ -290,6 +314,7 @@
             }
             catch (Throwable t)
             {
+                logger.debug("AsyncChain Runnable threw an Exception", t);
                 receiver.accept(null, t);
             }
         };
@@ -310,9 +335,16 @@
         return new Head<V>()
         {
             @Override
-            public void begin(BiConsumer<? super V, Throwable> next)
+            protected void start(BiConsumer<? super V, Throwable> callback)
             {
-                executor.execute(encapsulate(callable, next));
+                try
+                {
+                    executor.execute(encapsulate(callable, callback));
+                }
+                catch (Throwable t)
+                {
+                    callback.accept(null, t);
+                }
             }
         };
     }
@@ -322,24 +354,55 @@
         return new Head<Void>()
         {
             @Override
-            public void begin(BiConsumer<? super Void, Throwable> callback)
+            protected void start(BiConsumer<? super Void, Throwable> callback)
             {
-                executor.execute(AsyncChains.encapsulate(runnable, callback));
+                try
+                {
+                    executor.execute(AsyncChains.encapsulate(runnable, callback));
+                }
+                catch (Throwable t)
+                {
+                    callback.accept(null, t);
+                }
             }
         };
     }
 
-    public static <V> AsyncChain<List<V>> all(List<AsyncChain<V>> chains)
+    @VisibleForImplementation
+    public static AsyncChain<Void> ofRunnables(Executor executor, Iterable<? extends Runnable> runnables)
     {
-        Preconditions.checkArgument(!chains.isEmpty());
+        return ofRunnable(executor, () -> {
+            Throwable failure = null;
+            for (Runnable runnable : runnables)
+            {
+                try
+                {
+                    runnable.run();
+                }
+                catch (Throwable t)
+                {
+                    if (failure == null)
+                        failure = t;
+                    else
+                        failure.addSuppressed(t);
+                }
+            }
+            if (failure != null)
+                throw new RuntimeException(failure);
+        });
+    }
+
+    public static <V> AsyncChain<List<V>> all(List<? extends AsyncChain<? extends V>> chains)
+    {
+        Invariants.checkArgument(!chains.isEmpty());
         return new AsyncChainCombiner.All<>(chains);
     }
 
-    public static <V> AsyncChain<V> reduce(List<AsyncChain<V>> chains, BiFunction<V, V, V> reducer)
+    public static <V> AsyncChain<V> reduce(List<? extends AsyncChain<? extends V>> chains, BiFunction<V, V, V> reducer)
     {
-        Preconditions.checkArgument(!chains.isEmpty());
+        Invariants.checkArgument(!chains.isEmpty());
         if (chains.size() == 1)
-            return chains.get(0);
+            return (AsyncChain<V>) chains.get(0);
         if (Reduce.canAppendTo(chains.get(0), reducer))
         {
             AsyncChainCombiner.Reduce<V> appendTo = (AsyncChainCombiner.Reduce<V>) chains.get(0);
@@ -417,20 +480,39 @@
         else throw new ExecutionException(result.failure);
     }
 
-    public static <V> V getUninterruptibly(AsyncChain<V> chain)
+    public static <V> V getUninterruptibly(AsyncChain<V> chain) throws ExecutionException
     {
+        boolean interrupted = false;
         try
         {
-            return getBlocking(chain);
+            while (true)
+            {
+                try
+                {
+                    return getBlocking(chain);
+                }
+                catch (InterruptedException e)
+                {
+                    interrupted = true;
+                }
+            }
         }
-        catch (ExecutionException | InterruptedException e)
+        finally
         {
-            throw new RuntimeException(e);
+            if (interrupted)
+                Thread.currentThread().interrupt();
         }
     }
 
-    public static <V> void awaitUninterruptibly(AsyncChain<V> chain)
+    public static void awaitUninterruptibly(AsyncChain<?> chain)
     {
-        getUninterruptibly(chain);
+        try
+        {
+            getUninterruptibly(chain);
+        }
+        catch (ExecutionException e)
+        {
+            // ignore
+        }
     }
 }
\ No newline at end of file
diff --git a/accord-core/src/main/java/accord/utils/async/AsyncResult.java b/accord-core/src/main/java/accord/utils/async/AsyncResult.java
index da8bf8b..59f97b7 100644
--- a/accord-core/src/main/java/accord/utils/async/AsyncResult.java
+++ b/accord-core/src/main/java/accord/utils/async/AsyncResult.java
@@ -27,49 +27,38 @@
  */
 public interface AsyncResult<V> extends AsyncChain<V>
 {
+    @Override
     AsyncResult<V> addCallback(BiConsumer<? super V, Throwable> callback);
 
+    @Override
     default AsyncResult<V> addCallback(Runnable runnable)
     {
-        return addCallback((unused, failure) -> {
-            if (failure == null) runnable.run();
-            else throw new RuntimeException(failure);
-        });
+        return addCallback(AsyncCallbacks.toCallback(runnable));
     }
 
+    @Override
+    default AsyncResult<V> addCallback(BiConsumer<? super V, Throwable> callback, Executor executor)
+    {
+        return addCallback(AsyncCallbacks.inExecutor(callback, executor));
+    }
+
+    @Override
     default AsyncResult<V> addCallback(Runnable runnable, Executor executor)
     {
-        addCallback(AsyncCallbacks.inExecutor(runnable, executor));
-        return this;
+        return addCallback(AsyncCallbacks.inExecutor(runnable, executor));
     }
 
     boolean isDone();
     boolean isSuccess();
 
-    default AsyncResult<V> addCallback(BiConsumer<? super V, Throwable> callback, Executor executor)
-    {
-        addCallback(AsyncCallbacks.inExecutor(callback, executor));
-        return this;
-    }
-
-    default AsyncResult<V> addListener(Runnable runnable)
-    {
-        addCallback(runnable);
-        return this;
-    }
-
-    default AsyncResult<V> addListener(Runnable runnable, Executor executor)
-    {
-        addCallback(runnable, executor);
-        return this;
-    }
-
     @Override
     default void begin(BiConsumer<? super V, Throwable> callback)
     {
+        //TODO chain shouldn't allow double calling, but should result allow?
         addCallback(callback);
     }
 
+    @Override
     default AsyncResult<V> beginAsResult()
     {
         return this;
diff --git a/accord-core/src/main/java/accord/utils/async/AsyncResults.java b/accord-core/src/main/java/accord/utils/async/AsyncResults.java
index dd9032b..7c41ca0 100644
--- a/accord-core/src/main/java/accord/utils/async/AsyncResults.java
+++ b/accord-core/src/main/java/accord/utils/async/AsyncResults.java
@@ -18,14 +18,12 @@
 
 package accord.utils.async;
 
-import com.google.common.base.Preconditions;
+import accord.api.VisibleForImplementation;
+import accord.utils.Invariants;
 
-import java.util.List;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import java.util.function.BiConsumer;
-import java.util.function.BiFunction;
 import java.util.function.Function;
 
 public class AsyncResults
@@ -91,25 +89,18 @@
             return trySetResult(new Result<>(result, failure));
         }
 
-        void setResult(Result<V> result)
-        {
-            if (!trySetResult(result))
-                throw new IllegalStateException("Result has already been set on " + this);
-        }
-
         private  AsyncChain<V> newChain()
         {
             return new AsyncChains.Head<V>()
             {
                 @Override
-                public void begin(BiConsumer<? super V, Throwable> callback)
+                protected void start(BiConsumer<? super V, Throwable> callback)
                 {
                     AbstractResult.this.addCallback(callback);
                 }
             };
         }
 
-
         void setResult(V result, Throwable failure)
         {
             if (!trySetResult(result, failure))
@@ -139,7 +130,7 @@
                 {
                     Result<V> result = (Result<V>) current;
                     callback.accept(result.value, result.failure);
-                    return null;
+                    return this;
                 }
                 if (listener == null)
                     listener = new Listener<>(callback);
@@ -162,6 +153,29 @@
             Object current = state;
             return current instanceof Result && ((Result) current).failure == null;
         }
+
+        private Result<V> getResult()
+        {
+            Object current = state;
+            Invariants.checkState(current instanceof Result);
+            return (Result<V>) current;
+        }
+
+        public V result()
+        {
+            Result<V> result = getResult();
+            if (result.failure != null)
+                throw new IllegalStateException("Result failed", result.failure);
+            return result.value;
+        }
+
+        public Throwable failure()
+        {
+            Result<V> result = getResult();
+            if (result.failure == null)
+                throw new IllegalStateException("Result succeeded");
+            return result.failure;
+        }
     }
 
     static class Chain<V> extends AbstractResult<V>
@@ -172,7 +186,7 @@
         }
     }
 
-    public static class Settable<V> extends AbstractResult<V> implements AsyncResult.Settable<V>
+    public static class SettableResult<V> extends AbstractResult<V> implements AsyncResult.Settable<V>
     {
         @Override
         public boolean trySuccess(V value)
@@ -209,7 +223,7 @@
             return new AsyncChains.Head<V>()
             {
                 @Override
-                public void begin(BiConsumer<? super V, Throwable> callback)
+                protected void start(BiConsumer<? super V, Throwable> callback)
                 {
                     AsyncResults.Immediate.this.addCallback(callback);
                 }
@@ -266,103 +280,50 @@
         return new Immediate<>(failure);
     }
 
-    public static <V> AsyncResult<V> ofCallable(Executor executor, Callable<V> callable)
-    {
-        Settable<V> result = new Settable<V>();
-        executor.execute(() -> {
-            try
-            {
-                result.trySuccess(callable.call());
-            }
-            catch (Exception e)
-            {
-                result.tryFailure(e);
-            }
-        });
-        return result;
-    }
-
-    public static AsyncResult<Void> ofRunnable(Executor executor, Runnable runnable)
-    {
-        Settable<Void> result = new Settable<Void>();
-        executor.execute(() -> {
-            try
-            {
-                runnable.run();
-                result.trySuccess(null);
-            }
-            catch (Exception e)
-            {
-                result.tryFailure(e);
-            }
-        });
-        return result;
-    }
-
     public static <V> AsyncResult.Settable<V> settable()
     {
-        return new Settable<>();
+        return new SettableResult<>();
     }
 
-    public static <V> AsyncChain<List<V>> all(List<AsyncChain<V>> results)
+    /**
+     * An AsyncResult that also implements Runnable
+     * @param <V>
+     */
+    @VisibleForImplementation
+    public static class RunnableResult<V> extends AbstractResult<V> implements Runnable
     {
-        Preconditions.checkArgument(!results.isEmpty());
-        return new AsyncChainCombiner.All<>(results);
-    }
+        private final Callable<V> callable;
 
-    public static <V> AsyncChain<V> reduce(List<AsyncChain<V>> results, BiFunction<V, V, V> reducer)
-    {
-        Preconditions.checkArgument(!results.isEmpty());
-        if (results.size() == 1)
-            return results.get(0);
-        return new AsyncChainCombiner.Reduce<>(results, reducer);
-    }
-
-    public static <V> V getBlocking(AsyncResult<V> asyncResult) throws InterruptedException, ExecutionException
-    {
-        AtomicReference<Result<V>> callbackResult = new AtomicReference<>();
-        CountDownLatch latch = new CountDownLatch(1);
-        asyncResult.addCallback((result, failure) -> {
-            callbackResult.set(new Result<>(result, failure));
-            latch.countDown();
-        });
-
-        latch.await();
-        Result<V> result = callbackResult.get();
-        if (result.failure == null) return result.value;
-        else throw new ExecutionException(result.failure);
-    }
-
-    public static <V> V getBlocking(AsyncResult<V> asyncResult, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-    {
-        AtomicReference<Result<V>> callbackResult = new AtomicReference<>();
-        CountDownLatch latch = new CountDownLatch(1);
-        asyncResult.addCallback((result, failure) -> {
-            callbackResult.set(new Result(result, failure));
-            latch.countDown();
-        });
-
-        if (!latch.await(timeout, unit))
-            throw new TimeoutException();
-        Result<V> result = callbackResult.get();
-        if (result.failure == null) return result.value;
-        else throw new ExecutionException(result.failure);
-    }
-
-    public static <V> V getUninterruptibly(AsyncResult<V> asyncResult)
-    {
-        try
+        public RunnableResult(Callable<V> callable)
         {
-            return getBlocking(asyncResult);
+            this.callable = callable;
         }
-        catch (ExecutionException | InterruptedException e)
+
+        @Override
+        public void run()
         {
-            throw new RuntimeException(e);
+            try
+            {
+                trySetResult(callable.call(), null);
+            }
+            catch (Throwable t)
+            {
+                trySetResult(null, t);
+            }
         }
     }
 
-    public static <V> void awaitUninterruptibly(AsyncResult<V> asyncResult)
+    public static <V> RunnableResult<V> runnableResult(Callable<V> callable)
     {
-        getUninterruptibly(asyncResult);
+        return new RunnableResult<>(callable);
     }
+
+    public static RunnableResult<Void> runnableResult(Runnable runnable)
+    {
+        return new RunnableResult<>(() -> {
+            runnable.run();
+            return null;
+        });
+    }
+
 }
diff --git a/accord-core/src/test/java/accord/burn/BurnTest.java b/accord-core/src/test/java/accord/burn/BurnTest.java
index b458e1d..f144032 100644
--- a/accord-core/src/test/java/accord/burn/BurnTest.java
+++ b/accord-core/src/test/java/accord/burn/BurnTest.java
@@ -296,9 +296,8 @@
 
     public static void main(String[] args) throws Exception
     {
-//        Long overrideSeed = null;
+        Long overrideSeed = null;
         int count = 1;
-        Long overrideSeed = 8602265915508619975L;
         LongSupplier seedGenerator = ThreadLocalRandom.current()::nextLong;
         for (int i = 0 ; i < args.length ; i += 2)
         {
diff --git a/accord-core/src/test/java/accord/burn/BurnTestConfigurationService.java b/accord-core/src/test/java/accord/burn/BurnTestConfigurationService.java
index 06309b0..d2dd593 100644
--- a/accord-core/src/test/java/accord/burn/BurnTestConfigurationService.java
+++ b/accord-core/src/test/java/accord/burn/BurnTestConfigurationService.java
@@ -206,7 +206,7 @@
         }
     }
 
-    private class FetchTopology extends AsyncResults.Settable<Void> implements Callback<FetchTopologyReply>
+    private class FetchTopology extends AsyncResults.SettableResult<Void> implements Callback<FetchTopologyReply>
     {
         private final FetchTopologyRequest request;
         private final List<Node.Id> candidates;
@@ -280,14 +280,14 @@
         if (topology.epoch() > lastReceived + 1)
         {
             fetchTopologyForEpoch(lastReceived + 1);
-            epochs.receiveFuture(lastReceived + 1).addListener(() -> reportTopology(topology));
+            epochs.receiveFuture(lastReceived + 1).addCallback(() -> reportTopology(topology));
             return;
         }
 
         long lastAcked = epochs.lastAcknowledged;
         if (topology.epoch() > lastAcked + 1)
         {
-            epochs.acknowledgeFuture(lastAcked + 1).addListener(() -> reportTopology(topology));
+            epochs.acknowledgeFuture(lastAcked + 1).addCallback(() -> reportTopology(topology));
             return;
         }
         logger.trace("Epoch {} received by {}", topology.epoch(), node);
diff --git a/accord-core/src/test/java/accord/burn/TopologyUpdates.java b/accord-core/src/test/java/accord/burn/TopologyUpdates.java
index cf45fb4..335446d 100644
--- a/accord-core/src/test/java/accord/burn/TopologyUpdates.java
+++ b/accord-core/src/test/java/accord/burn/TopologyUpdates.java
@@ -23,14 +23,17 @@
 import accord.coordinate.FetchData;
 import accord.impl.InMemoryCommandStores;
 import accord.local.Command;
+import accord.local.Commands;
 import accord.local.Node;
 import accord.local.Status;
 import accord.messages.CheckStatus.CheckStatusOk;
 import accord.primitives.*;
 import accord.topology.Shard;
 import accord.topology.Topology;
+import accord.utils.MapReduce;
 import accord.utils.MessageTask;
 import accord.utils.Invariants;
+import accord.utils.async.AsyncChain;
 import accord.utils.async.AsyncResult;
 import accord.utils.async.AsyncResults;
 import com.google.common.collect.Sets;
@@ -39,6 +42,7 @@
 
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
 import java.util.function.BiConsumer;
 import java.util.function.Consumer;
 import java.util.function.Function;
@@ -48,7 +52,7 @@
 import static accord.local.PreLoadContext.contextFor;
 import static accord.local.Status.*;
 import static accord.local.Status.Known.*;
-import static accord.utils.async.AsyncChains.awaitUninterruptibly;
+import static accord.utils.async.AsyncChains.getUninterruptibly;
 
 public class TopologyUpdates
 {
@@ -75,59 +79,70 @@
             this.toEpoch = trgEpoch;
         }
 
+        @Override
+        public String toString()
+        {
+            return "CommandSync{" + "txnId:" + txnId + ", fromEpoch:" + fromEpoch + ", toEpoch:" + toEpoch + '}';
+        }
+
         public void process(Node node, Consumer<Boolean> onDone)
         {
             if (!node.topology().hasEpoch(toEpoch))
             {
                 node.configService().fetchTopologyForEpoch(toEpoch);
-                node.topology().awaitEpoch(toEpoch).addListener(() -> process(node, onDone));
+                node.topology().awaitEpoch(toEpoch).addCallback(() -> process(node, onDone));
                 return;
             }
 
-            // first check if already applied locally, and respond immediately
-            Status minStatus = ((InMemoryCommandStores.Synchronized)node.commandStores()).mapReduce(contextFor(txnId), route, toEpoch, toEpoch,
-                    instance -> instance.command(txnId).status(), (a, b) -> a.compareTo(b) <= 0 ? a : b);
-
-            if (minStatus == null || minStatus.phase.compareTo(status.phase) >= 0)
-            {
-                // TODO (low priority): minStatus == null means we're sending redundant messages
-                onDone.accept(true);
-                return;
-            }
-
-            BiConsumer<Status.Known, Throwable> callback = (outcome, fail) -> {
-                if (fail != null)
-                    process(node, onDone);
-                else if (outcome == Nothing)
-                    invalidate(node, txnId, route.with(route.homeKey()), (i1, i2) -> process(node, onDone));
-                else
+            AsyncChain<Status> statusChain = node.commandStores().mapReduce(contextFor(txnId), route, toEpoch, toEpoch,
+                                                                            MapReduce.of(safeStore -> safeStore.command(txnId).current().status(),
+                                                                                         (a, b) -> a.compareTo(b) <= 0 ? a : b));
+            AsyncResult<Object> sync = statusChain.map(minStatus -> {
+                if (minStatus == null || minStatus.phase.compareTo(status.phase) >= 0)
+                {
+                    // TODO (low priority): minStatus == null means we're sending redundant messages
                     onDone.accept(true);
-            };
-            switch (status)
-            {
-                case NotWitnessed:
-                    onDone.accept(true);
-                    break;
-                case PreAccepted:
-                case Accepted:
-                case AcceptedInvalidate:
-                    FetchData.fetch(DefinitionOnly, node, txnId, route, toEpoch, callback);
-                    break;
-                case Committed:
-                case ReadyToExecute:
-                    FetchData.fetch(Committed.minKnown, node, txnId, route, toEpoch, callback);
-                    break;
-                case PreApplied:
-                case Applied:
-                    node.withEpoch(Math.max(executeAt.epoch(), toEpoch), () -> {
-                        FetchData.fetch(PreApplied.minKnown, node, txnId, route, executeAt, toEpoch, callback);
-                    });
-                    break;
-                case Invalidated:
-                    node.forEachLocal(contextFor(txnId), route, txnId.epoch(), toEpoch, safeStore -> {
-                        safeStore.command(txnId).commitInvalidate(safeStore);
-                    });
-            }
+                    return null;
+                }
+
+                BiConsumer<Known, Throwable> callback = (outcome, fail) -> {
+                    if (fail != null)
+                        process(node, onDone);
+                    else if (outcome == Nothing)
+                        invalidate(node, txnId, route.with(route.homeKey()), (i1, i2) -> process(node, onDone));
+                    else
+                        onDone.accept(true);
+                };
+                switch (status)
+                {
+                    case NotWitnessed:
+                        onDone.accept(true);
+                        break;
+                    case PreAccepted:
+                    case Accepted:
+                    case AcceptedInvalidate:
+                        FetchData.fetch(DefinitionOnly, node, txnId, route, toEpoch, callback);
+                        break;
+                    case Committed:
+                    case ReadyToExecute:
+                        FetchData.fetch(Committed.minKnown, node, txnId, route, toEpoch, callback);
+                        break;
+                    case PreApplied:
+                    case Applied:
+                        node.withEpoch(Math.max(executeAt.epoch(), toEpoch), () -> {
+                            FetchData.fetch(PreApplied.minKnown, node, txnId, route, executeAt, toEpoch, callback);
+                        });
+                        break;
+                    case Invalidated:
+                        AsyncChain<Void> invalidate = node.forEachLocal(contextFor(txnId), route, txnId.epoch(), toEpoch, safeStore -> {
+                            Commands.commitInvalidate(safeStore, txnId);
+                        });
+
+                        dieExceptionally(invalidate.addCallback(((unused, failure) -> onDone.accept(failure == null))).beginAsResult());
+                }
+                return null;
+            }).beginAsResult();
+            dieExceptionally(sync);
         }
     }
 
@@ -170,18 +185,18 @@
         return result;
     }
 
-    private static Stream<MessageTask> syncEpochCommands(Node node, long srcEpoch, Ranges ranges, Function<CommandSync, Collection<Node.Id>> recipients, long trgEpoch, boolean committedOnly)
+    private static Stream<MessageTask> syncEpochCommands(Node node, long srcEpoch, Ranges ranges, Function<CommandSync, Collection<Node.Id>> recipients, long trgEpoch, boolean committedOnly) throws ExecutionException
     {
         Map<TxnId, CheckStatusOk> syncMessages = new ConcurrentHashMap<>();
         Consumer<Command> commandConsumer = command -> syncMessages.merge(command.txnId(), new CheckStatusOk(node, command), CheckStatusOk::merge);
         if (committedOnly)
-            awaitUninterruptibly(node.commandStores().forEach(commandStore -> InMemoryCommandStore.inMemory(commandStore).forCommittedInEpoch(ranges, srcEpoch, commandConsumer)));
+            getUninterruptibly(node.commandStores().forEach(commandStore -> InMemoryCommandStore.inMemory(commandStore).forCommittedInEpoch(ranges, srcEpoch, commandConsumer)));
         else
-            awaitUninterruptibly(node.commandStores().forEach(commandStore -> InMemoryCommandStore.inMemory(commandStore).forEpochCommands(ranges, srcEpoch, commandConsumer)));
+            getUninterruptibly(node.commandStores().forEach(commandStore -> InMemoryCommandStore.inMemory(commandStore).forEpochCommands(ranges, srcEpoch, commandConsumer)));
 
         return syncMessages.entrySet().stream().map(e -> {
             CommandSync sync = new CommandSync(e.getKey(), e.getValue(), srcEpoch, trgEpoch);
-            return MessageTask.of(node, recipients.apply(sync), "Sync:" + e.getKey() + ':' + srcEpoch + ':' + trgEpoch, sync::process);
+            return MessageTask.of(node, recipients.apply(sync), sync.toString(), sync::process);
         });
     }
 
@@ -191,7 +206,7 @@
     /**
      * Syncs all replicated commands. Overkill, but useful for confirming issues in optimizedSync
      */
-    private static Stream<MessageTask> thoroughSync(Node node, long syncEpoch)
+    private static Stream<MessageTask> thoroughSync(Node node, long syncEpoch) throws ExecutionException
     {
         Topology syncTopology = node.configService().getTopologyForEpoch(syncEpoch);
         Topology localTopology = syncTopology.forNode(node.id());
@@ -209,7 +224,7 @@
     /**
      * Syncs all newly replicated commands when nodes are gaining ranges and the current epoch
      */
-    private static Stream<MessageTask> optimizedSync(Node node, long srcEpoch)
+    private static Stream<MessageTask> optimizedSync(Node node, long srcEpoch) throws ExecutionException
     {
         long trgEpoch = srcEpoch + 1;
         Topology syncTopology = node.configService().getTopologyForEpoch(srcEpoch);
@@ -258,7 +273,15 @@
 
     public static AsyncResult<Void> sync(Node node, long syncEpoch)
     {
-        Stream<MessageTask> messageStream = optimizedSync(node, syncEpoch);
+        Stream<MessageTask> messageStream;
+        try
+        {
+            messageStream = optimizedSync(node, syncEpoch);
+        }
+        catch (ExecutionException e)
+        {
+            return AsyncResults.failure(e.getCause());
+        }
 
         Iterator<MessageTask> iter = messageStream.iterator();
         if (!iter.hasNext())
@@ -271,7 +294,7 @@
         while (iter.hasNext())
         {
             MessageTask next = iter.next();
-            last.addListener(next);
+            last.addCallback(next);
             last = next;
         }
 
diff --git a/accord-core/src/test/java/accord/coordinate/CoordinateTest.java b/accord-core/src/test/java/accord/coordinate/CoordinateTest.java
index f781f4d..0d6b43a 100644
--- a/accord-core/src/test/java/accord/coordinate/CoordinateTest.java
+++ b/accord-core/src/test/java/accord/coordinate/CoordinateTest.java
@@ -31,7 +31,7 @@
 import static accord.impl.IntKey.range;
 import static accord.primitives.Routable.Domain.Key;
 import static accord.primitives.Txn.Kind.Write;
-import static accord.utils.async.AsyncResults.getUninterruptibly;
+import static accord.utils.async.AsyncChains.getUninterruptibly;
 
 public class CoordinateTest
 {
diff --git a/accord-core/src/test/java/accord/coordinate/RecoverTest.java b/accord-core/src/test/java/accord/coordinate/RecoverTest.java
index 4f5761f..a6885f7 100644
--- a/accord-core/src/test/java/accord/coordinate/RecoverTest.java
+++ b/accord-core/src/test/java/accord/coordinate/RecoverTest.java
@@ -41,7 +41,7 @@
     {
         CommandStore commandStore = getCommandShard(node, key);
         Assertions.assertTrue(inMemory(commandStore).hasCommand(txnId));
-        return inMemory(commandStore).command(txnId);
+        return inMemory(commandStore).command(txnId).value();
     }
 
     private static void assertStatus(Node node, Key key, TxnId txnId, Status status)
diff --git a/accord-core/src/test/java/accord/coordinate/TopologyChangeTest.java b/accord-core/src/test/java/accord/coordinate/TopologyChangeTest.java
index aa727d4..06d6681 100644
--- a/accord-core/src/test/java/accord/coordinate/TopologyChangeTest.java
+++ b/accord-core/src/test/java/accord/coordinate/TopologyChangeTest.java
@@ -33,15 +33,16 @@
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
+import java.util.concurrent.ExecutionException;
+
 import static accord.Utils.*;
 import static accord.impl.IntKey.keys;
 import static accord.impl.IntKey.range;
-import static accord.local.PreLoadContext.empty;
 import static accord.primitives.Routable.Domain.Key;
 import static accord.primitives.Txn.Kind.Write;
 
-import static accord.utils.async.AsyncChains.awaitUninterruptibly;
-import static accord.utils.async.AsyncResults.awaitUninterruptibly;
+import static accord.local.PreLoadContext.contextFor;
+import static accord.utils.async.AsyncChains.getUninterruptibly;
 
 public class TopologyChangeTest
 {
@@ -65,9 +66,9 @@
             Node node1 = cluster.get(1);
             TxnId txnId1 = node1.nextTxnId(Write, Key);
             Txn txn1 = writeTxn(keys);
-            awaitUninterruptibly(node1.coordinate(txnId1, txn1));
-            awaitUninterruptibly(node1.commandStores().forEach(empty(), keys, 1, 1, commands -> {
-                Command command = commands.command(txnId1);
+            getUninterruptibly(node1.coordinate(txnId1, txn1));
+            getUninterruptibly(node1.commandStores().forEach(contextFor(txnId1), keys, 1, 1, commands -> {
+                Command command = commands.command(txnId1).current();
                 Assertions.assertTrue(command.partialDeps().isEmpty());
             }));
 
@@ -76,15 +77,21 @@
             Node node4 = cluster.get(4);
             TxnId txnId2 = node4.nextTxnId(Write, Key);
             Txn txn2 = writeTxn(keys);
-            awaitUninterruptibly(node4.coordinate(txnId2, txn2));
+            getUninterruptibly(node4.coordinate(txnId2, txn2));
 
             // new nodes should have the previous epochs operation as a dependency
-            PreLoadContext context = PreLoadContext.contextFor(txnId2);
             cluster.nodes(4, 5, 6).forEach(node -> {
-                awaitUninterruptibly(node.commandStores().forEach(context, keys, 2, 2, commands -> {
-                    Command command = commands.command(txnId2);
-                    Assertions.assertTrue(command.partialDeps().contains(txnId1));
-                }));
+                try
+                {
+                    getUninterruptibly(node.commandStores().forEach(contextFor(txnId1, txnId2), keys, 2, 2, commands -> {
+                        Command command = commands.command(txnId2).current();
+                        Assertions.assertTrue(command.partialDeps().contains(txnId1));
+                    }));
+                }
+                catch (ExecutionException e)
+                {
+                    throw new AssertionError(e.getCause());
+                }
             });
         }
     }
diff --git a/accord-core/src/test/java/accord/impl/TestAgent.java b/accord-core/src/test/java/accord/impl/TestAgent.java
index 53ff4f6..0389d60 100644
--- a/accord-core/src/test/java/accord/impl/TestAgent.java
+++ b/accord-core/src/test/java/accord/impl/TestAgent.java
@@ -18,10 +18,10 @@
 
 package accord.impl;
 
+import accord.local.Command;
 import accord.local.Node;
 import accord.api.Agent;
 import accord.api.Result;
-import accord.local.Command;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 
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 2d6bb1f..f36243a 100644
--- a/accord-core/src/test/java/accord/impl/list/ListAgent.java
+++ b/accord-core/src/test/java/accord/impl/list/ListAgent.java
@@ -21,10 +21,10 @@
 import java.util.function.Consumer;
 
 import accord.impl.mock.Network;
+import accord.local.Command;
 import accord.local.Node;
 import accord.api.Agent;
 import accord.api.Result;
-import accord.local.Command;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 
diff --git a/accord-core/src/test/java/accord/impl/mock/EpochSync.java b/accord-core/src/test/java/accord/impl/mock/EpochSync.java
index e8e7864..422c400 100644
--- a/accord-core/src/test/java/accord/impl/mock/EpochSync.java
+++ b/accord-core/src/test/java/accord/impl/mock/EpochSync.java
@@ -22,19 +22,20 @@
 import accord.coordinate.tracking.QuorumTracker;
 import accord.local.*;
 import accord.messages.*;
-import accord.primitives.KeyRoute;
 import accord.primitives.Route;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 import accord.topology.Topologies.Single;
 import accord.topology.Topology;
 
+import accord.utils.async.AsyncChains;
 import accord.utils.async.AsyncResults;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
 import java.util.function.Consumer;
 
 import static accord.impl.InMemoryCommandStore.inMemory;
@@ -95,7 +96,7 @@
         }
     }
 
-    private static class CommandSync extends AsyncResults.Settable<Void> implements Callback<SimpleReply>
+    private static class CommandSync extends AsyncResults.SettableResult<Void> implements Callback<SimpleReply>
     {
         private final QuorumTracker tracker;
 
@@ -127,9 +128,8 @@
             tryFailure(failure);
         }
 
-        public static void sync(Node node, Route<?> route, SyncCommitted message, Topology topology)
-        {
-            AsyncResults.getUninterruptibly(new CommandSync(node, route, message, topology));
+        public static void sync(Node node, Route<?> route, SyncCommitted message, Topology topology) throws ExecutionException {
+            AsyncChains.getUninterruptibly(new CommandSync(node, route, message, topology));
         }
     }
 
@@ -174,10 +174,17 @@
             Map<TxnId, SyncCommitted> syncMessages = new ConcurrentHashMap<>();
             Consumer<Command> commandConsumer = command -> syncMessages.computeIfAbsent(command.txnId(), id -> new SyncCommitted(command, syncEpoch))
                     .update(command);
-            getUninterruptibly(node.commandStores().forEach(commandStore -> inMemory(commandStore).forCommittedInEpoch(syncTopology.ranges(), syncEpoch, commandConsumer)));
+            try
+            {
+                getUninterruptibly(node.commandStores().forEach(commandStore -> inMemory(commandStore).forCommittedInEpoch(syncTopology.ranges(), syncEpoch, commandConsumer)));
 
-            for (SyncCommitted send : syncMessages.values())
-                CommandSync.sync(node, send.route, send, nextTopology);
+                for (SyncCommitted send : syncMessages.values())
+                    CommandSync.sync(node, send.route, send, nextTopology);
+            }
+            catch (ExecutionException e)
+            {
+                throw new RuntimeException(e.getCause());
+            }
 
             SyncComplete syncComplete = new SyncComplete(syncEpoch);
             node.send(nextTopology.nodes(), syncComplete);
diff --git a/accord-core/src/test/java/accord/local/CommandTest.java b/accord-core/src/test/java/accord/local/ImmutableCommandTest.java
similarity index 60%
rename from accord-core/src/test/java/accord/local/CommandTest.java
rename to accord-core/src/test/java/accord/local/ImmutableCommandTest.java
index 5867913..e8f965d 100644
--- a/accord-core/src/test/java/accord/local/CommandTest.java
+++ b/accord-core/src/test/java/accord/local/ImmutableCommandTest.java
@@ -21,10 +21,6 @@
 import accord.api.ProgressLog;
 import accord.api.RoutingKey;
 import accord.api.TestableConfigurationService;
-import accord.impl.InMemoryCommandStores;
-import accord.impl.IntKey;
-import accord.impl.TestAgent;
-import accord.impl.TopologyFactory;
 import accord.impl.*;
 import accord.impl.mock.MockCluster;
 import accord.impl.mock.MockConfigurationService;
@@ -38,10 +34,10 @@
 import org.junit.jupiter.api.Test;
 
 import javax.annotation.Nullable;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Consumer;
 
@@ -52,7 +48,7 @@
 import static accord.primitives.Txn.Kind.Write;
 import static accord.utils.async.AsyncChains.getUninterruptibly;
 
-public class CommandTest
+public class ImmutableCommandTest
 {
     private static final Node.Id ID1 = id(1);
     private static final Node.Id ID2 = id(2);
@@ -76,67 +72,24 @@
         topology.set(topology.get().withEpoch(epoch));
     }
 
-    private static CommandStore createStore(CommandStoreSupport storeSupport)
+    private static InMemoryCommandStore createStore(CommandStoreSupport storeSupport)
     {
-        return createNode(ID1, storeSupport).unsafeByIndex(0);
+        return (InMemoryCommandStore) createNode(ID1, storeSupport).unsafeByIndex(0);
     }
 
     private static class NoOpProgressLog implements ProgressLog
     {
-        @Override
-        public void unwitnessed(TxnId txnId, RoutingKey homeKey, ProgressShard shard)
-        {
-        }
-
-        @Override
-        public void preaccepted(Command command, ProgressShard shard)
-        {
-        }
-
-        @Override
-        public void accepted(Command command, ProgressShard shard)
-        {
-        }
-
-        @Override
-        public void committed(Command command, ProgressShard shard)
-        {
-        }
-
-        @Override
-        public void readyToExecute(Command command, ProgressShard shard)
-        {
-        }
-
-        @Override
-        public void executed(Command command, ProgressShard shard)
-        {
-        }
-
-        @Override
-        public void invalidated(Command command, ProgressShard shard)
-        {
-        }
-
-        @Override
-        public void durableLocal(TxnId txnId)
-        {
-        }
-
-        @Override
-        public void durable(Command command, @Nullable Set<Id> persistedOn)
-        {
-        }
-
-        @Override
-        public void durable(TxnId txnId, @Nullable Unseekables<?, ?> unseekables, ProgressShard shard)
-        {
-        }
-
-        @Override
-        public void waiting(TxnId blockedBy, Known blockedUntil, Unseekables<?, ?> blockedOn)
-        {
-        }
+        @Override public void unwitnessed(TxnId txnId, RoutingKey homeKey, ProgressShard shard) {}
+        @Override public void preaccepted(Command command, ProgressShard shard) {}
+        @Override public void accepted(Command command, ProgressShard shard) {}
+        @Override public void committed(Command command, ProgressShard shard) {}
+        @Override public void readyToExecute(Command command, ProgressShard shard) {}
+        @Override public void executed(Command command, ProgressShard shard) {}
+        @Override public void invalidated(Command command, ProgressShard shard) {}
+        @Override public void durableLocal(TxnId txnId) {}
+        @Override public void durable(Command command, @Nullable Set<Id> persistedOn) {}
+        @Override public void durable(TxnId txnId, @Nullable Unseekables<?, ?> unseekables, ProgressShard shard) {}
+        @Override public void waiting(TxnId blockedBy, Known blockedUntil, Unseekables<?, ?> blockedOn) {}
     }
 
     private static Node createNode(Id id, CommandStoreSupport storeSupport)
@@ -150,39 +103,51 @@
     void noConflictWitnessTest()
     {
         CommandStoreSupport support = new CommandStoreSupport();
-        CommandStore commands = createStore(support);
+        InMemoryCommandStore commands = createStore(support);
         MockCluster.Clock clock = new MockCluster.Clock(100);
         TxnId txnId = clock.idForNode(1, 1);
-        Txn txn = writeTxn(Keys.of(KEY));
+        Keys keys = Keys.of(KEY);
+        Txn txn = writeTxn(keys);
 
-        Command command = new InMemoryCommand(commands, txnId);
-        Assertions.assertEquals(Status.NotWitnessed, command.status());
-        Assertions.assertNull(command.executeAt());
-
-        command.preaccept(inMemory(commands), txn.slice(FULL_RANGES, true), ROUTE, HOME_KEY);
+        {
+            Command command = Command.NotWitnessed.notWitnessed(txnId);
+            Assertions.assertNull(inMemory(commands).command(txnId).value());
+            Assertions.assertEquals(Status.NotWitnessed, command.status());
+            Assertions.assertNull(command.executeAt());
+        }
+        SafeCommandStore safeStore = commands.beginOperation(PreLoadContext.contextFor(txnId, keys));
+        Commands.preaccept(safeStore, txnId, txn.slice(FULL_RANGES, true), ROUTE, HOME_KEY);
+        Command command = safeStore.command(txnId).current();
         Assertions.assertEquals(Status.PreAccepted, command.status());
         Assertions.assertEquals(txnId, command.executeAt());
     }
 
     @Test
-    void supersedingEpochWitnessTest()
-    {
+    void supersedingEpochWitnessTest() throws ExecutionException {
         CommandStoreSupport support = new CommandStoreSupport();
         Node node = createNode(ID1, support);
         CommandStore commands = node.unsafeByIndex(0);
         TxnId txnId = node.nextTxnId(Write, Key);
         ((MockCluster.Clock)node.unsafeGetNowSupplier()).increment(10);
-        Txn txn = writeTxn(Keys.of(KEY));
+        Keys keys = Keys.of(KEY);
+        Txn txn = writeTxn(keys);
 
-        Command command = new InMemoryCommand(commands, txnId);
-        Assertions.assertEquals(Status.NotWitnessed, command.status());
-        Assertions.assertNull(command.executeAt());
+        {
+            Command command = Command.NotWitnessed.notWitnessed(txnId);
+            Assertions.assertNull(inMemory(commands).command(txnId).value());
+            Assertions.assertEquals(Status.NotWitnessed, command.status());
+            Assertions.assertNull(command.executeAt());
+        }
+        PreLoadContext context = PreLoadContext.contextFor(txnId, keys);
 
         setTopologyEpoch(support.local, 2);
         ((TestableConfigurationService)node.configService()).reportTopology(support.local.get().withEpoch(2));
         Timestamp expectedTimestamp = Timestamp.fromValues(2, 110, ID1);
-        getUninterruptibly(commands.execute(null, (Consumer<? super SafeCommandStore>) store -> command.preaccept(store, txn.slice(FULL_RANGES, true), ROUTE, HOME_KEY)));
-        Assertions.assertEquals(Status.PreAccepted, command.status());
-        Assertions.assertEquals(expectedTimestamp, command.executeAt());
+        getUninterruptibly(commands.execute(context, (Consumer<? super SafeCommandStore>) store -> Commands.preaccept(store, txnId, txn.slice(FULL_RANGES, true), ROUTE, HOME_KEY)));
+        commands.execute(PreLoadContext.contextFor(txnId, txn.keys()), safeStore -> {
+            Command command = safeStore.command(txnId).current();
+            Assertions.assertEquals(Status.PreAccepted, command.status());
+            Assertions.assertEquals(expectedTimestamp, command.executeAt());
+        });
     }
 }
diff --git a/accord-core/src/test/java/accord/messages/PreAcceptTest.java b/accord-core/src/test/java/accord/messages/PreAcceptTest.java
index d041679..23b7b82 100644
--- a/accord-core/src/test/java/accord/messages/PreAcceptTest.java
+++ b/accord-core/src/test/java/accord/messages/PreAcceptTest.java
@@ -20,7 +20,8 @@
 
 import accord.api.RoutingKey;
 import accord.impl.*;
-import accord.impl.InMemoryCommandsForKey.InMemoryCommandTimeseries;
+import accord.impl.CommandsForKey.CommandLoader;
+import accord.impl.CommandsForKey.CommandTimeseries;
 import accord.impl.IntKey.Raw;
 import accord.impl.mock.*;
 import accord.local.Node;
@@ -39,6 +40,8 @@
 
 import java.util.List;
 import java.util.Random;
+import java.util.function.BiFunction;
+import java.util.stream.Stream;
 
 import static accord.Utils.*;
 import static accord.impl.InMemoryCommandStore.inMemory;
@@ -85,6 +88,11 @@
         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()
     {
@@ -105,8 +113,12 @@
             clock.increment(10);
             preAccept.process(node, ID2, REPLY_CONTEXT);
 
-            Command command = ((InMemoryCommandTimeseries)inMemory(commandStore).commandsForKey(key).byId()).all().findFirst().get();
-            Assertions.assertEquals(Status.PreAccepted, command.status());
+            commandStore.execute(PreLoadContext.contextFor(txnId, txn.keys()), safeStore -> {
+                CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).commandsForKey(key).current();
+                TxnId commandId = convert(cfk.byId(), CommandLoader::txnId).findFirst().get();
+                Command command = safeStore.command(commandId).current();
+                Assertions.assertEquals(Status.PreAccepted, command.status());
+            });
 
             messageSink.assertHistorySizes(0, 1);
             Assertions.assertEquals(ID2, messageSink.responses.get(0).to);
@@ -234,8 +246,12 @@
             clock.increment(10);
             preAccept.process(node, ID2, REPLY_CONTEXT);
 
-            Command command = ((InMemoryCommandTimeseries)inMemory(commandStore).commandsForKey(key).byId()).all().findFirst().get();
-            Assertions.assertEquals(Status.PreAccepted, command.status());
+            commandStore.execute(PreLoadContext.contextFor(txnId, txn.keys()), safeStore -> {
+                CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).commandsForKey(key).current();
+                TxnId commandId = convert(cfk.byId(), CommandLoader::txnId).findFirst().get();
+                Command command = safeStore.command(commandId).current();
+                Assertions.assertEquals(Status.PreAccepted, command.status());
+            });
 
             messageSink.assertHistorySizes(0, 1);
             Assertions.assertEquals(ID2, messageSink.responses.get(0).to);
diff --git a/accord-core/src/test/java/accord/topology/TopologyRandomizer.java b/accord-core/src/test/java/accord/topology/TopologyRandomizer.java
index 371c9ef..56e8656 100644
--- a/accord-core/src/test/java/accord/topology/TopologyRandomizer.java
+++ b/accord-core/src/test/java/accord/topology/TopologyRandomizer.java
@@ -98,8 +98,8 @@
 
         shards[idx] = new Shard(IntHashKey.range((Hash)leftRange.start(), newBound), left.nodes, left.fastPathElectorate, left.joining);
         shards[idx+1] = new Shard(IntHashKey.range(newBound, (Hash)rightRange.end()), right.nodes, right.fastPathElectorate, right.joining);
-        logger.debug("Updated boundary on {} & {} {} {} to {} {}", idx, idx + 1, left, right,
-                     shards[idx].toString(true), shards[idx + 1].toString(true));
+//        logger.debug("Updated boundary on {} & {} {} {} to {} {}", idx, idx + 1, left, right,
+//                     shards[idx].toString(true), shards[idx + 1].toString(true));
 
         return shards;
     }
@@ -148,10 +148,10 @@
 
         shards[idxLeft] = new Shard(shardLeft.range, nodesLeft, newFastPath(nodesLeft, random), shardLeft.joining);
         shards[idxRight] = new Shard(shardRight.range, nodesRight, newFastPath(nodesRight, random), shardLeft.joining);
-        logger.debug("updated membership on {} & {} {} {} to {} {}",
-                    idxLeft, idxRight,
-                    shardLeft.toString(true), shardRight.toString(true),
-                    shards[idxLeft].toString(true), shards[idxRight].toString(true));
+//        logger.debug("updated membership on {} & {} {} {} to {} {}",
+//                    idxLeft, idxRight,
+//                    shardLeft.toString(true), shardRight.toString(true),
+//                    shards[idxLeft].toString(true), shards[idxRight].toString(true));
 
         return shards;
     }
@@ -179,7 +179,7 @@
         int idx = random.nextInt(shards.length);
         Shard shard = shards[idx];
         shards[idx] = new Shard(shard.range, shard.nodes, newFastPath(shard.nodes, random), shard.joining);
-        logger.debug("Updated fast path on {} {} to {}", idx, shard.toString(true), shards[idx].toString(true));
+//        logger.debug("Updated fast path on {} {} to {}", idx, shard.toString(true), shards[idx].toString(true));
         return shards;
     }
 
@@ -227,7 +227,7 @@
         Topology current = epochs.get(epochs.size() - 1);
         Shard[] shards = current.unsafeGetShards().clone();
         int mutations = random.nextInt(current.size());
-        logger.debug("Updating topology with {} mutations", mutations);
+//        logger.debug("Updating topology with {} mutations", mutations);
         for (int i=0; i<mutations; i++)
         {
             shards = UpdateType.kind(random).apply(shards, random);
@@ -250,7 +250,7 @@
             previouslyReplicated.put(entry.getKey(), merged);
         }
 
-        logger.debug("topology update to: {} from: {}", nextTopology, current);
+//        logger.debug("topology update to: {} from: {}", nextTopology, current);
         epochs.add(nextTopology);
 
         List<Node.Id> nodes = new ArrayList<>(nextTopology.nodes());
diff --git a/accord-core/src/test/java/accord/utils/Gens.java b/accord-core/src/test/java/accord/utils/Gens.java
index cc56b12..0aa7d0f 100644
--- a/accord-core/src/test/java/accord/utils/Gens.java
+++ b/accord-core/src/test/java/accord/utils/Gens.java
@@ -18,19 +18,40 @@
 
 package accord.utils;
 
-import accord.utils.Invariants;
-
 import java.lang.reflect.Array;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
+import java.util.function.Supplier;
 
 public class Gens {
     private Gens() {
     }
 
+    public static <T> Gen<T> constant(T constant)
+    {
+        return ignore -> constant;
+    }
+
+    public static <T> Gen<T> constant(Supplier<T> constant)
+    {
+        return ignore -> constant.get();
+    }
+
+    public static <T> Gen<T> pick(T... ts)
+    {
+        return pick(Arrays.asList(ts));
+    }
+
+    public static <T> Gen<T> pick(List<T> ts)
+    {
+        Gen.IntGen offset = ints().between(0, ts.size() - 1);
+        return rs -> ts.get(offset.nextInt(rs));
+    }
+
     public static Gen<Gen.Random> random() {
         return r -> r;
     }
@@ -60,6 +81,11 @@
         return new LongArrayDSL(fn);
     }
 
+    public static EnumDSL enums()
+    {
+        return new EnumDSL();
+    }
+
     public static class IntDSL
     {
         public Gen.IntGen of(int value)
@@ -107,6 +133,14 @@
         }
     }
 
+    public static class EnumDSL
+    {
+        public <T extends Enum<T>> Gen<T> all(Class<T> klass)
+        {
+            return pick(klass.getEnumConstants());
+        }
+    }
+
     public static class ListDSL<T> implements BaseSequenceDSL<ListDSL<T>, List<T>> {
         private final Gen<T> fn;
 
diff --git a/accord-core/src/test/java/accord/utils/MessageTask.java b/accord-core/src/test/java/accord/utils/MessageTask.java
index 2589bb0..d868049 100644
--- a/accord-core/src/test/java/accord/utils/MessageTask.java
+++ b/accord-core/src/test/java/accord/utils/MessageTask.java
@@ -31,7 +31,7 @@
  * Message task that will continue sending messages to a set of nodes until all
  * nodes ack the message.
  */
-public class MessageTask extends AsyncResults.Settable<Void> implements Runnable
+public class MessageTask extends AsyncResults.SettableResult<Void> implements Runnable
 {
     public interface NodeProcess
     {
diff --git a/accord-core/src/test/java/accord/utils/Property.java b/accord-core/src/test/java/accord/utils/Property.java
index cf6759c..9bbbc6d 100644
--- a/accord-core/src/test/java/accord/utils/Property.java
+++ b/accord-core/src/test/java/accord/utils/Property.java
@@ -133,6 +133,11 @@
         return sb.toString();
     }
 
+    public interface FailingConsumer<A>
+    {
+        void accept(A value) throws Exception;
+    }
+
     public static class SingleBuilder<T> extends Common<SingleBuilder<T>>
     {
         private final Gen<T> gen;
@@ -142,7 +147,7 @@
             this.gen = Objects.requireNonNull(gen);
         }
 
-        public void check(Consumer<T> fn)
+        public void check(FailingConsumer<T> fn)
         {
             Random random = new Random(seed);
             for (int i = 0; i < examples; i++)
@@ -166,6 +171,11 @@
         }
     }
 
+    public interface FailingBiConsumer<A, B>
+    {
+        void accept(A a, B b) throws Exception;
+    }
+
     public static class DoubleBuilder<A, B> extends Common<DoubleBuilder<A, B>>
     {
         private final Gen<A> aGen;
@@ -177,7 +187,7 @@
             this.bGen = Objects.requireNonNull(bGen);
         }
 
-        public void check(BiConsumer<A, B> fn)
+        public void check(FailingBiConsumer<A, B> fn)
         {
             Random random = new Random(seed);
             for (int i = 0; i < examples; i++)
diff --git a/accord-core/src/test/java/accord/utils/async/AsyncChainsTest.java b/accord-core/src/test/java/accord/utils/async/AsyncChainsTest.java
index 8735aa2..12b3931 100644
--- a/accord-core/src/test/java/accord/utils/async/AsyncChainsTest.java
+++ b/accord-core/src/test/java/accord/utils/async/AsyncChainsTest.java
@@ -24,6 +24,9 @@
 import org.junit.jupiter.api.Test;
 
 import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
 import java.util.function.BiFunction;
@@ -177,4 +180,124 @@
         reduction2.begin(callback);
         Assertions.assertEquals(9, callback.value());
     }
+
+    @Test
+    void beginSeesException()
+    {
+        AsyncChains.ofCallable(ignore -> {
+                    throw new RejectedExecutionException();
+                }, () -> 42)
+                .map(i -> i + 1)
+                .begin((success, failure) -> {
+                    if (failure == null)
+                        throw new IllegalStateException("Should see failure");
+                });
+
+        AsyncChains.ofRunnable(ignore -> {
+                    throw new RejectedExecutionException();
+                }, () -> {})
+                .map(ignore -> 1)
+                .beginAsResult()
+                .addCallback((success, failure) -> {
+                    if (failure == null)
+                        throw new IllegalStateException("Expected to fail");
+                });
+
+        AsyncChains.<Integer>ofCallable(fn -> fn.run(), () -> {
+                    throw new RuntimeException("Unchecked");
+                }).map(i -> i + 1).map(i -> i + 1)
+                .begin((success, failure) -> {
+                    if (failure == null)
+                        throw new IllegalStateException("Should see failure");
+                });
+
+        AsyncChains.ofCallable(fn -> fn.run(), () -> 42
+                ).map(i -> i + 1)
+                .map(ignore -> {
+                    throw new RuntimeException("Unchecked");
+                })
+                .begin((success, failure) -> {
+                    if (failure == null)
+                        throw new IllegalStateException("Should see failure");
+                });
+    }
+
+    @Test
+    void headRejectsSecondBegin()
+    {
+        AsyncChain<String> chain = new AsyncChains.Head<String>() {
+            @Override
+            protected void start(BiConsumer<? super String, Throwable> callback) {
+                callback.accept("success", null);
+            }
+        };
+
+        chain.begin((i1, i2) -> {});
+        assertThrows(() -> chain.begin((i1, i2) -> {}));
+    }
+
+    @Test
+    void chainRejectsSecondBegin()
+    {
+        AsyncChain<String> chain = new AsyncChains.Head<String>() {
+            @Override
+            protected void start(BiConsumer<? super String, Throwable> callback) {
+                callback.accept("success", null);
+            }
+        };
+        chain = chain.map(s -> s + " is true");
+        chain.begin((i1, i2) -> {});
+        AsyncChain<String> finalChain = chain;
+        assertThrows(() -> finalChain.begin((i1, i2) -> {}));
+    }
+
+    private static void assertThrows(Runnable fn)
+    {
+        try
+        {
+            fn.run();
+            Assertions.fail("Should have been rejected");
+        }
+        catch (AssertionError e)
+        {
+            if ("Should have been rejected".equals(e.getMessage())) throw e;
+        }
+        catch (Throwable t)
+        {
+            // expected
+        }
+    }
+
+    @Test
+    void test3()
+    {
+        AtomicReference<Boolean> sawFailure = new AtomicReference<>(null);
+        AtomicBoolean sawCallback = new AtomicBoolean(false);
+        AsyncChains.failure(new NullPointerException("just kidding"))
+                .beginAsResult()
+                .addCallback(() -> sawCallback.set(true))
+                .begin((success, failure) -> {
+                    if (failure != null) sawFailure.set(true);
+                    else sawFailure.set(false);
+                });
+        Assertions.assertEquals(Boolean.TRUE, sawFailure.get());
+        Assertions.assertFalse(sawCallback.get());
+    }
+
+    @Test
+    void simpleHeadChain() throws ExecutionException, InterruptedException {
+        AsyncChain<Integer> chain = new AsyncChains.Head<Integer>() {
+            @Override
+            protected void start(BiConsumer<? super Integer, Throwable> callback) {
+                callback.accept(0, null);
+            }
+        };
+        chain = chain.map(i -> i + 1)
+                     .map(i -> i + 2)
+                     .map(i -> i + 3)
+                     .map(i -> i + 4)
+                     .map(i -> i + 5);
+
+        Assertions.assertEquals(15, AsyncChains.getBlocking(chain));
+    }
 }
diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromAgent.java b/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromAgent.java
index 598c632..7e8cdce 100644
--- a/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromAgent.java
+++ b/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromAgent.java
@@ -18,10 +18,10 @@
 
 package accord.maelstrom;
 
+import accord.local.Command;
 import accord.local.Node;
 import accord.api.Agent;
 import accord.api.Result;
-import accord.local.Command;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;