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

[cassandra-accord] branch trunk updated: Refactor routing and key API to support both Key, RoutingKey and Range consistently/transparently

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

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


The following commit(s) were added to refs/heads/trunk by this push:
     new f5d7ede  Refactor routing and key API to support both Key, RoutingKey and Range consistently/transparently
f5d7ede is described below

commit f5d7ede4a27cdcc4bf2fca87f62bf222f7b82025
Author: Benedict Elliott Smith <be...@apache.org>
AuthorDate: Mon Sep 19 15:50:23 2022 +0100

    Refactor routing and key API to support both Key, RoutingKey and Range consistently/transparently
    
    patch by Benedict; reviewed by Ariel for CASSANDRA-18087
---
 accord-core/src/main/java/accord/api/Key.java      |   8 +-
 .../src/main/java/accord/api/ProgressLog.java      |  10 +-
 accord-core/src/main/java/accord/api/Read.java     |   4 +-
 .../src/main/java/accord/api/RoutingKey.java       |  54 +--
 accord-core/src/main/java/accord/api/Update.java   |   4 +-
 .../src/main/java/accord/coordinate/CheckOn.java   |  33 +-
 .../main/java/accord/coordinate/CheckShards.java   |  18 +-
 .../main/java/accord/coordinate/CollectDeps.java   |  14 +-
 .../main/java/accord/coordinate/Coordinate.java    |  25 +-
 .../src/main/java/accord/coordinate/Execute.java   |  16 +-
 .../src/main/java/accord/coordinate/FetchData.java |  37 +-
 .../main/java/accord/coordinate/FindHomeKey.java   |  11 +-
 .../src/main/java/accord/coordinate/FindRoute.java |  16 +-
 .../main/java/accord/coordinate/Invalidate.java    |  48 +-
 .../main/java/accord/coordinate/MaybeRecover.java  |  20 +-
 .../src/main/java/accord/coordinate/Persist.java   |  21 +-
 .../src/main/java/accord/coordinate/Propose.java   |  19 +-
 .../src/main/java/accord/coordinate/Recover.java   |  37 +-
 .../java/accord/coordinate/RecoverWithHomeKey.java |  10 +-
 .../java/accord/coordinate/RecoverWithRoute.java   |  40 +-
 .../accord/coordinate/tracking/ReadTracker.java    |  12 +-
 .../src/main/java/accord/impl/InMemoryCommand.java |   6 +-
 .../java/accord/impl/InMemoryCommandStore.java     | 117 +++--
 .../java/accord/impl/InMemoryCommandStores.java    |  12 +-
 .../main/java/accord/impl/SimpleProgressLog.java   |  58 ++-
 .../main/java/accord/local/AsyncCommandStores.java |   4 +-
 .../src/main/java/accord/local/Command.java        | 187 ++++----
 .../src/main/java/accord/local/CommandStore.java   |  26 +-
 .../src/main/java/accord/local/CommandStores.java  | 112 +++--
 .../src/main/java/accord/local/CommandsForKey.java |   5 +-
 accord-core/src/main/java/accord/local/Node.java   |  63 +--
 .../src/main/java/accord/local/PreLoadContext.java |  29 +-
 .../main/java/accord/local/SafeCommandStore.java   |  14 +-
 accord-core/src/main/java/accord/local/Status.java |   7 +-
 .../main/java/accord/local/SyncCommandStores.java  |   4 +-
 .../java/accord/messages/AbstractEpochRequest.java |   9 +-
 .../src/main/java/accord/messages/Accept.java      |  17 +-
 .../src/main/java/accord/messages/Apply.java       |  19 +-
 .../java/accord/messages/BeginInvalidation.java    |  37 +-
 .../main/java/accord/messages/BeginRecovery.java   | 128 ++---
 .../src/main/java/accord/messages/CheckStatus.java |  55 +--
 .../src/main/java/accord/messages/Commit.java      |  72 +--
 .../main/java/accord/messages/EpochRequest.java    |  24 -
 .../src/main/java/accord/messages/GetDeps.java     |  32 +-
 .../main/java/accord/messages/InformDurable.java   |  17 +-
 .../main/java/accord/messages/InformOfTxnId.java   |   4 +-
 .../src/main/java/accord/messages/PreAccept.java   |  43 +-
 .../src/main/java/accord/messages/ReadData.java    |  18 +-
 .../src/main/java/accord/messages/Request.java     |   1 +
 .../src/main/java/accord/messages/TxnRequest.java  |  77 ++-
 .../main/java/accord/messages/WaitOnCommit.java    |  21 +-
 .../main/java/accord/primitives/AbstractKeys.java  | 250 ++++------
 .../java/accord/primitives/AbstractRanges.java     | 531 +++++++++++++++++++++
 .../accord/primitives/AbstractRoutableKeys.java    |  11 +
 .../main/java/accord/primitives/AbstractRoute.java |  37 --
 .../src/main/java/accord/primitives/Deps.java      | 240 ++++++----
 .../main/java/accord/primitives/FullKeyRoute.java  |  71 +++
 .../java/accord/primitives/FullRangeRoute.java     |  70 +++
 .../src/main/java/accord/primitives/FullRoute.java |   6 +
 .../src/main/java/accord/primitives/KeyRanges.java | 494 -------------------
 .../src/main/java/accord/primitives/KeyRoute.java  |  34 ++
 .../src/main/java/accord/primitives/Keys.java      |  64 +--
 .../main/java/accord/primitives/PartialDeps.java   |  33 +-
 .../java/accord/primitives/PartialKeyRoute.java    | 103 ++++
 .../java/accord/primitives/PartialRangeRoute.java  | 108 +++++
 .../main/java/accord/primitives/PartialRoute.java  |  80 +---
 .../main/java/accord/primitives/PartialTxn.java    |  60 ++-
 .../primitives/{KeyRange.java => Range.java}       | 112 ++---
 .../main/java/accord/primitives/RangeRoute.java    |  35 ++
 .../src/main/java/accord/primitives/Ranges.java    | 179 +++++++
 .../src/main/java/accord/primitives/Routable.java  |  14 +
 .../main/java/accord/primitives/RoutableKey.java   |  64 +++
 .../src/main/java/accord/primitives/Routables.java | 249 ++++++++++
 .../src/main/java/accord/primitives/Route.java     | 104 ++--
 .../main/java/accord/primitives/RoutingKeys.java   |  49 +-
 .../src/main/java/accord/primitives/Seekable.java  |   8 +
 .../src/main/java/accord/primitives/Seekables.java |  15 +
 .../src/main/java/accord/primitives/Txn.java       |  16 +-
 .../main/java/accord/primitives/Unseekable.java    |   8 +
 .../main/java/accord/primitives/Unseekables.java   |  59 +++
 .../src/main/java/accord/primitives/Writes.java    |   7 +-
 .../main/java/accord/primitives/package-info.java  |  10 +
 .../src/main/java/accord/topology/Shard.java       |  22 +-
 .../src/main/java/accord/topology/Topologies.java  |  14 +-
 .../src/main/java/accord/topology/Topology.java    | 159 +++---
 .../main/java/accord/topology/TopologyManager.java | 111 ++---
 .../src/main/java/accord/utils/ArrayBuffers.java   |  62 ++-
 .../src/main/java/accord/utils/Functions.java      |  10 +-
 .../src/main/java/accord/utils/Invariants.java     |  88 ++++
 .../src/main/java/accord/utils/SortedArrays.java   | 146 +++---
 accord-core/src/test/java/accord/KeysTest.java     |  56 +--
 accord-core/src/test/java/accord/Utils.java        |  15 +-
 .../src/test/java/accord/burn/BurnTest.java        |   2 +-
 .../accord/burn/BurnTestConfigurationService.java  |  12 +-
 .../src/test/java/accord/burn/TopologyUpdates.java |  27 +-
 .../java/accord/coordinate/CoordinateTest.java     |   6 +-
 .../accord/coordinate/PreAcceptTrackerTest.java    |   4 +-
 .../java/accord/coordinate/TopologyChangeTest.java |   4 +-
 .../coordinate/tracking/QuorumTrackerTest.java     |  10 +-
 .../coordinate/tracking/ReadTrackerTest.java       |   5 +-
 .../src/test/java/accord/impl/IntHashKey.java      |  96 ++--
 accord-core/src/test/java/accord/impl/IntKey.java  |  79 +--
 .../src/test/java/accord/impl/TopologyFactory.java |  12 +-
 .../src/test/java/accord/impl/TopologyUtils.java   |  14 +-
 .../src/test/java/accord/impl/list/ListRead.java   |   4 +-
 .../test/java/accord/impl/list/ListRequest.java    |   1 +
 .../src/test/java/accord/impl/list/ListUpdate.java |   4 +-
 .../src/test/java/accord/impl/mock/EpochSync.java  |  13 +-
 .../test/java/accord/impl/mock/MockCluster.java    |  12 +-
 .../src/test/java/accord/impl/mock/MockStore.java  |   6 +-
 .../src/test/java/accord/local/CommandTest.java    |  26 +-
 .../test/java/accord/messages/PreAcceptTest.java   |  31 +-
 .../java/accord/messages/TxnRequestScopeTest.java  |  35 +-
 .../java/accord/topology/TopologyManagerTest.java  |  28 +-
 .../java/accord/topology/TopologyRandomizer.java   |  39 +-
 .../test/java/accord/topology/TopologyTest.java    |  21 +-
 accord-core/src/test/java/accord/txn/DepsTest.java |  40 +-
 accord-core/src/test/java/accord/utils/Gens.java   |   6 +-
 .../src/test/java/accord/utils/MessageTask.java    |   6 +-
 .../utils/{KeyRangeTest.java => RangeTest.java}    |  42 +-
 .../utils/{KeyRangesTest.java => RangesTest.java}  |  30 +-
 .../test/java/accord/utils/SortedArraysTest.java   |  17 +-
 .../verify/StrictSerializabilityVerifier.java      |   2 +-
 .../src/main/java/accord/maelstrom/Datum.java      |  16 +-
 .../src/main/java/accord/maelstrom/Json.java       |  17 +-
 .../main/java/accord/maelstrom/MaelstromKey.java   |  82 +++-
 .../main/java/accord/maelstrom/MaelstromRead.java  |   4 +-
 .../main/java/accord/maelstrom/MaelstromReply.java |   2 +-
 .../java/accord/maelstrom/MaelstromRequest.java    |   6 +-
 .../java/accord/maelstrom/MaelstromResult.java     |   4 +-
 .../java/accord/maelstrom/MaelstromUpdate.java     |   4 +-
 .../java/accord/maelstrom/TopologyFactory.java     |  10 +-
 132 files changed, 3697 insertions(+), 2561 deletions(-)

diff --git a/accord-core/src/main/java/accord/api/Key.java b/accord-core/src/main/java/accord/api/Key.java
index 4d15413..5307828 100644
--- a/accord-core/src/main/java/accord/api/Key.java
+++ b/accord-core/src/main/java/accord/api/Key.java
@@ -18,10 +18,12 @@
 
 package accord.api;
 
+import accord.primitives.RoutableKey;
+import accord.primitives.Seekable;
+
 /**
- * A routing key for determining which shards are involved in a transaction
+ * A key we can find in both the cluster and on disk
  */
-public interface Key extends RoutingKey
+public interface Key extends Seekable, RoutableKey
 {
-    RoutingKey toRoutingKey();
 }
diff --git a/accord-core/src/main/java/accord/api/ProgressLog.java b/accord-core/src/main/java/accord/api/ProgressLog.java
index 47dfb6a..8d68b12 100644
--- a/accord-core/src/main/java/accord/api/ProgressLog.java
+++ b/accord-core/src/main/java/accord/api/ProgressLog.java
@@ -22,13 +22,13 @@ import java.util.Set;
 
 import javax.annotation.Nullable;
 
-import accord.coordinate.CheckOn;
 import accord.coordinate.InformHomeOfTxn;
 import accord.local.Command;
 import accord.local.CommandStore;
 import accord.local.Node.Id;
 import accord.local.Status.Known;
-import accord.primitives.*;
+import accord.primitives.Unseekables;
+import accord.primitives.TxnId;
 
 /**
  * This interface is responsible for managing incomplete transactions *and retrying them*.
@@ -169,7 +169,7 @@ public interface ProgressLog
      * If this replica has not witnessed the outcome of the transaction, it should poll a majority of each shard
      * for its outcome, using the provided route (if any).
      */
-    void durable(TxnId txnId, @Nullable RoutingKeys someKeys, ProgressShard shard);
+    void durable(TxnId txnId, @Nullable Unseekables<?, ?> unseekables, ProgressShard shard);
 
     /**
      * The parameter is a command that some other command's execution is most proximally blocked by.
@@ -189,7 +189,7 @@ public interface ProgressLog
      *
      * @param blockedBy     is the transaction id that is blocking progress
      * @param blockedUntil  either Committed or Executed; the state we are waiting for
-     * @param blockedOnKeys the keys we should locally report any progress updates to
+     * @param blockedOn the keys we should report any progress updates to
      */
-    void waiting(TxnId blockedBy, Known blockedUntil, RoutingKeys blockedOnKeys);
+    void waiting(TxnId blockedBy, Known blockedUntil, Unseekables<?, ?> blockedOn);
 }
diff --git a/accord-core/src/main/java/accord/api/Read.java b/accord-core/src/main/java/accord/api/Read.java
index 31ddf08..743b452 100644
--- a/accord-core/src/main/java/accord/api/Read.java
+++ b/accord-core/src/main/java/accord/api/Read.java
@@ -19,7 +19,7 @@
 package accord.api;
 
 import accord.local.SafeCommandStore;
-import accord.primitives.KeyRanges;
+import accord.primitives.Ranges;
 import accord.primitives.Keys;
 import accord.primitives.Timestamp;
 import accord.primitives.Txn;
@@ -75,6 +75,6 @@ public interface Read
         }
     }
 
-    Read slice(KeyRanges ranges);
+    Read slice(Ranges ranges);
     Read merge(Read other);
 }
diff --git a/accord-core/src/main/java/accord/api/RoutingKey.java b/accord-core/src/main/java/accord/api/RoutingKey.java
index 95460bc..a61bfec 100644
--- a/accord-core/src/main/java/accord/api/RoutingKey.java
+++ b/accord-core/src/main/java/accord/api/RoutingKey.java
@@ -1,55 +1,9 @@
 package accord.api;
 
-import accord.primitives.KeyRange;
+import accord.primitives.RoutableKey;
+import accord.primitives.Unseekable;
 
-import javax.annotation.Nonnull;
-
-public interface RoutingKey extends Comparable<RoutingKey>
+public interface RoutingKey extends Unseekable, RoutableKey
 {
-    /**
-     * A special RoutingKey that sorts before or after everything, so that exclusive bounds may still cover
-     * the full range of possible RoutingKey.
-     *
-     * All RoutingKey implementations must sort correctly with this type.
-     *
-     * TODO: need to partition range from/to -/+ infinity as otherwise we exclude at least one key
-     */
-    class InfiniteRoutingKey implements RoutingKey
-    {
-        public static final InfiniteRoutingKey POSITIVE_INFINITY = new InfiniteRoutingKey(1);
-        public static final InfiniteRoutingKey NEGATIVE_INFINITY = new InfiniteRoutingKey(-1);
-
-        final int compareTo;
-
-        public InfiniteRoutingKey(int compareTo)
-        {
-            this.compareTo = compareTo;
-        }
-
-        @Override
-        public int routingHash()
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public int compareTo(@Nonnull RoutingKey ignore)
-        {
-            return compareTo;
-        }
-    }
-
-    /**
-     * Implementations must be comparable with {@link InfiniteRoutingKey}
-     * @param that the object to be compared.
-     * @return
-     */
-    int compareTo(@Nonnull RoutingKey that);
-
-    /**
-     * Returns a hash code of a key to support accord internal sharding. Hash values for equal keys must be equal.
-     */
-    int routingHash();
-
-    default RoutingKey toRoutingKey() { return this; }
+    @Override default RoutingKey toUnseekable() { return this; }
 }
diff --git a/accord-core/src/main/java/accord/api/Update.java b/accord-core/src/main/java/accord/api/Update.java
index 3d6e501..e100ec4 100644
--- a/accord-core/src/main/java/accord/api/Update.java
+++ b/accord-core/src/main/java/accord/api/Update.java
@@ -18,7 +18,7 @@
 
 package accord.api;
 
-import accord.primitives.KeyRanges;
+import accord.primitives.Ranges;
 import accord.primitives.Keys;
 
 import javax.annotation.Nullable;
@@ -33,6 +33,6 @@ public interface Update
     Keys keys();
     // null is provided only if nothing was read
     Write apply(@Nullable Data data);
-    Update slice(KeyRanges ranges);
+    Update slice(Ranges ranges);
     Update merge(Update other);
 }
diff --git a/accord-core/src/main/java/accord/coordinate/CheckOn.java b/accord-core/src/main/java/accord/coordinate/CheckOn.java
index 1b65640..7819d9a 100644
--- a/accord-core/src/main/java/accord/coordinate/CheckOn.java
+++ b/accord-core/src/main/java/accord/coordinate/CheckOn.java
@@ -22,6 +22,7 @@ import java.util.function.BiConsumer;
 
 import accord.local.*;
 import accord.primitives.*;
+import accord.utils.Invariants;
 import accord.utils.MapReduceConsume;
 import com.google.common.base.Preconditions;
 
@@ -54,18 +55,18 @@ public class CheckOn extends CheckShards
      */
     final Known sufficient;
     final long untilLocalEpoch;
-    final AbstractRoute route;
+    final Route<?> route;
 
-    CheckOn(Node node, Known sufficient, TxnId txnId, AbstractRoute route, long srcEpoch, long untilLocalEpoch, BiConsumer<? super CheckStatusOkFull, Throwable> callback)
+    CheckOn(Node node, Known sufficient, TxnId txnId, Route<?> route, long srcEpoch, long untilLocalEpoch, BiConsumer<? super CheckStatusOkFull, Throwable> callback)
     {
-        this(node, sufficient, txnId, route, route.with(route.homeKey), srcEpoch, untilLocalEpoch, callback);
+        this(node, sufficient, txnId, route, route.with(route.homeKey()), srcEpoch, untilLocalEpoch, callback);
     }
 
-    CheckOn(Node node, Known sufficient, TxnId txnId, AbstractRoute route, RoutingKeys routeWithHomeKey, long srcEpoch, long untilLocalEpoch, BiConsumer<? super CheckStatusOkFull, Throwable> callback)
+    CheckOn(Node node, Known sufficient, TxnId txnId, Route<?> route, Unseekables<?, ?> routeWithHomeKey, long srcEpoch, long untilLocalEpoch, BiConsumer<? super CheckStatusOkFull, Throwable> callback)
     {
         // TODO (soon): 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));
+        Preconditions.checkArgument(routeWithHomeKey.contains(route.homeKey()));
         this.sufficient = sufficient;
         this.route = route;
         this.callback = callback;
@@ -73,14 +74,14 @@ public class CheckOn extends CheckShards
     }
 
     // TODO: many callers only need to consult precisely executeAt.epoch remotely
-    public static CheckOn checkOn(Known sufficientStatus, Node node, TxnId txnId, AbstractRoute route, long srcEpoch, long untilLocalEpoch, BiConsumer<? super CheckStatusOkFull, Throwable> callback)
+    public static CheckOn checkOn(Known sufficientStatus, Node node, TxnId txnId, Route<?> route, long srcEpoch, long untilLocalEpoch, BiConsumer<? super CheckStatusOkFull, Throwable> callback)
     {
         CheckOn checkOn = new CheckOn(node, sufficientStatus, txnId, route, srcEpoch, untilLocalEpoch, callback);
         checkOn.start();
         return checkOn;
     }
 
-    protected AbstractRoute route()
+    protected Route<?> route()
     {
         return route;
     }
@@ -88,8 +89,8 @@ public class CheckOn extends CheckShards
     @Override
     protected boolean isSufficient(Id from, CheckStatusOk ok)
     {
-        KeyRanges rangesForNode = topologies().computeRangesForNode(from);
-        PartialRoute scope = this.route.slice(rangesForNode);
+        Ranges rangesForNode = topologies().computeRangesForNode(from);
+        PartialRoute<?> scope = this.route.slice(rangesForNode);
         return isSufficient(scope, ok);
     }
 
@@ -99,7 +100,7 @@ public class CheckOn extends CheckShards
         return isSufficient(route, ok);
     }
 
-    protected boolean isSufficient(AbstractRoute scope, CheckStatusOk ok)
+    protected boolean isSufficient(Route<?> scope, CheckStatusOk ok)
     {
         return sufficient.isSatisfiedBy(((CheckStatusOkFull)ok).sufficientFor(scope));
     }
@@ -115,7 +116,7 @@ public class CheckOn extends CheckShards
         else
         {
             if (success == Success.Success)
-                Preconditions.checkState(isSufficient(merged));
+                Invariants.checkState(isSufficient(merged));
 
             if (merged.saveStatus == NotWitnessed)
                 callback.accept(CheckStatusOkFull.NOT_WITNESSED, null);
@@ -126,7 +127,7 @@ public class CheckOn extends CheckShards
 
     class OnDone implements MapReduceConsume<SafeCommandStore, Void>
     {
-        final AbstractRoute maxRoute;
+        final Route<?> maxRoute;
         final RoutingKey progressKey;
         final CheckStatusOkFull full;
         final Known sufficientFor;
@@ -135,11 +136,11 @@ public class CheckOn extends CheckShards
 
         public OnDone()
         {
-            KeyRanges localRanges = node.topology().localRangesForEpochs(txnId.epoch, untilLocalEpoch);
-            PartialRoute selfRoute = route().slice(localRanges);
+            Ranges localRanges = node.topology().localRangesForEpochs(txnId.epoch, untilLocalEpoch);
+            PartialRoute<?> selfRoute = route().slice(localRanges);
             full = (CheckStatusOkFull) merged;
             sufficientFor = full.sufficientFor(selfRoute);
-            maxRoute = Route.merge(route(), full.route);
+            maxRoute = Route.merge((Route)route(), full.route);
             progressKey = node.trySelectProgressKey(txnId, maxRoute);
 
             PartialTxn partialTxn = null;
@@ -155,7 +156,7 @@ public class CheckOn extends CheckShards
 
         void start()
         {
-            Keys keys = Keys.EMPTY;
+            Seekables<?, ?> keys = Keys.EMPTY;
             if (sufficientFor.definition.isKnown())
                 keys = partialTxn.keys();
 
diff --git a/accord-core/src/main/java/accord/coordinate/CheckShards.java b/accord-core/src/main/java/accord/coordinate/CheckShards.java
index 913db02..965e6a9 100644
--- a/accord-core/src/main/java/accord/coordinate/CheckShards.java
+++ b/accord-core/src/main/java/accord/coordinate/CheckShards.java
@@ -6,8 +6,7 @@ import accord.messages.CheckStatus;
 import accord.messages.CheckStatus.CheckStatusOk;
 import accord.messages.CheckStatus.CheckStatusReply;
 import accord.messages.CheckStatus.IncludeInfo;
-import accord.primitives.RoutingKeys;
-import accord.primitives.TxnId;
+import accord.primitives.*;
 import accord.topology.Topologies;
 
 /**
@@ -16,7 +15,7 @@ import accord.topology.Topologies;
  */
 public abstract class CheckShards extends ReadCoordinator<CheckStatusReply>
 {
-    final RoutingKeys contactKeys;
+    final Unseekables<?, ?> contact;
 
     /**
      * The epoch until which we want to fetch data from remotely
@@ -27,24 +26,23 @@ public abstract class CheckShards extends ReadCoordinator<CheckStatusReply>
 
     protected CheckStatusOk merged;
 
-    protected CheckShards(Node node, TxnId txnId, RoutingKeys contactKeys, long srcEpoch, IncludeInfo includeInfo)
+    protected CheckShards(Node node, TxnId txnId, Unseekables<?, ?> contact, long srcEpoch, IncludeInfo includeInfo)
     {
-        super(node, topologyFor(node, contactKeys, srcEpoch), txnId);
+        super(node, topologyFor(node, txnId, contact, srcEpoch), txnId);
         this.untilRemoteEpoch = srcEpoch;
-        this.contactKeys = contactKeys;
+        this.contact = contact;
         this.includeInfo = includeInfo;
     }
 
-    private static Topologies topologyFor(Node node, RoutingKeys someKeys, long epoch)
+    private static Topologies topologyFor(Node node, TxnId txnId, Unseekables<?, ?> contact, long epoch)
     {
-        // TODO (now): why were we contacting txnId.epoch...epoch?
-        return node.topology().preciseEpochs(someKeys, epoch, epoch);
+        return node.topology().preciseEpochs(contact, txnId.epoch, epoch);
     }
 
     @Override
     protected void contact(Id id)
     {
-        node.send(id, new CheckStatus(txnId, contactKeys, txnId.epoch, untilRemoteEpoch, includeInfo), this);
+        node.send(id, new CheckStatus(txnId, contact.slice(topologies().computeRangesForNode(id)), txnId.epoch, untilRemoteEpoch, includeInfo), this);
     }
 
     protected boolean isSufficient(Id from, CheckStatusOk ok) { return isSufficient(ok); }
diff --git a/accord-core/src/main/java/accord/coordinate/CollectDeps.java b/accord-core/src/main/java/accord/coordinate/CollectDeps.java
index 5b615bd..c12e4eb 100644
--- a/accord-core/src/main/java/accord/coordinate/CollectDeps.java
+++ b/accord-core/src/main/java/accord/coordinate/CollectDeps.java
@@ -10,11 +10,7 @@ import accord.local.Node.Id;
 import accord.messages.Callback;
 import accord.messages.GetDeps;
 import accord.messages.GetDeps.GetDepsOk;
-import accord.primitives.Deps;
-import accord.primitives.Route;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
+import accord.primitives.*;
 import accord.topology.Topologies;
 
 import static accord.coordinate.tracking.RequestStatus.Failed;
@@ -24,7 +20,7 @@ class CollectDeps implements Callback<GetDepsOk>
 {
     final Node node;
     final TxnId txnId;
-    final Route route;
+    final FullRoute<?> route;
     final Txn txn;
 
     final Timestamp executeAt;
@@ -34,7 +30,7 @@ class CollectDeps implements Callback<GetDepsOk>
     private final BiConsumer<Deps, Throwable> callback;
     private boolean isDone;
 
-    CollectDeps(Node node, Topologies topologies, TxnId txnId, Route route, Txn txn, Timestamp executeAt, BiConsumer<Deps, Throwable> callback)
+    CollectDeps(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, BiConsumer<Deps, Throwable> callback)
     {
         this.node = node;
         this.txnId = txnId;
@@ -46,7 +42,7 @@ class CollectDeps implements Callback<GetDepsOk>
         this.tracker = new QuorumTracker(topologies);
     }
 
-    public static void withDeps(Node node, TxnId txnId, Route route, Txn txn, Timestamp executeAt, BiConsumer<Deps, Throwable> callback)
+    public static void withDeps(Node node, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, BiConsumer<Deps, Throwable> callback)
     {
         Topologies topologies = node.topology().withUnsyncedEpochs(route, txnId, executeAt);
         CollectDeps collect = new CollectDeps(node, topologies, txnId, route, txn, executeAt, callback);
@@ -70,7 +66,7 @@ class CollectDeps implements Callback<GetDepsOk>
         if (tracker.recordFailure(from) == Failed)
         {
             isDone = true;
-            callback.accept(null, new Timeout(txnId, route.homeKey));
+            callback.accept(null, new Timeout(txnId, route.homeKey()));
         }
     }
 
diff --git a/accord-core/src/main/java/accord/coordinate/Coordinate.java b/accord-core/src/main/java/accord/coordinate/Coordinate.java
index 9c0e1a7..836bfdb 100644
--- a/accord-core/src/main/java/accord/coordinate/Coordinate.java
+++ b/accord-core/src/main/java/accord/coordinate/Coordinate.java
@@ -24,18 +24,13 @@ import java.util.function.BiConsumer;
 import accord.api.Result;
 import accord.coordinate.tracking.FastPathTracker;
 import accord.coordinate.tracking.RequestStatus;
-import accord.primitives.Route;
+import accord.primitives.*;
 import accord.topology.Topologies;
-import accord.primitives.Ballot;
 import accord.messages.Callback;
 import accord.local.Node;
-import accord.primitives.Deps;
 import accord.local.Node.Id;
-import accord.primitives.Timestamp;
 import accord.messages.PreAccept;
 import accord.messages.PreAccept.PreAcceptOk;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
 import accord.messages.PreAccept.PreAcceptReply;
 import com.google.common.base.Preconditions;
 
@@ -56,13 +51,13 @@ public class Coordinate extends AsyncFuture<Result> implements Callback<PreAccep
     final Node node;
     final TxnId txnId;
     final Txn txn;
-    final Route route;
+    final FullRoute<?> route;
 
     private FastPathTracker tracker;
     private boolean preAcceptIsDone;
     private final List<PreAcceptOk> successes;
 
-    private Coordinate(Node node, TxnId txnId, Txn txn, Route route)
+    private Coordinate(Node node, TxnId txnId, Txn txn, FullRoute<?> route)
     {
         this.node = node;
         this.txnId = txnId;
@@ -80,7 +75,7 @@ public class Coordinate extends AsyncFuture<Result> implements Callback<PreAccep
         node.send(tracker.nodes(), to -> new PreAccept(to, tracker.topologies(), txnId, txn, route), this);
     }
 
-    public static Future<Result> coordinate(Node node, TxnId txnId, Txn txn, Route route)
+    public static Future<Result> coordinate(Node node, TxnId txnId, Txn txn, FullRoute<?> route)
     {
         Coordinate coordinate = new Coordinate(node, txnId, txn, route);
         coordinate.start();
@@ -100,7 +95,7 @@ public class Coordinate extends AsyncFuture<Result> implements Callback<PreAccep
                 break;
             case Failed:
                 preAcceptIsDone = true;
-                tryFailure(new Timeout(txnId, route.homeKey));
+                tryFailure(new Timeout(txnId, route.homeKey()));
                 break;
             case Success:
                 onPreAccepted();
@@ -121,7 +116,7 @@ public class Coordinate extends AsyncFuture<Result> implements Callback<PreAccep
         if (!reply.isOk())
         {
             // we've been preempted by a recovery coordinator; defer to it, and wait to hear any result
-            tryFailure(new Preempted(txnId, route.homeKey));
+            tryFailure(new Preempted(txnId, route.homeKey()));
             return;
         }
 
@@ -161,7 +156,7 @@ public class Coordinate extends AsyncFuture<Result> implements Callback<PreAccep
             // TODO: if we receive a MAX response, perhaps defer to permit at least one other node to respond before invalidating
             if (node.agent().isExpired(txnId, executeAt.real))
             {
-                proposeInvalidate(node, Ballot.ZERO, txnId, route.homeKey, (success, fail) -> {
+                proposeInvalidate(node, Ballot.ZERO, txnId, route.homeKey(), (success, fail) -> {
                     if (fail != null)
                     {
                         accept(null, fail);
@@ -171,7 +166,7 @@ public class Coordinate extends AsyncFuture<Result> implements Callback<PreAccep
                         node.withEpoch(executeAt.epoch, () -> {
                             commitInvalidate(node, txnId, route, executeAt);
                             // TODO: this should be Invalidated rather than Timeout?
-                            accept(null, new Timeout(txnId, route.homeKey));
+                            accept(null, new Timeout(txnId, route.homeKey()));
                         });
                     }
                 });
@@ -181,7 +176,7 @@ public class Coordinate extends AsyncFuture<Result> implements Callback<PreAccep
                 node.withEpoch(executeAt.epoch, () -> {
                     Topologies topologies = tracker.topologies();
                     if (executeAt.epoch > txnId.epoch)
-                        topologies = node.topology().withUnsyncedEpochs(txn, txnId.epoch, executeAt.epoch);
+                        topologies = node.topology().withUnsyncedEpochs(route, txnId.epoch, executeAt.epoch);
                     Propose.propose(node, topologies, Ballot.ZERO, txnId, txn, route, executeAt, deps, this);
                 });
             }
@@ -192,7 +187,7 @@ public class Coordinate extends AsyncFuture<Result> implements Callback<PreAccep
     public void accept(Result success, Throwable failure)
     {
         if (failure instanceof CoordinateFailed)
-            ((CoordinateFailed) failure).set(txnId, route.homeKey);
+            ((CoordinateFailed) failure).set(txnId, route.homeKey());
 
         if (success != null) trySuccess(success);
         else tryFailure(failure);
diff --git a/accord-core/src/main/java/accord/coordinate/Execute.java b/accord-core/src/main/java/accord/coordinate/Execute.java
index 3bac876..6195f73 100644
--- a/accord-core/src/main/java/accord/coordinate/Execute.java
+++ b/accord-core/src/main/java/accord/coordinate/Execute.java
@@ -40,17 +40,17 @@ import static accord.messages.Commit.Kind.Maximal;
 class Execute extends ReadCoordinator<ReadReply>
 {
     final Txn txn;
-    final Keys readScope;
-    final Route route;
+    final Seekables<?, ?> readScope;
+    final FullRoute<?> route;
     final Timestamp executeAt;
     final Deps deps;
     final Topologies applyTo;
     final BiConsumer<Result, Throwable> callback;
     private Data data;
 
-    private Execute(Node node, TxnId txnId, Txn txn, Route route, Keys readScope, Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
+    private Execute(Node node, TxnId txnId, Txn txn, FullRoute<?> route, Seekables<?, ?> readScope, Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
     {
-        super(node, node.topology().forEpoch(readScope, executeAt.epoch), txnId);
+        super(node, node.topology().forEpoch(readScope.toUnseekables(), executeAt.epoch), txnId);
         this.txn = txn;
         this.route = route;
         this.readScope = readScope;
@@ -60,12 +60,12 @@ class Execute extends ReadCoordinator<ReadReply>
         this.callback = callback;
     }
 
-    public static void execute(Node node, TxnId txnId, Txn txn, Route route, Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
+    public static void execute(Node node, TxnId txnId, Txn txn, FullRoute<?> route, Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
     {
         if (txn.read().keys().isEmpty())
         {
-            Topologies sendTo = node.topology().forEpoch(route, txnId.epoch);
-            Topologies applyTo = node.topology().preciseEpochs(route, txnId.epoch, executeAt.epoch);
+            Topologies sendTo = node.topology().preciseEpochs(route, txnId.epoch, executeAt.epoch);
+            Topologies applyTo = node.topology().forEpoch(route, executeAt.epoch);
             Result result = txn.result(txnId, null);
             Persist.persist(node, sendTo, applyTo, txnId, route, txn, executeAt, deps, txn.execute(executeAt, null), result);
             callback.accept(result, null);
@@ -105,7 +105,7 @@ class Execute extends ReadCoordinator<ReadReply>
         {
             default: throw new IllegalStateException();
             case Redundant:
-                callback.accept(null, new Preempted(txnId, route.homeKey));
+                callback.accept(null, new Preempted(txnId, route.homeKey()));
                 return Action.Abort;
             case NotCommitted:
                 // the replica may be missing the original commit, or the additional commit, so send everything
diff --git a/accord-core/src/main/java/accord/coordinate/FetchData.java b/accord-core/src/main/java/accord/coordinate/FetchData.java
index 2faa2da..f1e876c 100644
--- a/accord-core/src/main/java/accord/coordinate/FetchData.java
+++ b/accord-core/src/main/java/accord/coordinate/FetchData.java
@@ -4,10 +4,9 @@ import java.util.function.BiConsumer;
 
 import accord.api.RoutingKey;
 import accord.local.Node;
-import accord.local.Status.Phase;
 import accord.local.Status.Known;
 import accord.primitives.*;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import javax.annotation.Nullable;
 
@@ -23,24 +22,24 @@ import static accord.local.Status.Outcome.OutcomeUnknown;
  */
 public class FetchData
 {
-    public static Object fetch(Known fetch, Node node, TxnId txnId, RoutingKeys someKeys, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
+    public static Object fetch(Known fetch, Node node, TxnId txnId, Unseekables<?, ?> someUnseekables, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
     {
-        return fetch(fetch, node, txnId, someKeys, null, untilLocalEpoch, callback);
+        return fetch(fetch, node, txnId, someUnseekables, null, untilLocalEpoch, callback);
     }
 
-    public static Object fetch(Known fetch, Node node, TxnId txnId, RoutingKeys someKeys, @Nullable Timestamp executeAt, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
+    public static Object fetch(Known fetch, Node node, TxnId txnId, Unseekables<?, ?> someUnseekables, @Nullable Timestamp executeAt, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
     {
-        if (someKeys instanceof AbstractRoute) return fetch(fetch, node, txnId, (AbstractRoute) someKeys, executeAt, untilLocalEpoch, callback);
-        else return fetchWithSomeKeys(fetch, node, txnId, someKeys, untilLocalEpoch, callback);
+        if (someUnseekables.kind().isRoute()) return fetch(fetch, node, txnId, Route.tryCastToRoute(someUnseekables), executeAt, untilLocalEpoch, callback);
+        else return fetchWithSomeRoutables(fetch, node, txnId, someUnseekables, untilLocalEpoch, callback);
     }
 
-    public static Object fetch(Known fetch, Node node, TxnId txnId, AbstractRoute route, @Nullable Timestamp executeAt, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
+    public static Object fetch(Known fetch, Node node, TxnId txnId, Route<?> route, @Nullable Timestamp executeAt, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
     {
-        Preconditions.checkArgument(node.topology().hasEpoch(untilLocalEpoch));
-        KeyRanges ranges = node.topology().localRangesForEpochs(txnId.epoch, untilLocalEpoch);
+        Invariants.checkArgument(node.topology().hasEpoch(untilLocalEpoch));
+        Ranges ranges = node.topology().localRangesForEpochs(txnId.epoch, untilLocalEpoch);
         if (!route.covers(ranges))
         {
-            return fetchWithHomeKey(fetch, node, txnId, route.homeKey, untilLocalEpoch, callback);
+            return fetchWithHomeKey(fetch, node, txnId, route.homeKey(), untilLocalEpoch, callback);
         }
         else
         {
@@ -48,10 +47,10 @@ public class FetchData
         }
     }
 
-    private static Object fetchWithSomeKeys(Known fetch, Node node, TxnId txnId, RoutingKeys someKeys, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
+    private static Object fetchWithSomeRoutables(Known fetch, Node node, TxnId txnId, Unseekables<?, ?> someUnseekables, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
     {
-        Preconditions.checkArgument(node.topology().hasEpoch(untilLocalEpoch));
-        return FindHomeKey.findHomeKey(node, txnId, someKeys, (foundHomeKey, fail) -> {
+        Invariants.checkArgument(node.topology().hasEpoch(untilLocalEpoch));
+        return FindHomeKey.findHomeKey(node, txnId, someUnseekables, (foundHomeKey, fail) -> {
             if (fail != null) callback.accept(null, fail);
             else if (foundHomeKey == null) callback.accept(Nothing, null);
             else fetchWithHomeKey(fetch, node, txnId, foundHomeKey, untilLocalEpoch, callback);
@@ -60,7 +59,7 @@ public class FetchData
 
     private static Object fetchWithHomeKey(Known fetch, Node node, TxnId txnId, RoutingKey homeKey, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
     {
-        Preconditions.checkArgument(node.topology().hasEpoch(untilLocalEpoch));
+        Invariants.checkArgument(node.topology().hasEpoch(untilLocalEpoch));
         return FindRoute.findRoute(node, txnId, homeKey, (foundRoute, fail) -> {
             if (fail != null) callback.accept(null, fail);
             else if (foundRoute == null) callback.accept(Nothing, null);
@@ -68,19 +67,19 @@ public class FetchData
         });
     }
 
-    public static Object fetch(Known fetch, Node node, TxnId txnId, Route route, @Nullable Timestamp executeAt, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
+    public static Object fetch(Known fetch, Node node, TxnId txnId, FullRoute<?> route, @Nullable Timestamp executeAt, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
     {
-        KeyRanges ranges = node.topology().localRangesForEpochs(txnId.epoch, untilLocalEpoch);
+        Ranges ranges = node.topology().localRangesForEpochs(txnId.epoch, untilLocalEpoch);
         return fetchInternal(ranges, fetch, node, txnId, route.sliceStrict(ranges), executeAt, untilLocalEpoch, callback);
     }
 
-    private static Object fetchInternal(KeyRanges ranges, Known target, Node node, TxnId txnId, PartialRoute route, @Nullable Timestamp executeAt, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
+    private static Object fetchInternal(Ranges ranges, Known target, Node node, TxnId txnId, PartialRoute<?> route, @Nullable Timestamp executeAt, long untilLocalEpoch, BiConsumer<Known, Throwable> callback)
     {
         long srcEpoch = executeAt == null || target.epoch() == Coordination ? txnId.epoch : executeAt.epoch;
         if (!node.topology().hasEpoch(srcEpoch))
             return node.topology().awaitEpoch(srcEpoch).map(ignore -> fetchInternal(ranges, target, node, txnId, route, executeAt, untilLocalEpoch, callback));
 
-        PartialRoute fetch = route.sliceStrict(ranges);
+        PartialRoute<?> fetch = route.sliceStrict(ranges);
         return CheckOn.checkOn(target, node, txnId, fetch, srcEpoch, untilLocalEpoch, (ok, fail) -> {
             if (fail != null) callback.accept(null, fail);
             else if (ok == null) callback.accept(Nothing, null);
diff --git a/accord-core/src/main/java/accord/coordinate/FindHomeKey.java b/accord-core/src/main/java/accord/coordinate/FindHomeKey.java
index 0972c8e..1f938c3 100644
--- a/accord-core/src/main/java/accord/coordinate/FindHomeKey.java
+++ b/accord-core/src/main/java/accord/coordinate/FindHomeKey.java
@@ -4,10 +4,9 @@ import java.util.function.BiConsumer;
 
 import accord.api.RoutingKey;
 import accord.local.Node;
-import accord.local.Node.Id;
 import accord.messages.CheckStatus.CheckStatusOk;
 import accord.messages.CheckStatus.IncludeInfo;
-import accord.primitives.RoutingKeys;
+import accord.primitives.Unseekables;
 import accord.primitives.TxnId;
 
 /**
@@ -16,15 +15,15 @@ import accord.primitives.TxnId;
 public class FindHomeKey extends CheckShards
 {
     final BiConsumer<RoutingKey, Throwable> callback;
-    FindHomeKey(Node node, TxnId txnId, RoutingKeys someKeys, BiConsumer<RoutingKey, Throwable> callback)
+    FindHomeKey(Node node, TxnId txnId, Unseekables<?, ?> unseekables, BiConsumer<RoutingKey, Throwable> callback)
     {
-        super(node, txnId, someKeys, txnId.epoch, IncludeInfo.No);
+        super(node, txnId, unseekables, txnId.epoch, IncludeInfo.No);
         this.callback = callback;
     }
 
-    public static FindHomeKey findHomeKey(Node node, TxnId txnId, RoutingKeys someKeys, BiConsumer<RoutingKey, Throwable> callback)
+    public static FindHomeKey findHomeKey(Node node, TxnId txnId, Unseekables<?, ?> unseekables, BiConsumer<RoutingKey, Throwable> callback)
     {
-        FindHomeKey findHomeKey = new FindHomeKey(node, txnId, someKeys, callback);
+        FindHomeKey findHomeKey = new FindHomeKey(node, txnId, unseekables, callback);
         findHomeKey.start();
         return findHomeKey;
     }
diff --git a/accord-core/src/main/java/accord/coordinate/FindRoute.java b/accord-core/src/main/java/accord/coordinate/FindRoute.java
index fa2256e..82b116e 100644
--- a/accord-core/src/main/java/accord/coordinate/FindRoute.java
+++ b/accord-core/src/main/java/accord/coordinate/FindRoute.java
@@ -4,14 +4,12 @@ import java.util.function.BiConsumer;
 
 import accord.api.RoutingKey;
 import accord.local.Node;
-import accord.local.Node.Id;
 import accord.local.Status;
 import accord.messages.CheckStatus.CheckStatusOk;
 import accord.messages.CheckStatus.IncludeInfo;
-import accord.primitives.Route;
-import accord.primitives.RoutingKeys;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
+import accord.primitives.*;
+
+import static accord.primitives.Route.isFullRoute;
 
 /**
  * Find the Route of a known (txnId, homeKey) pair
@@ -20,10 +18,10 @@ public class FindRoute extends CheckShards
 {
     public static class Result
     {
-        public final Route route;
+        public final FullRoute<?> route;
         public final Timestamp executeAt;
 
-        public Result(Route route, Timestamp executeAt)
+        public Result(FullRoute<?> route, Timestamp executeAt)
         {
             this.route = route;
             this.executeAt = executeAt;
@@ -31,7 +29,7 @@ public class FindRoute extends CheckShards
 
         public Result(CheckStatusOk ok)
         {
-            this.route = (Route)ok.route;
+            this.route = Route.castToFullRoute(ok.route);
             this.executeAt = ok.saveStatus.status.compareTo(Status.PreCommitted) >= 0 ? ok.executeAt : null;
         }
     }
@@ -53,7 +51,7 @@ public class FindRoute extends CheckShards
     @Override
     protected boolean isSufficient(CheckStatusOk ok)
     {
-        return ok.route instanceof Route;
+        return isFullRoute(ok.route);
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/coordinate/Invalidate.java b/accord-core/src/main/java/accord/coordinate/Invalidate.java
index 6a7d333..f923fe2 100644
--- a/accord-core/src/main/java/accord/coordinate/Invalidate.java
+++ b/accord-core/src/main/java/accord/coordinate/Invalidate.java
@@ -32,10 +32,10 @@ import accord.topology.Topologies;
 
 import accord.api.RoutingKey;
 import accord.local.Node;
-import accord.local.Node.Id;
 import accord.messages.BeginInvalidation;
 import accord.messages.BeginInvalidation.InvalidateReply;
 import accord.messages.Callback;
+import accord.utils.Invariants;
 import com.google.common.base.Preconditions;
 
 import javax.annotation.Nullable;
@@ -43,7 +43,6 @@ import javax.annotation.Nullable;
 import static accord.coordinate.Propose.Invalidate.proposeInvalidate;
 import static accord.local.PreLoadContext.contextFor;
 import static accord.local.Status.Accepted;
-import static accord.local.Status.AcceptedInvalidate;
 import static accord.primitives.ProgressToken.INVALIDATED;
 
 public class Invalidate implements Callback<InvalidateReply>
@@ -51,7 +50,7 @@ public class Invalidate implements Callback<InvalidateReply>
     private final Node node;
     private final Ballot ballot;
     private final TxnId txnId;
-    private final RoutingKeys invalidateWithKeys;
+    private final Unseekables<?, ?> invalidateWith;
     private final BiConsumer<Outcome, Throwable> callback;
 
     private boolean isDone;
@@ -61,38 +60,38 @@ public class Invalidate implements Callback<InvalidateReply>
     private final InvalidationTracker tracker;
     private Throwable failure;
 
-    private Invalidate(Node node, Ballot ballot, TxnId txnId, RoutingKeys invalidateWithKeys, boolean transitivelyInvokedByPriorInvalidation, BiConsumer<Outcome, Throwable> callback)
+    private Invalidate(Node node, Ballot ballot, TxnId txnId, Unseekables<?, ?> invalidateWith, boolean transitivelyInvokedByPriorInvalidation, BiConsumer<Outcome, Throwable> callback)
     {
         this.callback = callback;
         this.node = node;
         this.ballot = ballot;
         this.txnId = txnId;
-        this.invalidateWithKeys = invalidateWithKeys;
         this.transitivelyInvokedByPriorInvalidation = transitivelyInvokedByPriorInvalidation;
-        Topologies topologies = node.topology().forEpoch(invalidateWithKeys, txnId.epoch);
+        this.invalidateWith = invalidateWith;
+        Topologies topologies = node.topology().forEpoch(invalidateWith, txnId.epoch);
         this.tracker = new InvalidationTracker(topologies);
     }
 
-    public static Invalidate invalidate(Node node, TxnId txnId, RoutingKeys invalidateWithKeys, BiConsumer<Outcome, Throwable> callback)
+    public static Invalidate invalidate(Node node, TxnId txnId, Unseekables<?, ?> invalidateWith, BiConsumer<Outcome, Throwable> callback)
     {
-        return invalidate(node, txnId, invalidateWithKeys, false, callback);
+        return invalidate(node, txnId, invalidateWith, false, callback);
     }
 
-    public static Invalidate invalidate(Node node, TxnId txnId, RoutingKeys invalidateWithKeys, boolean transitivelyInvokedByPriorInvalidation, BiConsumer<Outcome, Throwable> callback)
+    public static Invalidate invalidate(Node node, TxnId txnId, Unseekables<?, ?> invalidateWith, boolean transitivelyInvokedByPriorInvalidation, BiConsumer<Outcome, Throwable> callback)
     {
         Ballot ballot = new Ballot(node.uniqueNow());
-        Invalidate invalidate = new Invalidate(node, ballot, txnId, invalidateWithKeys, transitivelyInvokedByPriorInvalidation, callback);
+        Invalidate invalidate = new Invalidate(node, ballot, txnId, invalidateWith, transitivelyInvokedByPriorInvalidation, callback);
         invalidate.start();
         return invalidate;
     }
 
     private void start()
     {
-        node.send(tracker.nodes(), to -> new BeginInvalidation(to, tracker.topologies(), txnId, invalidateWithKeys, ballot), this);
+        node.send(tracker.nodes(), to -> new BeginInvalidation(to, tracker.topologies(), txnId, invalidateWith, ballot), this);
     }
 
     @Override
-    public synchronized void onSuccess(Id from, InvalidateReply reply)
+    public synchronized void onSuccess(Node.Id from, InvalidateReply reply)
     {
         if (isDone || isPrepareDone)
             return;
@@ -102,7 +101,7 @@ public class Invalidate implements Callback<InvalidateReply>
     }
 
     @Override
-    public void onFailure(Id from, Throwable failure)
+    public void onFailure(Node.Id from, Throwable failure)
     {
         if (isDone || isPrepareDone)
             return;
@@ -140,13 +139,13 @@ public class Invalidate implements Callback<InvalidateReply>
 
     private void invalidate()
     {
-        Preconditions.checkState(!isPrepareDone);
+        Invariants.checkState(!isPrepareDone);
         isPrepareDone = true;
 
         // first look to see if it has already been
         {
-            Route route = InvalidateReply.findRoute(replies);
-            RoutingKey homeKey = route != null ? route.homeKey : InvalidateReply.findHomeKey(replies);
+            FullRoute<?> route = InvalidateReply.findRoute(replies);
+            RoutingKey homeKey = route != null ? route.homeKey() : InvalidateReply.findHomeKey(replies);
             InvalidateReply maxReply = InvalidateReply.max(replies);
 
             switch (maxReply.status)
@@ -186,7 +185,7 @@ public class Invalidate implements Callback<InvalidateReply>
                         if (!witnessedByInvalidation.hasBeen(Accepted))
                         {
                             Preconditions.checkState(tracker.all(InvalidationShardTracker::isPromised));
-                            if (!invalidateWithKeys.containsAll(route))
+                            if (!invalidateWith.containsAll(route))
                                 witnessedByInvalidation = null;
                         }
                         RecoverWithRoute.recover(node, ballot, txnId, route, witnessedByInvalidation, callback);
@@ -196,7 +195,7 @@ public class Invalidate implements Callback<InvalidateReply>
                         Preconditions.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 (invalidateWithKeys.contains(homeKey) && tracker.isPromisedForKey(homeKey, txnId.epoch))
+                        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
@@ -206,7 +205,7 @@ public class Invalidate implements Callback<InvalidateReply>
                         if (!witnessedByInvalidation.hasBeen(Accepted))
                         {
                             Preconditions.checkState(tracker.all(InvalidationShardTracker::isPromised));
-                            if (!invalidateWithKeys.contains(homeKey))
+                            if (!invalidateWith.contains(homeKey))
                                 witnessedByInvalidation = null;
                         }
 
@@ -228,7 +227,8 @@ public class Invalidate implements Callback<InvalidateReply>
 
         // if we have witnessed the transaction, but are able to invalidate, do we want to proceed?
         // Probably simplest to do so, but perhaps better for user if we don't.
-        RoutingKey invalidateWithKey = invalidateWithKeys.slice(KeyRanges.of(tracker.promisedShard().range)).get(0);
+        // TODO (RangeTxns): This should be a Routable, or we should guarantee it is safe to operate on any key in the range
+        RoutingKey invalidateWithKey = invalidateWith.slice(Ranges.of(tracker.promisedShard().range)).get(0).someIntersectingRoutingKey();
         proposeInvalidate(node, ballot, txnId, invalidateWithKey, (success, fail) -> {
             /**
              * We're now inside our *exactly once* callback we registered with proposeInvalidate, and we need to
@@ -256,12 +256,12 @@ public class Invalidate implements Callback<InvalidateReply>
 
     private void commitInvalidate()
     {
-        @Nullable AbstractRoute route = InvalidateReply.mergeRoutes(replies);
+        @Nullable Route<?> route = InvalidateReply.mergeRoutes(replies);
         // TODO: commitInvalidate (and others) should skip the network for local applications,
         //  so we do not need to explicitly do so here before notifying the waiter
-        Commit.Invalidate.commitInvalidate(node, txnId, route != null ? route.union(invalidateWithKeys) : invalidateWithKeys, txnId);
+        Commit.Invalidate.commitInvalidate(node, txnId, route != null ? Unseekables.merge(route, (Unseekables)invalidateWith) : invalidateWith, txnId);
         // TODO: 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), invalidateWithKeys, txnId, safeStore -> {
+        node.forEachLocalSince(contextFor(txnId), invalidateWith, txnId, safeStore -> {
             safeStore.command(txnId).commitInvalidate(safeStore);
         }).addCallback((s, f) -> {
             callback.accept(INVALIDATED, null);
@@ -271,7 +271,7 @@ public class Invalidate implements Callback<InvalidateReply>
     }
 
     @Override
-    public void onCallbackFailure(Id from, Throwable failure)
+    public void onCallbackFailure(Node.Id from, Throwable failure)
     {
         if (isDone)
             return;
diff --git a/accord-core/src/main/java/accord/coordinate/MaybeRecover.java b/accord-core/src/main/java/accord/coordinate/MaybeRecover.java
index 39e3236..7840666 100644
--- a/accord-core/src/main/java/accord/coordinate/MaybeRecover.java
+++ b/accord-core/src/main/java/accord/coordinate/MaybeRecover.java
@@ -24,7 +24,7 @@ import javax.annotation.Nullable;
 
 import accord.local.Status.Known;
 import accord.primitives.*;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import accord.api.RoutingKey;
 import accord.local.Node;
@@ -39,12 +39,12 @@ import static accord.utils.Functions.reduceNonNull;
  */
 public class MaybeRecover extends CheckShards
 {
-    @Nullable final AbstractRoute route;
+    @Nullable final Route<?> route;
     final RoutingKey homeKey;
     final ProgressToken prevProgress;
     final BiConsumer<Outcome, Throwable> callback;
 
-    MaybeRecover(Node node, TxnId txnId, RoutingKey homeKey, @Nullable AbstractRoute route, ProgressToken prevProgress, BiConsumer<Outcome, Throwable> callback)
+    MaybeRecover(Node node, TxnId txnId, RoutingKey homeKey, @Nullable Route<?> route, ProgressToken prevProgress, BiConsumer<Outcome, Throwable> callback)
     {
         // we only want to enquire with the home shard, but we prefer maximal route information for running Invalidation against, if necessary
         super(node, txnId, RoutingKeys.of(homeKey), txnId.epoch, IncludeInfo.Route);
@@ -54,7 +54,7 @@ public class MaybeRecover extends CheckShards
         this.callback = callback;
     }
 
-    public static void maybeRecover(Node node, TxnId txnId, RoutingKey homeKey, @Nullable AbstractRoute route,
+    public static void maybeRecover(Node node, TxnId txnId, RoutingKey homeKey, @Nullable Route<?> route,
                                     ProgressToken prevProgress, BiConsumer<Outcome, Throwable> callback)
     {
         MaybeRecover maybeRecover = new MaybeRecover(node, txnId, homeKey, route, prevProgress, callback);
@@ -82,17 +82,17 @@ public class MaybeRecover extends CheckShards
         }
         else
         {
-            Preconditions.checkState(merged != null);
+            Invariants.checkState(merged != null);
             Known known = merged.saveStatus.known;
 
             switch (known.outcome)
             {
                 default: throw new AssertionError();
                 case OutcomeUnknown:
-                    if (!known.isDefinitionKnown() && !(merged.route instanceof Route))
+                    if (!known.isDefinitionKnown() && !Route.isFullRoute(merged.route))
                     {
                         // order important, as route could be a Route which does not implement RoutingKeys.union
-                        RoutingKeys someKeys = reduceNonNull(RoutingKeys::union, this.contactKeys, merged.route, route);
+                        Unseekables<?, ?> someKeys = reduceNonNull(Unseekables::merge, (Unseekables)this.contact, merged.route, route);
                         // for correctness reasons, we have not necessarily preempted the initial pre-accept round and
                         // may have raced with it, so we must attempt to recover anything we see pre-accepted.
                         Invalidate.invalidate(node, txnId, someKeys.with(homeKey), callback);
@@ -100,14 +100,14 @@ public class MaybeRecover extends CheckShards
                     }
                 case OutcomeKnown:
                 case OutcomeApplied:
-                    Preconditions.checkState(merged.route instanceof Route);
+                    Invariants.checkState(Route.isFullRoute(merged.route));
                     if (hasMadeProgress(merged)) callback.accept(merged.toProgressToken(), null);
-                    else node.recover(txnId, (Route) merged.route).addCallback(callback);
+                    else node.recover(txnId, Route.castToFullRoute(merged.route)).addCallback(callback);
                     break;
 
                 case InvalidationApplied:
                     // TODO: we should simply invoke commitInvalidate
-                    RoutingKeys someKeys = reduceNonNull(RoutingKeys::union, this.contactKeys, merged.route, route);
+                    Unseekables<?, ?> someKeys = reduceNonNull(Unseekables::merge, (Unseekables)contact, merged.route, route);
                     Invalidate.invalidate(node, txnId, someKeys.with(homeKey), callback);
             }
         }
diff --git a/accord-core/src/main/java/accord/coordinate/Persist.java b/accord-core/src/main/java/accord/coordinate/Persist.java
index c4c427e..7d9c997 100644
--- a/accord-core/src/main/java/accord/coordinate/Persist.java
+++ b/accord-core/src/main/java/accord/coordinate/Persist.java
@@ -32,13 +32,10 @@ import accord.messages.Commit;
 import accord.messages.Commit.Kind;
 import accord.messages.InformHomeDurable;
 import accord.primitives.Deps;
-import accord.primitives.Route;
 import accord.primitives.Txn;
+import accord.primitives.*;
 import accord.topology.Shard;
 import accord.topology.Topologies;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
-import accord.primitives.Writes;
 
 import static accord.coordinate.tracking.RequestStatus.Success;
 import static accord.local.Status.Durability.Durable;
@@ -48,7 +45,7 @@ public class Persist implements Callback<ApplyReply>
 {
     final Node node;
     final TxnId txnId;
-    final Route route;
+    final FullRoute<?> route;
     final Txn txn;
     final Timestamp executeAt;
     final Deps deps;
@@ -56,13 +53,13 @@ public class Persist implements Callback<ApplyReply>
     final Set<Id> persistedOn;
     boolean isDone;
 
-    public static void persist(Node node, Topologies sendTo, Topologies applyTo, TxnId txnId, Route route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+    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);
     }
 
-    public static void persistAndCommit(Node node, TxnId txnId, Route route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+    public static void persistAndCommit(Node node, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
     {
         Topologies sendTo = node.topology().preciseEpochs(route, txnId.epoch, executeAt.epoch);
         Topologies applyTo = node.topology().forEpoch(route, executeAt.epoch);
@@ -70,7 +67,7 @@ public class Persist implements Callback<ApplyReply>
         node.send(sendTo.nodes(), to -> new Apply(to, sendTo, applyTo, executeAt.epoch, txnId, route, executeAt, deps, writes, result), persist);
     }
 
-    private Persist(Node node, Topologies topologies, TxnId txnId, Route route, Txn txn, Timestamp executeAt, Deps deps)
+    private Persist(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps)
     {
         this.node = node;
         this.txnId = txnId;
@@ -96,14 +93,14 @@ public class Persist implements Callback<ApplyReply>
                     if (!isDone)
                     {
                         // TODO: send to non-home replicas also, so they may clear their log more easily?
-                        Shard homeShard = node.topology().forEpochIfKnown(route.homeKey, txnId.epoch);
-                        node.send(homeShard, new InformHomeDurable(txnId, route.homeKey, executeAt, Durable, persistedOn));
+                        Shard homeShard = node.topology().forEpochIfKnown(route.homeKey(), txnId.epoch);
+                        node.send(homeShard, new InformHomeDurable(txnId, route.homeKey(), executeAt, Durable, persistedOn));
                         isDone = true;
                     }
                     else if (!tracker.hasInFlight() && !tracker.hasFailures())
                     {
-                        Shard homeShard = node.topology().forEpochIfKnown(route.homeKey, txnId.epoch);
-                        node.send(homeShard, new InformHomeDurable(txnId, route.homeKey, executeAt, Universal, persistedOn));
+                        Shard homeShard = node.topology().forEpochIfKnown(route.homeKey(), txnId.epoch);
+                        node.send(homeShard, new InformHomeDurable(txnId, route.homeKey(), executeAt, Universal, persistedOn));
                     }
                 }
                 break;
diff --git a/accord-core/src/main/java/accord/coordinate/Propose.java b/accord-core/src/main/java/accord/coordinate/Propose.java
index 69af3d3..6209f9d 100644
--- a/accord-core/src/main/java/accord/coordinate/Propose.java
+++ b/accord-core/src/main/java/accord/coordinate/Propose.java
@@ -29,16 +29,11 @@ import accord.coordinate.tracking.QuorumTracker;
 import accord.coordinate.tracking.QuorumTracker.QuorumShardTracker;
 import accord.coordinate.tracking.RequestStatus;
 import accord.messages.Callback;
-import accord.primitives.Route;
+import accord.primitives.*;
 import accord.topology.Shard;
 import accord.topology.Topologies;
-import accord.primitives.Ballot;
 import accord.local.Node;
 import accord.local.Node.Id;
-import accord.primitives.Timestamp;
-import accord.primitives.Deps;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
 import accord.messages.Accept;
 import accord.messages.Accept.AcceptReply;
 
@@ -51,7 +46,7 @@ class Propose implements Callback<AcceptReply>
     final Ballot ballot;
     final TxnId txnId;
     final Txn txn;
-    final Route route;
+    final FullRoute<?> route;
     final Deps deps;
 
     private final List<AcceptReply> acceptOks;
@@ -60,7 +55,7 @@ class Propose implements Callback<AcceptReply>
     private final BiConsumer<Result, Throwable> callback;
     private boolean isDone;
 
-    Propose(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, Route route, Deps deps, Timestamp executeAt, BiConsumer<Result, Throwable> callback)
+    Propose(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, Timestamp executeAt, BiConsumer<Result, Throwable> callback)
     {
         this.node = node;
         this.ballot = ballot;
@@ -74,14 +69,14 @@ class Propose implements Callback<AcceptReply>
         this.acceptTracker = new QuorumTracker(topologies);
     }
 
-    public static void propose(Node node, Ballot ballot, TxnId txnId, Txn txn, Route route,
+    public static void propose(Node node, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route,
                                Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
     {
         Topologies topologies = node.topology().withUnsyncedEpochs(route, txnId, executeAt);
         propose(node, topologies, ballot, txnId, txn, route, executeAt, deps, callback);
     }
 
-    public static void propose(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, Route route,
+    public static void propose(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route,
                                Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
     {
         Propose propose = new Propose(node, topologies, ballot, txnId, txn, route, deps, executeAt, callback);
@@ -100,7 +95,7 @@ class Propose implements Callback<AcceptReply>
             case Redundant:
             case RejectedBallot:
                 isDone = true;
-                callback.accept(null, new Preempted(txnId, route.homeKey));
+                callback.accept(null, new Preempted(txnId, route.homeKey()));
                 break;
             case Success:
                 acceptOks.add(reply);
@@ -115,7 +110,7 @@ class Propose implements Callback<AcceptReply>
         if (acceptTracker.recordFailure(from) == Failed)
         {
             isDone = true;
-            callback.accept(null, new Timeout(txnId, route.homeKey));
+            callback.accept(null, new Timeout(txnId, route.homeKey()));
         }
     }
 
diff --git a/accord-core/src/main/java/accord/coordinate/Recover.java b/accord-core/src/main/java/accord/coordinate/Recover.java
index b2b037d..12eee58 100644
--- a/accord-core/src/main/java/accord/coordinate/Recover.java
+++ b/accord-core/src/main/java/accord/coordinate/Recover.java
@@ -28,7 +28,7 @@ import java.util.function.BiConsumer;
 import accord.coordinate.tracking.*;
 import accord.primitives.*;
 import accord.messages.Commit;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import accord.api.Result;
 import accord.topology.Topologies;
@@ -61,11 +61,11 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
         //       are given earlier timestamps we can retry without restarting.
         final QuorumTracker tracker;
 
-        AwaitCommit(Node node, TxnId txnId, RoutingKeys someKeys)
+        AwaitCommit(Node node, TxnId txnId, Unseekables<?, ?> unseekables)
         {
-            Topology topology = node.topology().globalForEpoch(txnId.epoch).forKeys(someKeys);
+            Topology topology = node.topology().globalForEpoch(txnId.epoch).forSelection(unseekables);
             this.tracker = new QuorumTracker(new Topologies.Single(node.topology().sorter(), topology));
-            node.send(topology.nodes(), to -> new WaitOnCommit(to, topology, txnId, someKeys), this);
+            node.send(topology.nodes(), to -> new WaitOnCommit(to, topology, txnId, unseekables), this);
         }
 
         @Override
@@ -83,7 +83,7 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
             if (isDone()) return;
 
             if (tracker.recordFailure(from) == Failed)
-                tryFailure(new Timeout(txnId, route.homeKey));
+                tryFailure(new Timeout(txnId, route.homeKey()));
         }
 
         @Override
@@ -100,7 +100,8 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
         for (int i = 0 ; i < waitOn.txnIdCount() ; ++i)
         {
             TxnId txnId = waitOn.txnId(i);
-            new AwaitCommit(node, txnId, waitOn.someRoutingKeys(txnId)).addCallback((success, failure) -> {
+            // TODO (now): this should perhaps use RouteFragment as we might need to handle txns that are range-only
+            new AwaitCommit(node, txnId, waitOn.someRoutables(txnId)).addCallback((success, failure) -> {
                 if (future.isDone())
                     return;
                 if (success != null && remaining.decrementAndGet() == 0)
@@ -116,7 +117,7 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
     private final Ballot ballot;
     private final TxnId txnId;
     private final Txn txn;
-    private final Route route;
+    private final FullRoute<?> route;
     private final BiConsumer<Outcome, Throwable> callback;
     private boolean isDone;
 
@@ -124,7 +125,7 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
     private final RecoveryTracker tracker;
     private boolean isBallotPromised;
 
-    private Recover(Node node, Ballot ballot, TxnId txnId, Txn txn, Route route, BiConsumer<Outcome, Throwable> callback, Topologies topologies)
+    private Recover(Node node, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, BiConsumer<Outcome, Throwable> callback, Topologies topologies)
     {
         this.node = node;
         this.ballot = ballot;
@@ -145,23 +146,23 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
         node.agent().onRecover(node, result, failure);
     }
 
-    public static Recover recover(Node node, TxnId txnId, Txn txn, Route route, BiConsumer<Outcome, Throwable> callback)
+    public static Recover recover(Node node, TxnId txnId, Txn txn, FullRoute<?> route, BiConsumer<Outcome, Throwable> callback)
     {
         return recover(node, txnId, txn, route, callback, node.topology().forEpoch(route, txnId.epoch));
     }
 
-    public static Recover recover(Node node, TxnId txnId, Txn txn, Route route, BiConsumer<Outcome, Throwable> callback, Topologies topologies)
+    public static Recover recover(Node node, TxnId txnId, Txn txn, FullRoute<?> route, BiConsumer<Outcome, Throwable> callback, Topologies topologies)
     {
         Ballot ballot = new Ballot(node.uniqueNow());
         return recover(node, ballot, txnId, txn, route, callback, topologies);
     }
 
-    public static Recover recover(Node node, Ballot ballot, TxnId txnId, Txn txn, Route route, BiConsumer<Outcome, Throwable> callback)
+    public static Recover recover(Node node, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, BiConsumer<Outcome, Throwable> callback)
     {
         return recover(node, ballot, txnId, txn, route, callback, node.topology().forEpoch(route, txnId.epoch));
     }
 
-    public static Recover recover(Node node, Ballot ballot, TxnId txnId, Txn txn, Route route, BiConsumer<Outcome, Throwable> callback, Topologies topologies)
+    public static Recover recover(Node node, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, BiConsumer<Outcome, Throwable> callback, Topologies topologies)
     {
         Recover recover = new Recover(node, ballot, txnId, txn, route, callback, topologies);
         recover.start(topologies.nodes());
@@ -181,7 +182,7 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
 
         if (!reply.isOk())
         {
-            accept(null, new Preempted(txnId, route.homeKey));
+            accept(null, new Preempted(txnId, route.homeKey()));
             return;
         }
 
@@ -194,7 +195,7 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
 
     private void recover()
     {
-        Preconditions.checkState(!isBallotPromised);
+        Invariants.checkState(!isBallotPromised);
         isBallotPromised = true;
 
         // first look for the most recent Accept; if present, go straight to proposing it again
@@ -285,7 +286,7 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
 
     private void invalidate()
     {
-        proposeInvalidate(node, ballot, txnId, route.homeKey, (success, fail) -> {
+        proposeInvalidate(node, ballot, txnId, route.homeKey(), (success, fail) -> {
             if (fail != null) accept(null, fail);
             else commitInvalidate();
         });
@@ -306,8 +307,8 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
 
     private Deps mergeDeps()
     {
-        KeyRanges ranges = recoverOks.stream().map(r -> r.deps.covering).reduce(KeyRanges::union).orElseThrow(NoSuchElementException::new);
-        Preconditions.checkState(ranges.containsAll(txn.keys()));
+        Ranges ranges = recoverOks.stream().map(r -> r.deps.covering).reduce(Ranges::union).orElseThrow(NoSuchElementException::new);
+        Invariants.checkState(ranges.containsAll(txn.keys()));
         return Deps.merge(recoverOks, r -> r.deps);
     }
 
@@ -323,7 +324,7 @@ public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throw
             return;
 
         if (tracker.recordFailure(from) == Failed)
-            accept(null, new Timeout(txnId, route.homeKey));
+            accept(null, new Timeout(txnId, route.homeKey()));
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/coordinate/RecoverWithHomeKey.java b/accord-core/src/main/java/accord/coordinate/RecoverWithHomeKey.java
index 60b7bd6..1ea6856 100644
--- a/accord-core/src/main/java/accord/coordinate/RecoverWithHomeKey.java
+++ b/accord-core/src/main/java/accord/coordinate/RecoverWithHomeKey.java
@@ -7,12 +7,12 @@ import accord.local.Node;
 import accord.local.Status;
 import accord.messages.CheckStatus.CheckStatusOk;
 import accord.messages.CheckStatus.IncludeInfo;
-import accord.primitives.Route;
 import accord.primitives.RoutingKeys;
 import accord.primitives.TxnId;
 import com.google.common.base.Preconditions;
 
-import javax.annotation.Nonnull;
+import static accord.primitives.Route.castToFullRoute;
+import static accord.primitives.Route.isFullRoute;
 
 /**
  * A result of null indicates the transaction is globally persistent
@@ -62,7 +62,7 @@ public class RecoverWithHomeKey extends CheckShards implements BiConsumer<Object
         {
             callback.accept(null, fail);
         }
-        else if (merged == null || !(merged.route instanceof Route))
+        else if (merged == null || !isFullRoute(merged.route))
         {
             switch (success)
             {
@@ -75,13 +75,13 @@ public class RecoverWithHomeKey extends CheckShards implements BiConsumer<Object
                 case Quorum:
                     if (witnessedByInvalidation != null && witnessedByInvalidation.compareTo(Status.PreAccepted) > 0)
                         throw new IllegalStateException("We previously invalidated, finding a status that should be recoverable");
-                    Invalidate.invalidate(node, txnId, contactKeys.with(homeKey), true, callback);
+                    Invalidate.invalidate(node, txnId, contact.with(homeKey), true, callback);
             }
         }
         else
         {
             // start recovery
-            RecoverWithRoute.recover(node, txnId, (Route)merged.route, witnessedByInvalidation, callback);
+            RecoverWithRoute.recover(node, txnId, castToFullRoute(merged.route), witnessedByInvalidation, callback);
         }
     }
 }
diff --git a/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java b/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java
index 00bfeb5..59c95a1 100644
--- a/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java
+++ b/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java
@@ -5,7 +5,7 @@ import java.util.function.BiConsumer;
 import accord.local.Status;
 import accord.local.Status.Known;
 import accord.primitives.*;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import accord.local.Node;
 import accord.local.Node.Id;
@@ -20,21 +20,22 @@ import javax.annotation.Nullable;
 import static accord.messages.Commit.Invalidate.commitInvalidate;
 import static accord.primitives.ProgressToken.APPLIED;
 import static accord.primitives.ProgressToken.INVALIDATED;
+import static accord.primitives.Route.castToFullRoute;
 
 public class RecoverWithRoute extends CheckShards
 {
     final @Nullable Ballot promisedBallot; // if non-null, has already been promised by some shard
-    final Route route;
+    final FullRoute<?> route;
     final BiConsumer<Outcome, Throwable> callback;
     final Status witnessedByInvalidation;
 
-    private RecoverWithRoute(Node node, Topologies topologies, @Nullable Ballot promisedBallot, TxnId txnId, Route route, Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
+    private RecoverWithRoute(Node node, Topologies topologies, @Nullable Ballot promisedBallot, TxnId txnId, FullRoute<?> route, Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
     {
         super(node, txnId, route, txnId.epoch, IncludeInfo.All);
         // 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.promisedBallot = promisedBallot;
         this.route = route;
         this.callback = callback;
@@ -42,28 +43,33 @@ public class RecoverWithRoute extends CheckShards
         assert topologies.oldestEpoch() == topologies.currentEpoch() && topologies.currentEpoch() == txnId.epoch;
     }
 
-    public static RecoverWithRoute recover(Node node, TxnId txnId, Route route, @Nullable Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
+    public static RecoverWithRoute recover(Node node, TxnId txnId, FullRoute<?> route, @Nullable Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
     {
         return recover(node, node.topology().forEpoch(route, txnId.epoch), txnId, route, witnessedByInvalidation, callback);
     }
 
-    public static RecoverWithRoute recover(Node node, Topologies topologies, TxnId txnId, Route route, @Nullable Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
+    public static RecoverWithRoute recover(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, @Nullable Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
     {
         return recover(node, topologies, null, txnId, route, witnessedByInvalidation, callback);
     }
 
-    public static RecoverWithRoute recover(Node node, @Nullable Ballot promisedBallot, TxnId txnId, Route route, @Nullable Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
+    public static RecoverWithRoute recover(Node node, @Nullable Ballot promisedBallot, TxnId txnId, FullRoute<?> route, @Nullable Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
     {
         return recover(node, node.topology().forEpoch(route, txnId.epoch), promisedBallot, txnId, route, witnessedByInvalidation, callback);
     }
 
-    public static RecoverWithRoute recover(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Route route, Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
+    public static RecoverWithRoute recover(Node node, Topologies topologies, Ballot ballot, TxnId txnId, FullRoute<?> route, Status witnessedByInvalidation, BiConsumer<Outcome, Throwable> callback)
     {
         RecoverWithRoute recover = new RecoverWithRoute(node, topologies, ballot, txnId, route, witnessedByInvalidation, callback);
         recover.start();
         return recover;
     }
 
+    private FullRoute<?> route()
+    {
+        return castToFullRoute(contact);
+    }
+
     @Override
     public void contact(Id to)
     {
@@ -73,8 +79,8 @@ public class RecoverWithRoute extends CheckShards
     @Override
     protected boolean isSufficient(Id from, CheckStatusOk ok)
     {
-        KeyRanges rangesForNode = topologies().forEpoch(txnId.epoch).rangesForNode(from);
-        PartialRoute route = this.route.slice(rangesForNode);
+        Ranges rangesForNode = topologies().forEpoch(txnId.epoch).rangesForNode(from);
+        PartialRoute<?> route = this.route.slice(rangesForNode);
         return isSufficient(route, ok);
     }
 
@@ -84,7 +90,7 @@ public class RecoverWithRoute extends CheckShards
         return isSufficient(route, merged);
     }
 
-    protected boolean isSufficient(AbstractRoute route, CheckStatusOk ok)
+    protected boolean isSufficient(Route<?> route, CheckStatusOk ok)
     {
         CheckStatusOkFull full = (CheckStatusOkFull)ok;
         Known sufficientTo = full.sufficientFor(route);
@@ -94,7 +100,7 @@ public class RecoverWithRoute extends CheckShards
         if (sufficientTo.outcome.isInvalidated())
             return true;
 
-        Preconditions.checkState(full.partialTxn.covers(route));
+        Invariants.checkState(full.partialTxn.covers(route));
         return true;
     }
 
@@ -128,15 +134,15 @@ public class RecoverWithRoute extends CheckShards
 
             case OutcomeApplied:
             case OutcomeKnown:
-                Preconditions.checkState(known.definition.isKnown());
-                Preconditions.checkState(known.executeAt.isDecisionKnown());
+                Invariants.checkState(known.definition.isKnown());
+                Invariants.checkState(known.executeAt.isDecisionKnown());
                 // TODO: we might not be able to reconstitute Txn if we have GC'd on some shards
                 Txn txn = merged.partialTxn.reconstitute(route);
                 if (known.deps.isDecisionKnown())
                 {
-                    Deps deps = merged.committedDeps.reconstitute(route);
+                    Deps deps = merged.committedDeps.reconstitute(route());
                     node.withEpoch(merged.executeAt.epoch, () -> {
-                        Persist.persistAndCommit(node, txnId, route, txn, merged.executeAt, deps, merged.writes, merged.result);
+                        Persist.persistAndCommit(node, txnId, route(), txn, merged.executeAt, deps, merged.writes, merged.result);
                     });
                     callback.accept(APPLIED, null);
                 }
diff --git a/accord-core/src/main/java/accord/coordinate/tracking/ReadTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/ReadTracker.java
index 30f1513..4d4142e 100644
--- a/accord-core/src/main/java/accord/coordinate/tracking/ReadTracker.java
+++ b/accord-core/src/main/java/accord/coordinate/tracking/ReadTracker.java
@@ -25,7 +25,7 @@ import java.util.function.BiFunction;
 import accord.topology.Shard;
 import accord.topology.ShardSelection;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import accord.local.Node.Id;
 import accord.topology.Topologies;
@@ -62,7 +62,7 @@ public class ReadTracker extends AbstractTracker<ReadTracker.ReadShardTracker, B
 
         public ShardOutcome<? super ReadTracker> recordSlowResponse(boolean ignore)
         {
-            Preconditions.checkState(!hasFailed());
+            Invariants.checkState(!hasFailed());
             ++slow;
 
             if (shouldRead() && canRead())
@@ -76,7 +76,7 @@ public class ReadTracker extends AbstractTracker<ReadTracker.ReadShardTracker, B
          */
         public ShardOutcome<? super ReadTracker> recordReadSuccess(boolean isSlow)
         {
-            Preconditions.checkState(inflight > 0);
+            Invariants.checkState(inflight > 0);
             boolean hadSucceeded = hasSucceeded();
             --inflight;
             if (isSlow) --slow;
@@ -86,7 +86,7 @@ public class ReadTracker extends AbstractTracker<ReadTracker.ReadShardTracker, B
 
         public ShardOutcome<? super ReadTracker> recordQuorumReadSuccess(boolean isSlow)
         {
-            Preconditions.checkState(inflight > 0);
+            Invariants.checkState(inflight > 0);
             boolean hadSucceeded = hasSucceeded();
             --inflight;
             ++quorum;
@@ -103,7 +103,7 @@ public class ReadTracker extends AbstractTracker<ReadTracker.ReadShardTracker, B
 
         public ShardOutcomes recordReadFailure(boolean isSlow)
         {
-            Preconditions.checkState(inflight > 0);
+            Invariants.checkState(inflight > 0);
             --inflight;
             if (isSlow) --slow;
 
@@ -255,7 +255,7 @@ public class ReadTracker extends AbstractTracker<ReadTracker.ReadShardTracker, B
             toRead = tmp;
         }
 
-        Preconditions.checkState(toRead != null, "We were asked to read more, but found no shards in need of reading more");
+        Invariants.checkState(toRead != null, "We were asked to read more, but found no shards in need of reading more");
 
         // TODO: maybe for each additional candidate do one linear compare run to find better secondary match
         //       OR at least discount candidates that do not contribute additional knowledge beyond those additional
diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommand.java b/accord-core/src/main/java/accord/impl/InMemoryCommand.java
index 5d0eeaa..79a5ec7 100644
--- a/accord-core/src/main/java/accord/impl/InMemoryCommand.java
+++ b/accord-core/src/main/java/accord/impl/InMemoryCommand.java
@@ -37,7 +37,7 @@ public class InMemoryCommand extends Command
     public final CommandStore commandStore;
     private final TxnId txnId;
 
-    private AbstractRoute route;
+    private Route<?> route;
     private RoutingKey homeKey, progressKey;
     private PartialTxn partialTxn;
     private Kind kind;
@@ -135,13 +135,13 @@ public class InMemoryCommand extends Command
     }
 
     @Override
-    public AbstractRoute route()
+    public Route<?> route()
     {
         return route;
     }
 
     @Override
-    protected void setRoute(AbstractRoute route)
+    protected void setRoute(Route<?> route)
     {
         this.route = route;
     }
diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
index 4363b25..8cd1de2 100644
--- a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
+++ b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
@@ -22,9 +22,11 @@ import accord.api.Agent;
 import accord.api.DataStore;
 import accord.api.Key;
 import accord.api.ProgressLog;
-import accord.api.RoutingKey;
+import accord.local.CommandStore; // java8 fails compilation if this is in correct position
+import accord.local.SyncCommandStores.SyncCommandStore; // java8 fails compilation if this is in correct position
+import accord.impl.InMemoryCommandStore.SingleThread.AsyncState;
+import accord.impl.InMemoryCommandStore.Synchronized.SynchronizedState;
 import accord.local.Command;
-import accord.local.CommandStore;
 import accord.local.CommandStore.RangesForEpoch;
 import accord.local.CommandsForKey;
 import accord.local.CommandListener;
@@ -33,25 +35,19 @@ import accord.local.NodeTimeService;
 import accord.local.PreLoadContext;
 import accord.local.SafeCommandStore;
 import accord.local.SyncCommandStores;
-import accord.local.SyncCommandStores.SyncCommandStore;
-import accord.impl.InMemoryCommandStore.SingleThread.AsyncState;
-import accord.impl.InMemoryCommandStore.Synchronized.SynchronizedState;
-import accord.primitives.KeyRange;
-import accord.primitives.KeyRanges;
-import accord.primitives.Keys;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
+import accord.primitives.*;
 import org.apache.cassandra.utils.concurrent.AsyncPromise;
 import org.apache.cassandra.utils.concurrent.Future;
 
 import java.util.Collection;
-import java.util.Comparator;
 import java.util.NavigableMap;
-import java.util.Objects;
 import java.util.TreeMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BinaryOperator;
 import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.stream.Collectors;
@@ -68,7 +64,7 @@ public class InMemoryCommandStore
 
         private final CommandStore commandStore;
         private final NavigableMap<TxnId, Command> commands = new TreeMap<>();
-        private final NavigableMap<RoutingKey, InMemoryCommandsForKey> commandsForKey = new TreeMap<>();
+        private final NavigableMap<RoutableKey, InMemoryCommandsForKey> commandsForKey = new TreeMap<>();
 
         public State(NodeTimeService time, Agent agent, DataStore store, ProgressLog progressLog, RangesForEpoch rangesForEpoch, CommandStore commandStore)
         {
@@ -158,9 +154,9 @@ public class InMemoryCommandStore
         }
 
         @Override
-        public Timestamp preaccept(TxnId txnId, Keys keys)
+        public Timestamp preaccept(TxnId txnId, Seekables<?, ?> keys)
         {
-            Timestamp max = maxConflict(keys);
+            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;
@@ -174,26 +170,21 @@ public class InMemoryCommandStore
             return time;
         }
 
-        private Timestamp maxConflict(Keys keys)
+        private Timestamp maxConflict(Seekables<?, ?> keysOrRanges, Ranges slice)
         {
-            return keys.stream()
-                    .map(this::maybeCommandsForKey)
-                    .filter(Objects::nonNull)
-                    .map(CommandsForKey::max)
-                    .max(Comparator.naturalOrder())
-                    .orElse(Timestamp.NONE);
+            return mapReduce(keysOrRanges, slice, CommandsForKey::max, (a, b) -> a.compareTo(b) >= 0 ? a : b, Timestamp.NONE);
         }
 
-        public void forEpochCommands(KeyRanges ranges, long epoch, Consumer<Command> consumer)
+        public void forEpochCommands(Ranges ranges, long epoch, Consumer<Command> consumer)
         {
             Timestamp minTimestamp = new Timestamp(epoch, Long.MIN_VALUE, Integer.MIN_VALUE, Node.Id.NONE);
             Timestamp maxTimestamp = new Timestamp(epoch, Long.MAX_VALUE, Integer.MAX_VALUE, Node.Id.MAX);
-            for (KeyRange range : ranges)
+            for (Range range : ranges)
             {
-                Iterable<InMemoryCommandsForKey> rangeCommands = commandsForKey.subMap(range.start(),
-                        range.startInclusive(),
-                        range.end(),
-                        range.endInclusive()).values();
+                Iterable<InMemoryCommandsForKey> rangeCommands = commandsForKey.subMap(
+                        range.start(), range.startInclusive(),
+                        range.end(), range.endInclusive()
+                ).values();
                 for (InMemoryCommandsForKey commands : rangeCommands)
                 {
                     commands.forWitnessed(minTimestamp, maxTimestamp, cmd -> consumer.accept((Command) cmd));
@@ -201,11 +192,11 @@ public class InMemoryCommandStore
             }
         }
 
-        public void forCommittedInEpoch(KeyRanges ranges, long epoch, Consumer<Command> consumer)
+        public void forCommittedInEpoch(Ranges ranges, long epoch, Consumer<Command> consumer)
         {
             Timestamp minTimestamp = new Timestamp(epoch, Long.MIN_VALUE, Integer.MIN_VALUE, Node.Id.NONE);
             Timestamp maxTimestamp = new Timestamp(epoch, Long.MAX_VALUE, Integer.MAX_VALUE, Node.Id.MAX);
-            for (KeyRange range : ranges)
+            for (Range range : ranges)
             {
                 Iterable<InMemoryCommandsForKey> rangeCommands = commandsForKey.subMap(range.start(),
                         range.startInclusive(),
@@ -213,13 +204,77 @@ public class InMemoryCommandStore
                         range.endInclusive()).values();
                 for (InMemoryCommandsForKey commands : rangeCommands)
                 {
-
                     Collection<Command> committed = commands.committedByExecuteAt()
                             .between(minTimestamp, maxTimestamp).map(cmd -> (Command) cmd).collect(Collectors.toList());
                     committed.forEach(consumer);
                 }
             }
         }
+
+        public <T> T mapReduce(Routables<?, ?> keysOrRanges, Ranges slice, Function<CommandsForKey, T> map, BinaryOperator<T> reduce, T initialValue)
+        {
+            switch (keysOrRanges.kindOfContents()) {
+                default:
+                    throw new AssertionError();
+                case Key:
+                    // TODO: efficiency
+                    AbstractKeys<Key, ?> keys = (AbstractKeys<Key, ?>) keysOrRanges;
+                    return keys.stream()
+                            .filter(slice::contains)
+                            .filter(commandStore::hashIntersects)
+                            .map(this::commandsForKey)
+                            .map(map)
+                            .reduce(initialValue, reduce);
+                case Range:
+                    // TODO: efficiency
+                    Ranges ranges = (Ranges) keysOrRanges;
+                    return ranges.slice(slice).stream().flatMap(range ->
+                            commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive()).values().stream()
+                    ).map(map).reduce(initialValue, reduce);
+            }
+        }
+
+        public void forEach(Routables<?, ?> keysOrRanges, Ranges slice, Consumer<CommandsForKey> forEach)
+        {
+            switch (keysOrRanges.kindOfContents()) {
+                default:
+                    throw new AssertionError();
+                case Key:
+                    AbstractKeys<Key, ?> keys = (AbstractKeys<Key, ?>) keysOrRanges;
+                    keys.forEach(slice, key -> {
+                        if (commandStore.hashIntersects(key))
+                            forEach.accept(commandsForKey(key));
+                    });
+                    break;
+                case Range:
+                    Ranges ranges = (Ranges) keysOrRanges;
+                    // TODO: zero allocation
+                    ranges.slice(slice).forEach(range -> {
+                        commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive())
+                                .values().forEach(forEach);
+                    });
+            }
+        }
+
+        public void forEach(Routable keyOrRange, Ranges slice, Consumer<CommandsForKey> forEach)
+        {
+            switch (keyOrRange.kind())
+            {
+                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;
+                    // TODO: zero allocation
+                    Ranges.of(range).slice(slice).forEach(r -> {
+                        commandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive())
+                                .values().forEach(forEach);
+                    });
+            }
+        }
     }
 
     public static class Synchronized extends SyncCommandStore
@@ -451,14 +506,14 @@ public class InMemoryCommandStore
             }
 
             @Override
-            public void forEpochCommands(KeyRanges ranges, long epoch, Consumer<Command> consumer)
+            public void forEpochCommands(Ranges ranges, long epoch, Consumer<Command> consumer)
             {
                 assertThread();
                 super.forEpochCommands(ranges, epoch, consumer);
             }
 
             @Override
-            public void forCommittedInEpoch(KeyRanges ranges, long epoch, Consumer<Command> consumer)
+            public void forCommittedInEpoch(Ranges ranges, long epoch, Consumer<Command> consumer)
             {
                 assertThread();
                 super.forCommittedInEpoch(ranges, epoch, consumer);
diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommandStores.java b/accord-core/src/main/java/accord/impl/InMemoryCommandStores.java
index 90aafca..d7b726f 100644
--- a/accord-core/src/main/java/accord/impl/InMemoryCommandStores.java
+++ b/accord-core/src/main/java/accord/impl/InMemoryCommandStores.java
@@ -18,9 +18,13 @@
 
 package accord.impl;
 
-import accord.api.*;
 import accord.local.*;
-import accord.primitives.AbstractKeys;
+import accord.api.Agent;
+import accord.api.DataStore;
+import accord.api.ProgressLog;
+import accord.local.CommandStore;
+import accord.local.Node;
+import accord.primitives.Routables;
 import accord.utils.MapReduce;
 
 import java.util.function.BiFunction;
@@ -35,12 +39,12 @@ public class InMemoryCommandStores
             super(num, node, agent, store, progressLogFactory, InMemoryCommandStore.Synchronized::new);
         }
 
-        public <T> T mapReduce(PreLoadContext context, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, MapReduce<? super SafeCommandStore, T> map)
+        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, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, Function<? super SafeCommandStore, T> map, BiFunction<T, T, T> reduce)
+        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
diff --git a/accord-core/src/main/java/accord/impl/SimpleProgressLog.java b/accord-core/src/main/java/accord/impl/SimpleProgressLog.java
index e2c22ad..cfde871 100644
--- a/accord-core/src/main/java/accord/impl/SimpleProgressLog.java
+++ b/accord-core/src/main/java/accord/impl/SimpleProgressLog.java
@@ -33,7 +33,7 @@ import accord.coordinate.*;
 import accord.local.*;
 import accord.local.Status.Known;
 import accord.primitives.*;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import accord.api.ProgressLog;
 import accord.api.RoutingKey;
@@ -66,6 +66,7 @@ import static accord.local.Status.Durability.Durable;
 import static accord.local.Status.Known.Nothing;
 import static accord.local.Status.PreApplied;
 import static accord.local.Status.PreCommitted;
+import static accord.primitives.Route.isFullRoute;
 
 // TODO: consider propagating invalidations in the same way as we do applied
 public class SimpleProgressLog implements Runnable, ProgressLog.Factory
@@ -205,7 +206,7 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
                                         commandStore.execute(contextFor(txnId), safeStore -> {
                                             Command cmd = safeStore.command(txnId);
                                             cmd.setDurability(safeStore, token.durability, homeKey, null);
-                                            safeStore.progressLog().durable(txnId, cmd.maxRoutingKeys(), null);
+                                            safeStore.progressLog().durable(txnId, cmd.maxUnseekables(), null);
                                         }).addCallback(commandStore.agent());
                                     }
 
@@ -296,7 +297,7 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
             if (!command.status().hasBeen(Status.PreCommitted))
                 return false;
 
-            if (!(command.route() instanceof Route))
+            if (!isFullRoute(command.route()))
                 return false;
 
             if (!node.topology().hasEpoch(command.executeAt().epoch))
@@ -384,7 +385,7 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
                 return;
             }
 
-            Route route = (Route) command.route();
+            FullRoute<?> route = Route.castToFullRoute(command.route());
             Timestamp executeAt = command.executeAt();
             investigating = new CoordinateAwareness();
             Topologies topologies = node.topology().preciseEpochs(route, txnId.epoch, executeAt.epoch);
@@ -402,15 +403,16 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
     {
         Known blockedUntil = Nothing;
         Progress progress = NoneExpected;
-        RoutingKeys blockedOnKeys;
+
+        Unseekables<?, ?> blockedOn;
 
         Object debugInvestigating;
 
-        void recordBlocking(Known blockedUntil, RoutingKeys blockedOnKeys)
+        void recordBlocking(Known blockedUntil, Unseekables<?, ?> blockedOn)
         {
-            Preconditions.checkState(!blockedOnKeys.isEmpty());
-            if (this.blockedOnKeys == null) this.blockedOnKeys = blockedOnKeys;
-            else this.blockedOnKeys = this.blockedOnKeys.union(blockedOnKeys);
+            Invariants.checkState(!blockedOn.isEmpty());
+            if (this.blockedOn == null) this.blockedOn = blockedOn;
+            else this.blockedOn = Unseekables.merge(this.blockedOn, (Unseekables)blockedOn);
             if (!blockedUntil.isSatisfiedBy(this.blockedUntil))
             {
                 this.blockedUntil = this.blockedUntil.merge(blockedUntil);
@@ -444,7 +446,7 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
             long srcEpoch = (executeAt != null ? executeAt : txnId).epoch;
             // TODO: compute fromEpoch, the epoch we already have this txn replicated until
             long toEpoch = Math.max(srcEpoch, node.topology().epoch());
-            RoutingKeys someKeys = someKeys(command);
+            Unseekables<?, ?> someKeys = unseekables(command);
 
             BiConsumer<Known, Throwable> callback = (success, fail) -> {
                 if (progress != Investigating)
@@ -463,20 +465,16 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
             });
         }
 
-        private RoutingKeys someKeys(Command command)
+        private Unseekables<?, ?> unseekables(Command command)
         {
-            AbstractRoute someRoute = command.route();
-            if (someRoute == null)
-                return blockedOnKeys;
-            if (blockedOnKeys instanceof AbstractRoute)
-                someRoute = AbstractRoute.merge(someRoute, (AbstractRoute) blockedOnKeys);
-            return someRoute;
+            return Unseekables.merge((Route)command.route(), blockedOn);
         }
 
-        private void invalidate(Node node, TxnId txnId, RoutingKeys someKeys)
+        private void invalidate(Node node, TxnId txnId, Unseekables<?, ?> someKeys)
         {
             progress = Investigating;
-            RoutingKey someKey = someKeys instanceof AbstractRoute ? ((AbstractRoute) someKeys).homeKey : someKeys.get(0);
+            // TODO (RangeTxns): This should be a Routable, or we should guarantee it is safe to operate on any key in the range
+            RoutingKey someKey = Route.isRoute(someKeys) ? (Route.castToRoute(someKeys)).homeKey() : someKeys.get(0).someIntersectingRoutingKey();
             someKeys = someKeys.with(someKey);
             debugInvestigating = Invalidate.invalidate(node, txnId, someKeys, (success, fail) -> {
                 if (progress != Investigating)
@@ -515,12 +513,12 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
             this.commandStore = commandStore;
         }
 
-        void recordBlocking(TxnId txnId, Known waitingFor, RoutingKeys someKeys)
+        void recordBlocking(TxnId txnId, Known waitingFor, Unseekables<?, ?> unseekables)
         {
-            Preconditions.checkArgument(txnId.equals(this.txnId));
+            Invariants.checkArgument(txnId.equals(this.txnId));
             if (blockingState == null)
                 blockingState = new BlockingState();
-            blockingState.recordBlocking(waitingFor, someKeys);
+            blockingState.recordBlocking(waitingFor, unseekables);
         }
 
         void ensureAtLeast(NonHomeState ensureAtLeast)
@@ -647,7 +645,7 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
         @Override
         public void preaccepted(Command command, ProgressShard shard)
         {
-            Preconditions.checkState(shard != Unsure);
+            Invariants.checkState(shard != Unsure);
 
             if (shard.isProgress())
             {
@@ -675,7 +673,7 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
 
         private void ensureSafeOrAtLeast(Command command, ProgressShard shard, CoordinateStatus newStatus, Progress newProgress)
         {
-            Preconditions.checkState(shard != Unsure);
+            Invariants.checkState(shard != Unsure);
 
             State state = null;
             assert newStatus.isAtMost(ReadyToExecute);
@@ -722,7 +720,7 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
         {
             State state = recordApply(command.txnId());
 
-            Preconditions.checkState(shard == Home || state == null || state.coordinateState == null);
+            Invariants.checkState(shard == Home || state == null || state.coordinateState == null);
 
             // note: we permit Unsure here, so we check if we have any local home state
             if (shard.isProgress())
@@ -746,25 +744,25 @@ public class SimpleProgressLog implements Runnable, ProgressLog.Factory
         {
             State state = ensure(command.txnId());
             if (!command.status().hasBeen(PreApplied))
-                state.recordBlocking(command.txnId(), PreApplied.minKnown, command.maxRoutingKeys());
+                state.recordBlocking(command.txnId(), PreApplied.minKnown, command.maxUnseekables());
             state.local().durableGlobal();
             state.global().durableGlobal(node, command, persistedOn);
         }
 
         @Override
-        public void durable(TxnId txnId, RoutingKeys someKeys, ProgressShard shard)
+        public void durable(TxnId txnId, Unseekables<?, ?> unseekables, ProgressShard shard)
         {
             State state = ensure(txnId);
             // TODO: we can probably simplify things by requiring (empty) Apply messages to be sent also to the coordinating topology
-            state.recordBlocking(txnId, PreApplied.minKnown, someKeys);
+            state.recordBlocking(txnId, PreApplied.minKnown, unseekables);
         }
 
-        public void waiting(TxnId blockedBy, Known blockedUntil, RoutingKeys blockedOnKeys)
+        public void waiting(TxnId blockedBy, Known blockedUntil, Unseekables<?, ?> blockedOn)
         {
             // TODO (soon): forward to progress shard for processing (if known)
             // TODO (soon): if we are co-located with the home shard, don't need to do anything unless we're in a
             //              later topology that wasn't covered by its coordination
-            ensure(blockedBy).recordBlocking(blockedBy, blockedUntil, blockedOnKeys);
+            ensure(blockedBy).recordBlocking(blockedBy, blockedUntil, blockedOn);
         }
     }
 
diff --git a/accord-core/src/main/java/accord/local/AsyncCommandStores.java b/accord-core/src/main/java/accord/local/AsyncCommandStores.java
index f64abfa..6c0a6dd 100644
--- a/accord-core/src/main/java/accord/local/AsyncCommandStores.java
+++ b/accord-core/src/main/java/accord/local/AsyncCommandStores.java
@@ -3,7 +3,7 @@ package accord.local;
 import accord.api.Agent;
 import accord.api.DataStore;
 import accord.api.ProgressLog;
-import accord.primitives.AbstractKeys;
+import accord.primitives.Routables;
 import accord.utils.MapReduce;
 import accord.utils.MapReduceConsume;
 import accord.utils.ReducingFuture;
@@ -61,7 +61,7 @@ public class AsyncCommandStores extends CommandStores<CommandStore>
     }
 
     @Override
-    public <O> void mapReduceConsume(PreLoadContext context, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
+    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);
     }
diff --git a/accord-core/src/main/java/accord/local/Command.java b/accord-core/src/main/java/accord/local/Command.java
index a7ab7fc..cf483a5 100644
--- a/accord-core/src/main/java/accord/local/Command.java
+++ b/accord-core/src/main/java/accord/local/Command.java
@@ -24,7 +24,7 @@ import accord.local.Status.Known;
 import accord.primitives.*;
 import accord.primitives.Txn.Kind;
 import accord.primitives.Writes;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -37,21 +37,24 @@ 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.AbstractRoute;
-import accord.primitives.KeyRanges;
+import accord.primitives.Ranges;
 import accord.primitives.Ballot;
 import accord.primitives.PartialDeps;
 import accord.primitives.PartialTxn;
 import accord.primitives.Route;
-import accord.primitives.RoutingKeys;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
+import accord.api.Result;
+import accord.api.RoutingKey;
 
 import static accord.api.ProgressLog.ProgressShard.Home;
 import static accord.api.ProgressLog.ProgressShard.Local;
@@ -121,8 +124,8 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
      * TODO: maybe set this for all local shards, but slice to only those participating keys
      * (would probably need to remove hashIntersects)
      */
-    public abstract AbstractRoute route();
-    protected abstract void setRoute(AbstractRoute route);
+    public abstract @Nullable Route<?> route();
+    protected abstract void setRoute(Route<?> route);
 
     public abstract PartialTxn partialTxn();
     protected abstract void setPartialTxn(PartialTxn txn);
@@ -188,7 +191,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
         // TODO (now): when do we need this, and will it always be sufficient?
         return partialTxn().keys();
@@ -207,17 +210,17 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
         Success, Redundant, RejectedBallot
     }
 
-    public AcceptOutcome preaccept(SafeCommandStore safeStore, PartialTxn partialTxn, AbstractRoute route, @Nullable RoutingKey progressKey)
+    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, AbstractRoute route, @Nullable RoutingKey progressKey, Ballot ballot)
+    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, AbstractRoute route, @Nullable RoutingKey progressKey, Ballot ballot)
+    private AcceptOutcome preacceptOrRecover(SafeCommandStore safeStore, PartialTxn partialTxn, Route<?> route, @Nullable RoutingKey progressKey, Ballot ballot)
     {
         int compareBallots = promised().compareTo(ballot);
         if (compareBallots > 0)
@@ -233,16 +236,16 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
 
         if (known().definition.isKnown())
         {
-            Preconditions.checkState(status() == Invalidated || executeAt() != null);
+            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;
         }
 
-        KeyRanges coordinateRanges = coordinateRanges(safeStore);
+        Ranges coordinateRanges = coordinateRanges(safeStore);
         ProgressShard shard = progressShard(safeStore, route, progressKey, coordinateRanges);
-        if (!validate(KeyRanges.EMPTY, coordinateRanges, shard, route, Set, partialTxn, Set, null, Ignore))
+        if (!validate(Ranges.EMPTY, coordinateRanges, shard, route, Set, partialTxn, Set, null, Ignore))
             throw new IllegalStateException();
 
         if (executeAt() == null)
@@ -265,7 +268,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
             // TODO: 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, KeyRanges.EMPTY, coordinateRanges, shard, route, partialTxn, Set, null, Ignore);
+        set(safeStore, Ranges.EMPTY, coordinateRanges, shard, route, partialTxn, Set, null, Ignore);
 
         notifyListeners(safeStore);
         return AcceptOutcome.Success;
@@ -282,7 +285,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
         return true;
     }
 
-    public AcceptOutcome accept(SafeCommandStore safeStore, Ballot ballot, Kind kind, PartialRoute route, Keys keys, @Nullable RoutingKey progressKey, Timestamp executeAt, PartialDeps partialDeps)
+    public AcceptOutcome accept(SafeCommandStore safeStore, Ballot ballot, Kind kind, PartialRoute<?> route, Seekables<?, ?> keys, @Nullable RoutingKey progressKey, Timestamp executeAt, PartialDeps partialDeps)
     {
         if (this.promised().compareTo(ballot) > 0)
         {
@@ -300,29 +303,27 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
             throw new IllegalArgumentException("Transaction kind is different to the definition we have already received");
 
         TxnId txnId = txnId();
-        KeyRanges coordinateRanges = coordinateRanges(safeStore);
-        KeyRanges executeRanges = txnId.epoch == executeAt.epoch ? coordinateRanges : safeStore.ranges().at(executeAt.epoch);
-        KeyRanges acceptRanges = txnId.epoch == executeAt.epoch ? coordinateRanges : safeStore.ranges().between(txnId.epoch, executeAt.epoch);
+        Ranges coordinateRanges = coordinateRanges(safeStore);
+        Ranges acceptRanges = txnId.epoch == executeAt.epoch ? coordinateRanges : safeStore.ranges().between(txnId.epoch, executeAt.epoch);
         ProgressShard shard = progressShard(safeStore, route, progressKey, coordinateRanges);
 
-        if (!validate(coordinateRanges, executeRanges, shard, route, Ignore, null, Ignore, partialDeps, Set))
+        if (!validate(coordinateRanges, Ranges.EMPTY, shard, route, Ignore, null, Ignore, partialDeps, Set))
+        {
+            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);
         setKind(kind);
-        set(safeStore, coordinateRanges, executeRanges, shard, route, null, Ignore, partialDeps, Set);
+        set(safeStore, coordinateRanges, Ranges.EMPTY, shard, route, null, Ignore, partialDeps, Set);
         switch (status())
         {
             // if we haven't already registered, do so, to correctly maintain max per-key timestamp
             case NotWitnessed:
             case AcceptedInvalidate:
-                keys.foldl(acceptRanges, (i, k, p, v) -> {
-                    if (safeStore.commandStore().hashIntersects(k))
-                        safeStore.commandsForKey(k).register(this);
-                    return 0L;
-                }, 0L, 0L, 1L);
+                safeStore.forEach(keys, acceptRanges, forKey -> forKey.register(this));
         }
         setStatus(Accepted);
 
@@ -359,7 +360,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
     public enum CommitOutcome { Success, Redundant, Insufficient }
 
     // relies on mutual exclusion for each key
-    public CommitOutcome commit(SafeCommandStore safeStore, AbstractRoute route, @Nullable RoutingKey progressKey, @Nullable PartialTxn partialTxn, Timestamp executeAt, PartialDeps partialDeps)
+    public CommitOutcome commit(SafeCommandStore safeStore, Route<?> route, @Nullable RoutingKey progressKey, @Nullable PartialTxn partialTxn, Timestamp executeAt, PartialDeps partialDeps)
     {
         if (hasBeen(PreCommitted))
         {
@@ -371,9 +372,9 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
                 return CommitOutcome.Redundant;
         }
 
-        KeyRanges coordinateRanges = coordinateRanges(safeStore);
+        Ranges coordinateRanges = coordinateRanges(safeStore);
         // TODO (now): consider ranges between coordinateRanges and executeRanges? Perhaps don't need them
-        KeyRanges executeRanges = executeRanges(safeStore, executeAt);
+        Ranges executeRanges = executeRanges(safeStore, executeAt);
         ProgressShard shard = progressShard(safeStore, route, progressKey, coordinateRanges);
 
         if (!validate(coordinateRanges, executeRanges, shard, route, Check, partialTxn, Add, partialDeps, Set))
@@ -413,7 +414,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
 
     protected void populateWaitingOn(SafeCommandStore safeStore)
     {
-        KeyRanges ranges = safeStore.ranges().since(executeAt().epoch);
+        Ranges ranges = safeStore.ranges().since(executeAt().epoch);
         if (ranges != null) {
             partialDeps().forEachOn(ranges, safeStore.commandStore()::hashIntersects, txnId -> {
                 Command command = safeStore.ifLoaded(txnId);
@@ -478,7 +479,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
 
     public enum ApplyOutcome { Success, Redundant, Insufficient }
 
-    public ApplyOutcome apply(SafeCommandStore safeStore, long untilEpoch, AbstractRoute route, Timestamp executeAt, @Nullable PartialDeps partialDeps, Writes writes, Result result)
+    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()))
         {
@@ -490,12 +491,12 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
             safeStore.agent().onInconsistentTimestamp(this, this.executeAt(), executeAt);
         }
 
-        KeyRanges coordinateRanges = coordinateRanges(safeStore);
-        KeyRanges executeRanges = executeRanges(safeStore, executeAt);
+        Ranges coordinateRanges = coordinateRanges(safeStore);
+        Ranges executeRanges = executeRanges(safeStore, executeAt);
         if (untilEpoch < safeStore.latestEpoch())
         {
-            KeyRanges expectedRanges = safeStore.ranges().between(executeAt.epoch, untilEpoch);
-            Preconditions.checkState(expectedRanges.contains(executeRanges));
+            Ranges expectedRanges = safeStore.ranges().between(executeAt.epoch, untilEpoch);
+            Invariants.checkState(expectedRanges.containsAll(executeRanges));
         }
         ProgressShard shard = progressShard(safeStore, route, coordinateRanges);
 
@@ -521,7 +522,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
     @Override
     public PreLoadContext listenerPreLoadContext(TxnId caller)
     {
-        return PreLoadContext.contextFor(listOf(txnId(), caller), Collections.emptyList());
+        return PreLoadContext.contextFor(listOf(txnId(), caller));
     }
 
     @Override
@@ -582,6 +583,8 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
     }
 
     // TODO: maybe split into maybeExecute and maybeApply?
+    // TODO (performance): If we are a no-op on this shard, just immediately apply.
+    //      NOTE: if we ever do transitive dependency elision this could be dangerous
     private boolean maybeExecute(SafeCommandStore safeStore, ProgressShard shard, boolean alwaysNotifyListeners, boolean notifyWaitingOn)
     {
         if (logger.isTraceEnabled())
@@ -636,7 +639,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
      */
     private boolean updatePredecessor(Command dependency)
     {
-        Preconditions.checkState(dependency.hasBeen(PreCommitted));
+        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());
@@ -674,7 +677,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
 
     private void insertPredecessor(Command dependency)
     {
-        Preconditions.checkState(dependency.hasBeen(PreCommitted));
+        Invariants.checkState(dependency.hasBeen(PreCommitted));
         if (dependency.hasBeen(Invalidated))
         {
             logger.trace("{}: {} is invalidated. Do not insert.", txnId(), dependency.txnId());
@@ -744,7 +747,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
                 else if (cur.has(until))
                 {
                     // we're done; have already applied
-                    Preconditions.checkState(depth == 0);
+                    Invariants.checkState(depth == 0);
                     break;
                 }
 
@@ -768,9 +771,9 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
                         continue;
                     }
 
-                    RoutingKeys someKeys = cur.maxRoutingKeys();
-                    if (someKeys == null && prev != null) someKeys = prev.partialDeps().someRoutingKeys(cur.txnId());
-                    Preconditions.checkState(someKeys != null);
+                    Unseekables<?, ?> someKeys = cur.maxUnseekables();
+                    if (someKeys == null && prev != null) someKeys = prev.partialDeps().someRoutables(cur.txnId());
+                    Invariants.checkState(someKeys != null);
                     logger.trace("{} blocked on {} until {}", txnIds[0], cur.txnId(), until);
                     safeStore.progressLog().waiting(cur.txnId(), until, someKeys);
                     return;
@@ -802,9 +805,9 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
         }
 
         @Override
-        public Iterable<Key> keys()
+        public Seekables<?, ?> keys()
         {
-            return Collections.emptyList();
+            return Keys.EMPTY;
         }
     }
 
@@ -841,9 +844,9 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
         }
     }
 
-    private ProgressShard progressShard(SafeCommandStore safeStore, AbstractRoute route, @Nullable RoutingKey progressKey, KeyRanges coordinateRanges)
+    private ProgressShard progressShard(SafeCommandStore safeStore, Route<?> route, @Nullable RoutingKey progressKey, Ranges coordinateRanges)
     {
-        updateHomeKey(safeStore, route.homeKey);
+        updateHomeKey(safeStore, route.homeKey());
 
         if (progressKey == null || progressKey == NO_PROGRESS_KEY)
         {
@@ -880,7 +883,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
         else if (!current.equals(progressKey)) throw new AssertionError();
     }
 
-    private ProgressShard progressShard(SafeCommandStore safeStore, AbstractRoute route, KeyRanges coordinateRanges)
+    private ProgressShard progressShard(SafeCommandStore safeStore, Route<?> route, Ranges coordinateRanges)
     {
         if (progressKey() == null)
             return Unsure;
@@ -897,7 +900,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
         if (progressKey == NO_PROGRESS_KEY)
             return No;
 
-        KeyRanges coordinateRanges = safeStore.ranges().at(txnId().epoch);
+        Ranges coordinateRanges = safeStore.ranges().at(txnId().epoch);
         if (!coordinateRanges.contains(progressKey))
             return No;
 
@@ -907,12 +910,12 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
         return progressKey.equals(homeKey()) ? Home : Local;
     }
 
-    private KeyRanges coordinateRanges(SafeCommandStore safeStore)
+    private Ranges coordinateRanges(SafeCommandStore safeStore)
     {
         return safeStore.ranges().at(txnId().epoch);
     }
 
-    private KeyRanges executeRanges(SafeCommandStore safeStore, Timestamp executeAt)
+    private Ranges executeRanges(SafeCommandStore safeStore, Timestamp executeAt)
     {
         return safeStore.ranges().since(executeAt.epoch);
     }
@@ -923,8 +926,8 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
      * 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(KeyRanges existingRanges, KeyRanges additionalRanges, ProgressShard shard,
-                             AbstractRoute route, EnsureAction ensureRoute,
+    private boolean validate(Ranges existingRanges, Ranges additionalRanges, ProgressShard shard,
+                             Route<?> route, EnsureAction ensureRoute,
                              @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn,
                              @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps)
     {
@@ -941,17 +944,17 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
                 {
                     default: throw new AssertionError();
                     case Check:
-                        if (!(route() instanceof Route) && !(route instanceof Route))
+                        if (!isFullRoute(route()) && !isFullRoute(route))
                             return false;
                     case Ignore:
                         break;
                     case Add:
                     case Set:
-                        if (!(route instanceof Route))
+                        if (!isFullRoute(route))
                             throw new IllegalArgumentException("Incomplete route (" + route + ") sent to home shard");
                         break;
                     case TrySet:
-                        if (!(route instanceof Route))
+                        if (!isFullRoute(route))
                             return false;
                 }
             }
@@ -979,7 +982,7 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
         // 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)
-            Preconditions.checkState(status() != Accepted && status() != AcceptedInvalidate);
+            Invariants.checkState(status() != Accepted && status() != AcceptedInvalidate);
 
         // validate new partial txn
         if (!validate(ensurePartialTxn, existingRanges, additionalRanges, covers(partialTxn()), covers(partialTxn), "txn", partialTxn))
@@ -998,15 +1001,15 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
     }
 
     private void set(SafeCommandStore safeStore,
-                     KeyRanges existingRanges, KeyRanges additionalRanges, ProgressShard shard, AbstractRoute route,
+                     Ranges existingRanges, Ranges additionalRanges, ProgressShard shard, Route<?> route,
                      @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn,
                      @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps)
     {
-        Preconditions.checkState(progressKey() != null);
-        KeyRanges allRanges = existingRanges.union(additionalRanges);
+        Invariants.checkState(progressKey() != null);
+        Ranges allRanges = existingRanges.union(additionalRanges);
 
-        if (shard.isProgress()) setRoute(AbstractRoute.merge(route(), route));
-        else setRoute(AbstractRoute.merge(route(), route.slice(allRanges)));
+        if (shard.isProgress()) setRoute(Route.merge(route(), (Route)route));
+        else setRoute(Route.merge(route(), (Route)route.slice(allRanges)));
 
         // TODO (soon): stop round-robin hashing; partition only on ranges
         switch (ensurePartialTxn)
@@ -1018,9 +1021,9 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
                 if (partialTxn() != null)
                 {
                     partialTxn = partialTxn.slice(allRanges, shard.isHome());
-                    partialTxn.keys().foldlDifference(partialTxn().keys(), (i, key, p, v) -> {
-                        if (safeStore.commandStore().hashIntersects(key))
-                            safeStore.commandsForKey(key).register(this);
+                    Routables.foldlMissing((Seekables)partialTxn.keys(), partialTxn().keys(), (i, keyOrRange, p, v) -> {
+                        // TODO: duplicate application of ranges
+                        safeStore.forEach(keyOrRange, allRanges, forKey -> forKey.register(this));
                         return v;
                     }, 0, 0, 1);
                     this.setPartialTxn(partialTxn().with(partialTxn));
@@ -1031,10 +1034,10 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
             case TrySet:
                 setKind(partialTxn.kind());
                 setPartialTxn(partialTxn = partialTxn.slice(allRanges, shard.isHome()));
-                partialTxn.keys().forEach(key -> {
+                // TODO: duplicate application of ranges
+                safeStore.forEach(partialTxn.keys(), allRanges, forKey -> {
                     // TODO: no need to register on PreAccept if already Accepted
-                    if (safeStore.commandStore().hashIntersects(key))
-                        safeStore.commandsForKey(key).register(this);
+                    forKey.register(this);
                 });
                 break;
         }
@@ -1058,8 +1061,8 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
         }
     }
 
-    private static boolean validate(EnsureAction action, KeyRanges existingRanges, KeyRanges additionalRanges,
-                                    KeyRanges existing, KeyRanges adding, String kind, Object obj)
+    private static boolean validate(EnsureAction action, Ranges existingRanges, Ranges additionalRanges,
+                                    Ranges existing, Ranges adding, String kind, Object obj)
     {
         switch (action)
         {
@@ -1070,21 +1073,21 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
             case TrySet:
                 if (adding != null)
                 {
-                    if (!adding.contains(existingRanges))
+                    if (!adding.containsAll(existingRanges))
                         return false;
 
-                    if (additionalRanges != existingRanges && !adding.contains(additionalRanges))
+                    if (additionalRanges != existingRanges && !adding.containsAll(additionalRanges))
                         return false;
 
                     break;
                 }
             case Set:
                 // failing any of these tests is always an illegal state
-                Preconditions.checkState(adding != null);
-                if (!adding.contains(existingRanges))
+                Invariants.checkState(adding != null);
+                if (!adding.containsAll(existingRanges))
                     throw new IllegalArgumentException("Incomplete " + kind + " (" + obj + ") provided; does not cover " + existingRanges);
 
-                if (additionalRanges != existingRanges && !adding.contains(additionalRanges))
+                if (additionalRanges != existingRanges && !adding.containsAll(additionalRanges))
                     throw new IllegalArgumentException("Incomplete " + kind + " (" + obj + ") provided; does not cover " + additionalRanges);
                 break;
 
@@ -1095,8 +1098,8 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
                     if (existing == null)
                         return false;
 
-                    Preconditions.checkState(existing.contains(existingRanges));
-                    if (existingRanges != additionalRanges && !existing.contains(additionalRanges))
+                    Invariants.checkState(existing.containsAll(existingRanges));
+                    if (existingRanges != additionalRanges && !existing.containsAll(additionalRanges))
                     {
                         if (action == Check)
                             return false;
@@ -1106,9 +1109,9 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
                 }
                 else if (existing != null)
                 {
-                    KeyRanges covering = adding.union(existing);
-                    Preconditions.checkState(covering.contains(existingRanges));
-                    if (existingRanges != additionalRanges && !covering.contains(additionalRanges))
+                    Ranges covering = adding.union(existing);
+                    Invariants.checkState(covering.containsAll(existingRanges));
+                    if (existingRanges != additionalRanges && !covering.containsAll(additionalRanges))
                     {
                         if (action == Check)
                             return false;
@@ -1118,10 +1121,10 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
                 }
                 else
                 {
-                    if (!adding.contains(existingRanges))
+                    if (!adding.containsAll(existingRanges))
                         return false;
 
-                    if (existingRanges != additionalRanges && !adding.contains(additionalRanges))
+                    if (existingRanges != additionalRanges && !adding.containsAll(additionalRanges))
                     {
                         if (action == Check)
                             return false;
@@ -1136,24 +1139,24 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
     }
 
     // TODO: callers should try to consult the local progress shard (if any) to obtain the full set of keys owned locally
-    public AbstractRoute someRoute()
+    public Route<?> someRoute()
     {
         if (route() != null)
             return route();
 
         if (homeKey() != null)
-            return new PartialRoute(KeyRanges.EMPTY, homeKey(), new RoutingKey[0]);
+            return new PartialKeyRoute(Ranges.EMPTY, homeKey(), new RoutingKey[0]);
 
         return null;
     }
 
-    public RoutingKeys maxRoutingKeys()
+    public Unseekables<?, ?> maxUnseekables()
     {
-        AbstractRoute route = someRoute();
+        Route<?> route = someRoute();
         if (route == null)
             return null;
 
-        return route.with(route.homeKey);
+        return route.toMaximalUnseekables();
     }
 
     /**
@@ -1185,12 +1188,12 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
                '}';
     }
 
-    private static KeyRanges covers(@Nullable PartialTxn txn)
+    private static Ranges covers(@Nullable PartialTxn txn)
     {
         return txn == null ? null : txn.covering();
     }
 
-    private static KeyRanges covers(@Nullable PartialDeps deps)
+    private static Ranges covers(@Nullable PartialDeps deps)
     {
         return deps == null ? null : deps.covering;
     }
@@ -1202,18 +1205,20 @@ public abstract class Command implements CommandListener, BiConsumer<SafeCommand
 
     // TODO: this is an ugly hack, need to encode progress/homeKey/Route state combinations much more clearly
     //  (perhaps introduce encapsulating class representing each possible arrangement)
-    private static final RoutingKey NO_PROGRESS_KEY = new RoutingKey()
+    static class NoProgressKey implements RoutingKey
     {
         @Override
-        public int routingHash()
+        public int compareTo(@Nonnull RoutableKey that)
         {
             throw new UnsupportedOperationException();
         }
 
         @Override
-        public int compareTo(@Nonnull RoutingKey ignore)
+        public int routingHash()
         {
             throw new UnsupportedOperationException();
         }
-    };
+    }
+
+    private static final NoProgressKey NO_PROGRESS_KEY = new NoProgressKey();
 }
diff --git a/accord-core/src/main/java/accord/local/CommandStore.java b/accord-core/src/main/java/accord/local/CommandStore.java
index 5bfa815..d28d618 100644
--- a/accord-core/src/main/java/accord/local/CommandStore.java
+++ b/accord-core/src/main/java/accord/local/CommandStore.java
@@ -20,16 +20,16 @@ package accord.local;
 
 import accord.api.*;
 import accord.local.CommandStores.ShardedRanges;
-import accord.primitives.AbstractKeys;
-import accord.primitives.KeyRanges;
-import accord.primitives.Keys;
+import accord.api.ProgressLog;
+import accord.primitives.*;
+import accord.api.DataStore;
 import org.apache.cassandra.utils.concurrent.Future;
 
-import com.google.common.base.Preconditions;
-
 import java.util.function.Consumer;
 import java.util.function.Function;
 
+import static accord.utils.Invariants.checkArgument;
+
 /**
  * Single threaded internal shard of accord transaction metadata
  */
@@ -50,11 +50,10 @@ public abstract class CommandStore
 
     public interface RangesForEpoch
     {
-        KeyRanges at(long epoch);
-        KeyRanges between(long fromInclusive, long toInclusive);
-        KeyRanges since(long epoch);
+        Ranges at(long epoch);
+        Ranges between(long fromInclusive, long toInclusive);
+        Ranges since(long epoch);
         boolean owns(long epoch, RoutingKey key);
-        boolean intersects(long epoch, AbstractKeys<?, ?> keys);
     }
 
     private final int id; // unique id
@@ -67,10 +66,9 @@ public abstract class CommandStore
                         int shardIndex,
                         int numShards)
     {
-        Preconditions.checkArgument(shardIndex < numShards);
         this.id = id;
         this.generation = generation;
-        this.shardIndex = shardIndex;
+        this.shardIndex = checkArgument(shardIndex, shardIndex < numShards);
         this.numShards = numShards;
     }
 
@@ -91,14 +89,14 @@ public abstract class CommandStore
         return generation;
     }
 
-    public boolean hashIntersects(RoutingKey key)
+    public boolean hashIntersects(RoutableKey key)
     {
         return ShardedRanges.keyIndex(key, numShards) == shardIndex;
     }
 
-    public boolean intersects(Keys keys, KeyRanges ranges)
+    public boolean hashIntersects(Routable routable)
     {
-        return keys.any(ranges, this::hashIntersects);
+        return routable instanceof Range || hashIntersects((RoutableKey) routable);
     }
 
     public abstract Agent agent();
diff --git a/accord-core/src/main/java/accord/local/CommandStores.java b/accord-core/src/main/java/accord/local/CommandStores.java
index fb06cf7..cecacba 100644
--- a/accord-core/src/main/java/accord/local/CommandStores.java
+++ b/accord-core/src/main/java/accord/local/CommandStores.java
@@ -20,16 +20,14 @@ package accord.local;
 
 import accord.api.*;
 import accord.local.CommandStore.RangesForEpoch;
-import accord.primitives.AbstractKeys;
-import accord.primitives.KeyRanges;
-import accord.primitives.RoutingKeys;
+import accord.primitives.*;
+import accord.api.RoutingKey;
 import accord.topology.Topology;
 import accord.utils.MapReduce;
 import accord.utils.MapReduceConsume;
 
 import accord.utils.ReducingFuture;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import org.apache.cassandra.utils.concurrent.Future;
 
 import java.util.ArrayList;
@@ -40,6 +38,8 @@ import java.util.stream.IntStream;
 
 import static accord.local.PreLoadContext.empty;
 
+import static accord.utils.Invariants.checkArgument;
+
 /**
  * Manages the single threaded metadata shards
  */
@@ -78,7 +78,7 @@ public abstract class CommandStores<S extends CommandStore>
             return shardFactory.create(id, generation, shardIndex, numShards, time, agent, store, progressLogFactory, rangesForEpoch);
         }
 
-        ShardedRanges createShardedRanges(int generation, long epoch, KeyRanges ranges, RangesForEpoch rangesForEpoch)
+        ShardedRanges createShardedRanges(int generation, long epoch, Ranges ranges, RangesForEpoch rangesForEpoch)
         {
             CommandStore[] newStores = new CommandStore[numShards];
             for (int i=0; i<numShards; i++)
@@ -92,42 +92,40 @@ public abstract class CommandStores<S extends CommandStore>
     {
         final CommandStore[] shards;
         final long[] epochs;
-        final KeyRanges[] ranges;
+        final Ranges[] ranges;
 
-        protected ShardedRanges(CommandStore[] shards, long epoch, KeyRanges ranges)
+        protected ShardedRanges(CommandStore[] shards, long epoch, Ranges ranges)
         {
-            Preconditions.checkArgument(shards.length <= 64);
-            this.shards = shards;
+            this.shards = checkArgument(shards, shards.length <= 64);
             this.epochs = new long[] { epoch };
-            this.ranges = new KeyRanges[] { ranges };
+            this.ranges = new Ranges[] { ranges };
         }
 
-        private ShardedRanges(CommandStore[] shards, long[] epochs, KeyRanges[] ranges)
+        private ShardedRanges(CommandStore[] shards, long[] epochs, Ranges[] ranges)
         {
-            Preconditions.checkArgument(shards.length <= 64);
-            this.shards = shards;
+            this.shards = checkArgument(shards, shards.length <= 64);
             this.epochs = epochs;
             this.ranges = ranges;
         }
 
-        ShardedRanges withRanges(long epoch, KeyRanges ranges)
+        ShardedRanges withRanges(long epoch, Ranges ranges)
         {
             long[] newEpochs = Arrays.copyOf(this.epochs, this.epochs.length + 1);
-            KeyRanges[] newRanges = Arrays.copyOf(this.ranges, this.ranges.length + 1);
+            Ranges[] newRanges = Arrays.copyOf(this.ranges, this.ranges.length + 1);
             newEpochs[this.epochs.length] = epoch;
             newRanges[this.ranges.length] = ranges;
             return new ShardedRanges(shards, newEpochs, newRanges);
         }
 
-        KeyRanges rangesForEpoch(long epoch)
+        Ranges rangesForEpoch(long epoch)
         {
             int i = Arrays.binarySearch(epochs, epoch);
             if (i < 0) i = -2 -i;
-            if (i < 0) return KeyRanges.EMPTY;
+            if (i < 0) return Ranges.EMPTY;
             return ranges[i];
         }
 
-        KeyRanges rangesBetweenEpochs(long fromInclusive, long toInclusive)
+        Ranges rangesBetweenEpochs(long fromInclusive, long toInclusive)
         {
             if (fromInclusive > toInclusive)
                 throw new IndexOutOfBoundsException();
@@ -141,19 +139,19 @@ public abstract class CommandStores<S extends CommandStore>
 
             int j = Arrays.binarySearch(epochs, toInclusive);
             if (j < 0) j = -2 - j;
-            if (i > j) return KeyRanges.EMPTY;
+            if (i > j) return Ranges.EMPTY;
 
-            KeyRanges result = ranges[i++];
+            Ranges result = ranges[i++];
             while (i <= j)
                 result = result.union(ranges[i++]);
             return result;
         }
 
-        KeyRanges rangesSinceEpoch(long epoch)
+        Ranges rangesSinceEpoch(long epoch)
         {
             int i = Arrays.binarySearch(epochs, epoch);
             if (i < 0) i = Math.max(0, -2 -i);
-            KeyRanges result = ranges[i++];
+            Ranges result = ranges[i++];
             while (i < ranges.length)
                 result = ranges[i++].union(result);
             return result;
@@ -171,27 +169,46 @@ public abstract class CommandStores<S extends CommandStore>
             return -1L >>> (64 - shards.length);
         }
 
-        public long shards(AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch)
+        public <T extends Routable> long shards(Routables<T, ?> keysOrRanges, long minEpoch, long maxEpoch)
         {
-            long accumulate = 0L;
-            for (int i = Math.max(0, indexForEpoch(minEpoch)), maxi = indexForEpoch(maxEpoch); i <= maxi ; ++i)
+            long terminalValue = -1L >>> (32 - shards.length);
+            switch (keysOrRanges.kindOfContents())
             {
-                accumulate = keys.foldl(ranges[i], ShardedRanges::addKeyIndex, shards.length, accumulate, -1L);
+                default: throw new AssertionError();
+                case Key:
+                {
+                    long accumulate = 0L;
+                    for (int i = Math.max(0, indexForEpoch(minEpoch)), maxi = indexForEpoch(maxEpoch); i <= maxi ; ++i)
+                    {
+                        accumulate = Routables.foldl((AbstractKeys<?, ?>)keysOrRanges, ranges[i], ShardedRanges::addKeyIndex, shards.length, accumulate, terminalValue);
+                    }
+                    return accumulate;
+                }
+
+                case Range:
+                {
+                    long accumulate = 0L;
+                    for (int i = Math.max(0, indexForEpoch(minEpoch)), maxi = indexForEpoch(maxEpoch); i <= maxi ; ++i)
+                    {
+                        // include every shard if we match a range
+                        accumulate = Routables.foldl((Ranges)keysOrRanges, ranges[i], (idx, k, p, a) -> p, terminalValue, accumulate, terminalValue);
+                    }
+                    return accumulate;
+                }
             }
-            return accumulate;
         }
 
-        KeyRanges currentRanges()
+        Ranges currentRanges()
         {
             return ranges[ranges.length - 1];
         }
 
-        static long keyIndex(RoutingKey key, long numShards)
+        static long keyIndex(RoutableKey key, long numShards)
         {
             return Integer.toUnsignedLong(key.routingHash()) % numShards;
         }
 
-        private static long addKeyIndex(int i, RoutingKey key, long numShards, long accumulate)
+        private static long addKeyIndex(int i, RoutableKey key, long numShards, long accumulate)
         {
             return accumulate | (1L << keyIndex(key, numShards));
         }
@@ -243,21 +260,21 @@ public abstract class CommandStores<S extends CommandStore>
 
     private Snapshot updateTopology(Snapshot prev, Topology newTopology)
     {
-        Preconditions.checkArgument(!newTopology.isSubset(), "Use full topology for CommandStores.updateTopology");
+        checkArgument(!newTopology.isSubset(), "Use full topology for CommandStores.updateTopology");
 
         long epoch = newTopology.epoch();
         if (epoch <= prev.global.epoch())
             return prev;
 
         Topology newLocalTopology = newTopology.forNode(supplier.time.id()).trim();
-        KeyRanges added = newLocalTopology.ranges().difference(prev.local.ranges());
-        KeyRanges subtracted = prev.local.ranges().difference(newLocalTopology.ranges());
+        Ranges added = newLocalTopology.ranges().difference(prev.local.ranges());
+        Ranges subtracted = prev.local.ranges().difference(newLocalTopology.ranges());
 //            for (ShardedRanges range : stores.ranges)
 //            {
 //                // FIXME: remove this (and the corresponding check in TopologyRandomizer) once lower bounds are implemented.
 //                //  In the meantime, the logic needed to support acquiring ranges that we previously replicated is pretty
 //                //  convoluted without the ability to jettison epochs.
-//                Preconditions.checkState(!range.ranges.intersects(added));
+//                Invariants.checkState(!range.ranges.intersects(added));
 //            }
 
         if (added.isEmpty() && subtracted.isEmpty())
@@ -292,19 +309,19 @@ public abstract class CommandStores<S extends CommandStore>
         return new RangesForEpoch()
         {
             @Override
-            public KeyRanges at(long epoch)
+            public Ranges at(long epoch)
             {
                 return current.ranges[generation].rangesForEpoch(epoch);
             }
 
             @Override
-            public KeyRanges between(long fromInclusive, long toInclusive)
+            public Ranges between(long fromInclusive, long toInclusive)
             {
                 return current.ranges[generation].rangesBetweenEpochs(fromInclusive, toInclusive);
             }
 
             @Override
-            public KeyRanges since(long epoch)
+            public Ranges since(long epoch)
             {
                 return current.ranges[generation].rangesSinceEpoch(epoch);
             }
@@ -315,11 +332,6 @@ public abstract class CommandStores<S extends CommandStore>
                 return current.ranges[generation].rangesForEpoch(epoch).contains(key);
             }
 
-            @Override
-            public boolean intersects(long epoch, AbstractKeys<?, ?> keys)
-            {
-                return at(epoch).intersects(keys);
-            }
         };
     }
 
@@ -356,12 +368,12 @@ public abstract class CommandStores<S extends CommandStore>
         return forEach(context, RoutingKeys.of(key), minEpoch, maxEpoch, forEach, true);
     }
 
-    public Future<Void> forEach(PreLoadContext context, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, Consumer<SafeCommandStore> forEach)
+    public Future<Void> forEach(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, Consumer<SafeCommandStore> forEach)
     {
         return forEach(context, keys, minEpoch, maxEpoch, forEach, true);
     }
 
-    private Future<Void> forEach(PreLoadContext context, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, Consumer<SafeCommandStore> forEach, boolean matchesMultiple)
+    private Future<Void> forEach(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, Consumer<SafeCommandStore> forEach, boolean matchesMultiple)
     {
         return this.mapReduce(context, keys, minEpoch, maxEpoch, new MapReduce<SafeCommandStore, Void>()
         {
@@ -384,7 +396,7 @@ public abstract class CommandStores<S extends CommandStore>
     }
 
     /**
-     * See {@link #mapReduceConsume(PreLoadContext, AbstractKeys, long, long, MapReduceConsume)}
+     * See {@link #mapReduceConsume(PreLoadContext, Routables, long, long, MapReduceConsume)}
      */
     public <O> void mapReduceConsume(PreLoadContext context, RoutingKey key, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
     {
@@ -398,13 +410,13 @@ public abstract class CommandStores<S extends CommandStore>
      * 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, AbstractKeys, long, long, MapReduceConsume, MapReduceAdapter)}
+     * Implementations are expected to invoke {@link #mapReduceConsume(PreLoadContext, Routables, long, long, MapReduceConsume, MapReduceAdapter)}
      */
-    public abstract <O> void mapReduceConsume(PreLoadContext context, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> 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, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume,
-                                                   MapReduceAdapter<? super S, T1, T2, O> adapter)
+    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)
     {
         T1 reduced = mapReduce(context, keys, minEpoch, maxEpoch, mapReduceConsume, adapter);
         adapter.consume(mapReduceConsume, reduced);
@@ -417,7 +429,7 @@ public abstract class CommandStores<S extends CommandStore>
         adapter.consume(mapReduceConsume, reduced);
     }
 
-    protected <T1, T2, O> T1 mapReduce(PreLoadContext context, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, MapReduce<? super SafeCommandStore, O> mapReduce,
+    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)
     {
         T2 accumulator = adapter.allocate();
diff --git a/accord-core/src/main/java/accord/local/CommandsForKey.java b/accord-core/src/main/java/accord/local/CommandsForKey.java
index 6c76d7e..eb03f28 100644
--- a/accord-core/src/main/java/accord/local/CommandsForKey.java
+++ b/accord-core/src/main/java/accord/local/CommandsForKey.java
@@ -21,6 +21,7 @@ package accord.local;
 import java.util.stream.Stream;
 
 import accord.api.Key;
+import accord.primitives.Keys;
 import accord.primitives.Timestamp;
 import accord.primitives.TxnId;
 import org.slf4j.Logger;
@@ -28,8 +29,6 @@ import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
 
-import static accord.utils.Utils.*;
-
 public abstract class CommandsForKey implements CommandListener
 {
     private static final Logger logger = LoggerFactory.getLogger(CommandsForKey.class);
@@ -103,7 +102,7 @@ public abstract class CommandsForKey implements CommandListener
     @Override
     public PreLoadContext listenerPreLoadContext(TxnId caller)
     {
-        return PreLoadContext.contextFor(caller, listOf(key()));
+        return PreLoadContext.contextFor(caller, Keys.of(key()));
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/local/Node.java b/accord-core/src/main/java/accord/local/Node.java
index a24f1a4..4e52b67 100644
--- a/accord-core/src/main/java/accord/local/Node.java
+++ b/accord-core/src/main/java/accord/local/Node.java
@@ -48,12 +48,13 @@ import accord.messages.Callback;
 import accord.messages.ReplyContext;
 import accord.messages.Request;
 import accord.messages.Reply;
+import accord.coordinate.RecoverWithRoute;
 import accord.topology.Shard;
 import accord.topology.Topology;
 import accord.topology.TopologyManager;
+import net.nicoulaj.compilecommand.annotations.Inline;
 import org.apache.cassandra.utils.concurrent.AsyncFuture;
 import org.apache.cassandra.utils.concurrent.Future;
-import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 
 public class Node implements ConfigurationService.Listener, NodeTimeService
 {
@@ -200,6 +201,7 @@ public class Node implements ConfigurationService.Listener, NodeTimeService
         }
     }
 
+    @Inline
     public <T> Future<T> withEpoch(long epoch, Supplier<Future<T>> supplier)
     {
         if (topology.hasEpoch(epoch))
@@ -252,14 +254,14 @@ public class Node implements ConfigurationService.Listener, NodeTimeService
         return nowSupplier.getAsLong();
     }
 
-    public Future<Void> forEachLocal(PreLoadContext context, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, Consumer<SafeCommandStore> forEach)
+    public Future<Void> forEachLocal(PreLoadContext context, Unseekables<?, ?> unseekables, long minEpoch, long maxEpoch, Consumer<SafeCommandStore> forEach)
     {
-        return commandStores.forEach(context, keys, minEpoch, maxEpoch, forEach);
+        return commandStores.forEach(context, unseekables, minEpoch, maxEpoch, forEach);
     }
 
-    public Future<Void> forEachLocalSince(PreLoadContext context, AbstractKeys<?, ?> keys, Timestamp since, Consumer<SafeCommandStore> forEach)
+    public Future<Void> forEachLocalSince(PreLoadContext context, Unseekables<?, ?> unseekables, Timestamp since, Consumer<SafeCommandStore> forEach)
     {
-        return commandStores.forEach(context, keys, since.epoch, Long.MAX_VALUE, forEach);
+        return commandStores.forEach(context, unseekables, since.epoch, Long.MAX_VALUE, forEach);
     }
 
     public Future<Void> ifLocal(PreLoadContext context, RoutingKey key, long epoch, Consumer<SafeCommandStore> ifLocal)
@@ -287,7 +289,7 @@ public class Node implements ConfigurationService.Listener, NodeTimeService
         commandStores.mapReduceConsume(context, key, minEpoch, maxEpoch, mapReduceConsume);
     }
 
-    public <T> void mapReduceConsumeLocal(PreLoadContext context, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<SafeCommandStore, T> mapReduceConsume)
+    public <T> void mapReduceConsumeLocal(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<SafeCommandStore, T> mapReduceConsume)
     {
         commandStores.mapReduceConsume(context, keys, minEpoch, maxEpoch, mapReduceConsume);
     }
@@ -383,60 +385,65 @@ public class Node implements ConfigurationService.Listener, NodeTimeService
         return Coordinate.coordinate(this, txnId, txn, computeRoute(txnId, txn.keys()));
     }
 
-    public Route computeRoute(TxnId txnId, Keys keys)
+    public FullRoute<?> computeRoute(TxnId txnId, Seekables<?, ?> keysOrRanges)
     {
-        RoutingKey homeKey = trySelectHomeKey(txnId, keys);
+        RoutingKey homeKey = trySelectHomeKey(txnId, keysOrRanges);
         if (homeKey == null)
             homeKey = selectRandomHomeKey(txnId);
 
-        return keys.toRoute(homeKey);
+        return keysOrRanges.toRoute(homeKey);
     }
 
-    private @Nullable RoutingKey trySelectHomeKey(TxnId txnId, Keys keys)
+    private @Nullable RoutingKey trySelectHomeKey(TxnId txnId, Seekables<?, ?> keysOrRanges)
     {
-        int i = topology().localForEpoch(txnId.epoch).ranges().findFirstKey(keys);
-        return i >= 0 ? keys.get(i).toRoutingKey() : null;
+        int i = (int)keysOrRanges.findNextIntersection(0, topology().localForEpoch(txnId.epoch).ranges(), 0);
+        return i >= 0 ? keysOrRanges.get(i).someIntersectingRoutingKey() : null;
     }
 
-    public RoutingKey selectProgressKey(long epoch, AbstractKeys<?, ?> keys, RoutingKey homeKey)
+    public RoutingKey selectProgressKey(TxnId txnId, Route<?> route, RoutingKey homeKey)
     {
-        RoutingKey progressKey = trySelectProgressKey(epoch, keys, homeKey);
+        return selectProgressKey(txnId.epoch, route, homeKey);
+    }
+
+    public RoutingKey selectProgressKey(long epoch, Route<?> route, RoutingKey homeKey)
+    {
+        RoutingKey progressKey = trySelectProgressKey(epoch, route, homeKey);
         if (progressKey == null)
             throw new IllegalStateException();
         return progressKey;
     }
 
-    public RoutingKey trySelectProgressKey(TxnId txnId, AbstractRoute route)
+    public RoutingKey trySelectProgressKey(TxnId txnId, Route<?> route)
     {
-        return trySelectProgressKey(txnId, route, route.homeKey);
+        return trySelectProgressKey(txnId, route, route.homeKey());
     }
 
-    public RoutingKey trySelectProgressKey(TxnId txnId, AbstractKeys<?, ?> keys, RoutingKey homeKey)
+    public RoutingKey trySelectProgressKey(TxnId txnId, Route<?> route, RoutingKey homeKey)
     {
-        return trySelectProgressKey(txnId.epoch, keys, homeKey);
+        return trySelectProgressKey(txnId.epoch, route, homeKey);
     }
 
-    public RoutingKey trySelectProgressKey(long epoch, AbstractKeys<?, ?> keys, RoutingKey homeKey)
+    public RoutingKey trySelectProgressKey(long epoch, Route<?> route, RoutingKey homeKey)
     {
-        return trySelectProgressKey(this.topology.localForEpoch(epoch), keys, homeKey);
+        return trySelectProgressKey(this.topology.localForEpoch(epoch), route, homeKey);
     }
 
-    private static RoutingKey trySelectProgressKey(Topology topology, AbstractKeys<?, ?> keys, RoutingKey homeKey)
+    private static RoutingKey trySelectProgressKey(Topology topology, Route<?> route, RoutingKey homeKey)
     {
         if (topology.ranges().contains(homeKey))
             return homeKey;
 
-        int i = topology.ranges().findFirstKey(keys);
+        int i = (int)route.findNextIntersection(0, topology.ranges(), 0);
         if (i < 0)
             return null;
-        return keys.get(i).toRoutingKey();
+        return route.get(i).someIntersectingRoutingKey();
     }
 
     public RoutingKey selectRandomHomeKey(TxnId txnId)
     {
-        KeyRanges ranges = topology().localForEpoch(txnId.epoch).ranges();
-        KeyRange range = ranges.get(ranges.size() == 1 ? 0 : random.nextInt(ranges.size()));
-        return range.endInclusive() ? range.end() : range.start();
+        Ranges ranges = topology().localForEpoch(txnId.epoch).ranges();
+        Range range = ranges.get(ranges.size() == 1 ? 0 : random.nextInt(ranges.size()));
+        return range.someIntersectingRoutingKey();
     }
 
     static class RecoverFuture<T> extends AsyncFuture<T> implements BiConsumer<T, Throwable>
@@ -449,7 +456,7 @@ public class Node implements ConfigurationService.Listener, NodeTimeService
         }
     }
 
-    public Future<? extends Outcome> recover(TxnId txnId, Route route)
+    public Future<? extends Outcome> recover(TxnId txnId, FullRoute<?> route)
     {
         {
             Future<? extends Outcome> result = coordinating.get(txnId);
@@ -471,7 +478,7 @@ public class Node implements ConfigurationService.Listener, NodeTimeService
     }
 
     // TODO: coalesce other maybeRecover calls also? perhaps have mutable knownStatuses so we can inject newer ones?
-    public Future<? extends Outcome> maybeRecover(TxnId txnId, RoutingKey homeKey, @Nullable AbstractRoute route, ProgressToken prevProgress)
+    public Future<? extends Outcome> maybeRecover(TxnId txnId, RoutingKey homeKey, @Nullable Route<?> route, ProgressToken prevProgress)
     {
         Future<? extends Outcome> result = coordinating.get(txnId);
         if (result != null)
diff --git a/accord-core/src/main/java/accord/local/PreLoadContext.java b/accord-core/src/main/java/accord/local/PreLoadContext.java
index 85e4ae1..0c5b5ab 100644
--- a/accord-core/src/main/java/accord/local/PreLoadContext.java
+++ b/accord-core/src/main/java/accord/local/PreLoadContext.java
@@ -19,7 +19,8 @@
 package accord.local;
 
 import accord.api.Key;
-import accord.api.RoutingKey;
+import accord.primitives.Keys;
+import accord.primitives.Seekables;
 import accord.primitives.TxnId;
 
 import java.util.Collections;
@@ -38,9 +39,9 @@ public interface PreLoadContext
     /**
      * @return keys of the {@link CommandsForKey} objects that need to be loaded into memory before this operation is run
      */
-    Iterable<Key> keys();
+    Seekables<?, ?> keys();
 
-    static PreLoadContext contextFor(Iterable<TxnId> txnIds, Iterable<Key> keys)
+    static PreLoadContext contextFor(Iterable<TxnId> txnIds, Seekables<?, ?> keys)
     {
         return new PreLoadContext()
         {
@@ -48,27 +49,37 @@ public interface PreLoadContext
             public Iterable<TxnId> txnIds() { return txnIds; }
 
             @Override
-            public Iterable<Key> keys() { return keys; }
+            public Seekables<?, ?> keys() { return keys; }
         };
     }
 
-    static PreLoadContext contextFor(TxnId txnId, Iterable<Key> keys)
+    static PreLoadContext contextFor(TxnId txnId, Seekables<?, ?> keysOrRanges)
     {
-        return contextFor(Collections.singleton(txnId), keys);
+        switch (keysOrRanges.kindOfContents())
+        {
+            default: throw new AssertionError();
+            case Range: return contextFor(txnId); // TODO (soon): this won't work for actual range queries
+            case Key: return contextFor(Collections.singleton(txnId), keysOrRanges);
+        }
     }
 
     static PreLoadContext contextFor(TxnId txnId)
     {
-        return contextFor(Collections.singleton(txnId), Collections.emptyList());
+        return contextFor(Collections.singleton(txnId), Keys.EMPTY);
+    }
+
+    static PreLoadContext contextFor(Iterable<TxnId> txnIds)
+    {
+        return contextFor(txnIds, Keys.EMPTY);
     }
 
     static PreLoadContext contextFor(Key key)
     {
-        return contextFor(Collections.emptyList(), Collections.singleton(key));
+        return contextFor(Collections.emptyList(), Keys.of(key));
     }
 
     static PreLoadContext empty()
     {
-        return contextFor(Collections.emptyList(), Collections.emptyList());
+        return contextFor(Collections.emptyList(), Keys.EMPTY);
     }
 }
diff --git a/accord-core/src/main/java/accord/local/SafeCommandStore.java b/accord-core/src/main/java/accord/local/SafeCommandStore.java
index 99df4a9..649837a 100644
--- a/accord-core/src/main/java/accord/local/SafeCommandStore.java
+++ b/accord-core/src/main/java/accord/local/SafeCommandStore.java
@@ -22,11 +22,10 @@ import accord.api.Agent;
 import accord.api.DataStore;
 import accord.api.Key;
 import accord.api.ProgressLog;
-import accord.primitives.Keys;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
+import accord.primitives.*;
 import org.apache.cassandra.utils.concurrent.Future;
 
+import java.util.function.BinaryOperator;
 import java.util.function.Consumer;
 import java.util.function.Function;
 
@@ -58,14 +57,19 @@ public interface SafeCommandStore
      */
     void addAndInvokeListener(TxnId txnId, CommandListener listener);
 
-    NodeTimeService time();
+    <T> T mapReduce(Routables<?, ?> keys, Ranges slice, Function<CommandsForKey, T> map, BinaryOperator<T> reduce, T initialValue);
+    void forEach(Routables<?, ?> keys, Ranges slice, Consumer<CommandsForKey> forEach);
+    void forEach(Routable keyOrRange, Ranges slice, Consumer<CommandsForKey> forEach);
+
+
     CommandStore commandStore();
     DataStore dataStore();
     Agent agent();
     ProgressLog progressLog();
+    NodeTimeService time();
     CommandStore.RangesForEpoch ranges();
     long latestEpoch();
-    Timestamp preaccept(TxnId txnId, Keys keys);
+    Timestamp preaccept(TxnId txnId, Seekables<?, ?> keys);
 
     Future<Void> execute(PreLoadContext context, Consumer<? super SafeCommandStore> consumer);
     <T> Future<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> function);
diff --git a/accord-core/src/main/java/accord/local/Status.java b/accord-core/src/main/java/accord/local/Status.java
index db1229e..ad60e00 100644
--- a/accord-core/src/main/java/accord/local/Status.java
+++ b/accord-core/src/main/java/accord/local/Status.java
@@ -20,6 +20,9 @@ package accord.local;
 
 import accord.messages.BeginRecovery;
 import accord.primitives.Ballot;
+import accord.primitives.Ranges;
+import accord.primitives.Seekables;
+import accord.primitives.TxnId;
 import com.google.common.base.Preconditions;
 
 import java.util.List;
@@ -51,8 +54,8 @@ public enum Status
      * So, for execution of other transactions we may treat a PreCommitted transaction as Committed,
      * using the timestamp to update our dependency set to rule it out as a dependency.
      * But we do not have enough information to execute the transaction, and when recovery calculates
-     * {@link BeginRecovery#acceptedStartedBeforeAndDidNotWitness}, {@link BeginRecovery#committedExecutesAfterAndDidNotWitness}
-     * and {@link BeginRecovery#committedStartedBeforeAndDidWitness} we may not have the dependencies
+     * {@link BeginRecovery#acceptedStartedBeforeWithoutWitnessing}, {@link BeginRecovery#committedExecutesAfterWithoutWitnessing}
+     * and {@link BeginRecovery#committedStartedBeforeAndWitnessed} we may not have the dependencies
      * to calculate the result. For these operations we treat ourselves as whatever Accepted status
      * we may have previously taken, using any proposed dependencies to compute the result.
      *
diff --git a/accord-core/src/main/java/accord/local/SyncCommandStores.java b/accord-core/src/main/java/accord/local/SyncCommandStores.java
index 4e2d0fd..61b5de7 100644
--- a/accord-core/src/main/java/accord/local/SyncCommandStores.java
+++ b/accord-core/src/main/java/accord/local/SyncCommandStores.java
@@ -3,7 +3,7 @@ package accord.local;
 import accord.api.Agent;
 import accord.api.DataStore;
 import accord.api.ProgressLog;
-import accord.primitives.AbstractKeys;
+import accord.primitives.Routables;
 import accord.utils.MapReduce;
 import accord.utils.MapReduceConsume;
 
@@ -69,7 +69,7 @@ public class SyncCommandStores extends CommandStores<SyncCommandStores.SyncComma
     }
 
     @Override
-    public <O> void mapReduceConsume(PreLoadContext context, AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
+    public <O> void mapReduceConsume(PreLoadContext context, Routables<?, ?> keys, long minEpoch, long maxEpoch, MapReduceConsume<? super SafeCommandStore, O> mapReduceConsume)
     {
         try
         {
diff --git a/accord-core/src/main/java/accord/messages/AbstractEpochRequest.java b/accord-core/src/main/java/accord/messages/AbstractEpochRequest.java
index df67f10..4cc7538 100644
--- a/accord-core/src/main/java/accord/messages/AbstractEpochRequest.java
+++ b/accord-core/src/main/java/accord/messages/AbstractEpochRequest.java
@@ -1,15 +1,16 @@
 package accord.messages;
 
-import accord.api.Key;
 import accord.local.Node;
 import accord.local.PreLoadContext;
 import accord.local.SafeCommandStore;
+import accord.primitives.Keys;
+import accord.primitives.Seekables;
 import accord.primitives.TxnId;
 import accord.utils.MapReduceConsume;
 
 import java.util.Collections;
 
-public abstract class AbstractEpochRequest<R extends Reply> implements PreLoadContext, EpochRequest, MapReduceConsume<SafeCommandStore, R>
+public abstract class AbstractEpochRequest<R extends Reply> implements PreLoadContext, Request, MapReduceConsume<SafeCommandStore, R>
 {
     public final TxnId txnId;
     protected transient Node node;
@@ -51,8 +52,8 @@ public abstract class AbstractEpochRequest<R extends Reply> implements PreLoadCo
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
-        return Collections.emptyList();
+        return Keys.EMPTY;
     }
 }
diff --git a/accord-core/src/main/java/accord/messages/Accept.java b/accord-core/src/main/java/accord/messages/Accept.java
index e60675e..59dcc10 100644
--- a/accord-core/src/main/java/accord/messages/Accept.java
+++ b/accord-core/src/main/java/accord/messages/Accept.java
@@ -18,7 +18,6 @@
 
 package accord.messages;
 
-import accord.api.Key;
 import accord.local.SafeCommandStore;
 import accord.primitives.*;
 import accord.local.Node.Id;
@@ -27,7 +26,7 @@ import accord.topology.Topologies;
 import accord.api.RoutingKey;
 import accord.local.Command.AcceptOutcome;
 import accord.primitives.PartialDeps;
-import accord.primitives.Route;
+import accord.primitives.FullRoute;
 import accord.primitives.Txn;
 import accord.primitives.Ballot;
 import accord.local.Command;
@@ -47,7 +46,7 @@ public class Accept extends TxnRequest.WithUnsynced<Accept.AcceptReply>
 {
     public static class SerializerSupport
     {
-        public static Accept create(TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, Ballot ballot, Timestamp executeAt, Keys keys, PartialDeps partialDeps, Txn.Kind kind)
+        public static Accept create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, Ballot ballot, Timestamp executeAt, Seekables<?, ?> keys, PartialDeps partialDeps, Txn.Kind kind)
         {
             return new Accept(txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey, ballot, executeAt, keys, partialDeps, kind);
         }
@@ -55,21 +54,21 @@ public class Accept extends TxnRequest.WithUnsynced<Accept.AcceptReply>
 
     public final Ballot ballot;
     public final Timestamp executeAt;
-    public final Keys keys;
+    public final Seekables<?, ?> keys;
     public final PartialDeps partialDeps;
     public final Txn.Kind kind;
 
-    public Accept(Id to, Topologies topologies, Ballot ballot, TxnId txnId, Route route, Timestamp executeAt, Keys keys, Deps deps, Txn.Kind kind)
+    public Accept(Id to, Topologies topologies, Ballot ballot, TxnId txnId, FullRoute<?> route, Timestamp executeAt, Seekables<?, ?> keys, Deps deps, Txn.Kind kind)
     {
         super(to, topologies, txnId, route);
         this.ballot = ballot;
         this.executeAt = executeAt;
-        this.keys = keys.slice(scope.covering);
-        this.partialDeps = deps.slice(scope.covering);
+        this.keys = keys.slice(scope.covering());
+        this.partialDeps = deps.slice(scope.covering());
         this.kind = kind;
     }
 
-    private Accept(TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, Ballot ballot, Timestamp executeAt, Keys keys, PartialDeps partialDeps, Txn.Kind kind)
+    private Accept(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, Ballot ballot, Timestamp executeAt, Seekables<?, ?> keys, PartialDeps partialDeps, Txn.Kind kind)
     {
         super(txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey);
         this.ballot = ballot;
@@ -126,7 +125,7 @@ public class Accept extends TxnRequest.WithUnsynced<Accept.AcceptReply>
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
         return keys;
     }
diff --git a/accord-core/src/main/java/accord/messages/Apply.java b/accord-core/src/main/java/accord/messages/Apply.java
index 7305a20..c1709ba 100644
--- a/accord-core/src/main/java/accord/messages/Apply.java
+++ b/accord-core/src/main/java/accord/messages/Apply.java
@@ -18,21 +18,18 @@
 
 package accord.messages;
 
-import accord.api.Key;
-import accord.local.CommandStore;
-import accord.local.PreLoadContext;
 import accord.local.SafeCommandStore;
 import accord.primitives.*;
 import accord.local.Command;
 import accord.local.Node.Id;
 import accord.api.Result;
 import accord.topology.Topologies;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
 
 import java.util.Collections;
 import accord.messages.Apply.ApplyReply;
 
+import static accord.local.PreLoadContext.empty;
 import static accord.messages.MessageType.APPLY_REQ;
 import static accord.messages.MessageType.APPLY_RSP;
 
@@ -40,7 +37,7 @@ public class Apply extends TxnRequest<ApplyReply>
 {
     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, Timestamp executeAt, PartialDeps deps, Writes writes, Result result)
         {
             return new Apply(txnId, scope, waitForEpoch, untilEpoch, executeAt, deps, writes, result);
         }
@@ -52,19 +49,19 @@ public class Apply extends TxnRequest<ApplyReply>
     public final Writes writes;
     public final Result result;
 
-    public Apply(Id to, Topologies sendTo, Topologies applyTo, long untilEpoch, TxnId txnId, AbstractRoute route, Timestamp executeAt, Deps deps, Writes writes, Result result)
+    public Apply(Id to, Topologies sendTo, Topologies applyTo, long untilEpoch, TxnId txnId, Route<?> route, Timestamp executeAt, Deps deps, Writes writes, Result result)
     {
         super(to, sendTo, route, txnId);
         this.untilEpoch = untilEpoch;
         // TODO: we shouldn't send deps unless we need to (but need to implement fetching them if they're not present)
-        KeyRanges slice = applyTo == sendTo ? scope.covering : applyTo.computeRangesForNode(to);
+        Ranges slice = applyTo == sendTo ? scope.covering() : applyTo.computeRangesForNode(to);
         this.deps = deps.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, Timestamp executeAt, PartialDeps deps, Writes writes, Result result)
     {
         super(txnId, route, waitForEpoch);
         this.untilEpoch = untilEpoch;
@@ -105,7 +102,7 @@ public class Apply extends TxnRequest<ApplyReply>
     {
         if (reply == ApplyReply.Applied)
         {
-            node.ifLocal(PreLoadContext.empty(), scope.homeKey, txnId.epoch, instance -> {
+            node.ifLocal(empty(), scope.homeKey(), txnId.epoch, instance -> {
                 node.withEpoch(executeAt.epoch, () -> instance.progressLog().durableLocal(txnId));
             }).addCallback(node.agent());
         }
@@ -119,9 +116,9 @@ public class Apply extends TxnRequest<ApplyReply>
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
-        return Collections.emptyList();
+        return Keys.EMPTY;
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/messages/BeginInvalidation.java b/accord-core/src/main/java/accord/messages/BeginInvalidation.java
index 9a1e653..d4face3 100644
--- a/accord-core/src/main/java/accord/messages/BeginInvalidation.java
+++ b/accord-core/src/main/java/accord/messages/BeginInvalidation.java
@@ -1,6 +1,5 @@
 package accord.messages;
 
-import accord.api.Key;
 import accord.api.RoutingKey;
 import accord.local.*;
 import accord.local.Node.Id;
@@ -11,30 +10,32 @@ import javax.annotation.Nullable;
 import java.util.Collections;
 import java.util.List;
 
+import static accord.primitives.Route.castToFullRoute;
+import static accord.primitives.Route.isFullRoute;
 import static accord.utils.Functions.mapReduceNonNull;
 
-public class BeginInvalidation extends AbstractEpochRequest<BeginInvalidation.InvalidateReply> implements EpochRequest, PreLoadContext
+public class BeginInvalidation extends AbstractEpochRequest<BeginInvalidation.InvalidateReply> implements Request, PreLoadContext
 {
     public final Ballot ballot;
-    public final RoutingKeys someKeys;
+    public final Unseekables<?, ?> someUnseekables;
 
-    public BeginInvalidation(Id to, Topologies topologies, TxnId txnId, RoutingKeys someKeys, Ballot ballot)
+    public BeginInvalidation(Id to, Topologies topologies, TxnId txnId, Unseekables<?, ?> someUnseekables, Ballot ballot)
     {
         super(txnId);
-        this.someKeys = someKeys.slice(topologies.computeRangesForNode(to));
+        this.someUnseekables = someUnseekables.slice(topologies.computeRangesForNode(to));
         this.ballot = ballot;
     }
 
-    public BeginInvalidation(TxnId txnId, RoutingKeys someKeys, Ballot ballot)
+    public BeginInvalidation(TxnId txnId, Unseekables<?, ?> someUnseekables, Ballot ballot)
     {
         super(txnId);
-        this.someKeys = someKeys;
+        this.someUnseekables = someUnseekables;
         this.ballot = ballot;
     }
 
     public void process()
     {
-        node.mapReduceConsumeLocal(this, someKeys, txnId.epoch, txnId.epoch, this);
+        node.mapReduceConsumeLocal(this, someUnseekables, txnId.epoch, txnId.epoch, this);
     }
 
     @Override
@@ -57,7 +58,7 @@ public class BeginInvalidation extends AbstractEpochRequest<BeginInvalidation.In
         boolean isOk = o1.isPromised() && o2.isPromised();
         Ballot supersededBy = isOk ? null : Ballot.nonNullOrMax(o1.supersededBy, o2.supersededBy);
         boolean acceptedFastPath = o1.acceptedFastPath && o2.acceptedFastPath;
-        AbstractRoute route =  AbstractRoute.merge(o1.route, o2.route);
+        Route<?> route =  Route.merge((Route)o1.route, o2.route);
         RoutingKey homeKey = o1.homeKey != null ? o1.homeKey : o2.homeKey != null ? o2.homeKey : null;
         InvalidateReply maxStatus = Status.max(o1, o1.status, o1.accepted, o2, o2.status, o2.accepted);
         return new InvalidateReply(supersededBy, maxStatus.accepted, maxStatus.status, acceptedFastPath, route, homeKey);
@@ -70,9 +71,9 @@ public class BeginInvalidation extends AbstractEpochRequest<BeginInvalidation.In
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
-        return Collections.emptyList();
+        return Keys.EMPTY;
     }
 
     @Override
@@ -102,10 +103,10 @@ public class BeginInvalidation extends AbstractEpochRequest<BeginInvalidation.In
         public final Ballot accepted;
         public final Status status;
         public final boolean acceptedFastPath;
-        public final @Nullable AbstractRoute route;
+        public final @Nullable Route<?> route;
         public final @Nullable RoutingKey homeKey;
 
-        public InvalidateReply(Ballot supersededBy, Ballot accepted, Status status, boolean acceptedFastPath, @Nullable AbstractRoute route, @Nullable RoutingKey homeKey)
+        public InvalidateReply(Ballot supersededBy, Ballot accepted, Status status, boolean acceptedFastPath, @Nullable Route<?> route, @Nullable RoutingKey homeKey)
         {
             this.supersededBy = supersededBy;
             this.accepted = accepted;
@@ -132,19 +133,19 @@ public class BeginInvalidation extends AbstractEpochRequest<BeginInvalidation.In
             return MessageType.BEGIN_INVALIDATE_RSP;
         }
 
-        public static Route findRoute(List<InvalidateReply> invalidateOks)
+        public static FullRoute<?> findRoute(List<InvalidateReply> invalidateOks)
         {
             for (InvalidateReply ok : invalidateOks)
             {
-                if (ok.route instanceof Route)
-                    return (Route)ok.route;
+                if (isFullRoute(ok.route))
+                    return castToFullRoute(ok.route);
             }
             return null;
         }
 
-        public static AbstractRoute mergeRoutes(List<InvalidateReply> invalidateOks)
+        public static Route<?> mergeRoutes(List<InvalidateReply> invalidateOks)
         {
-            return mapReduceNonNull(ok -> ok.route, AbstractRoute::union, invalidateOks);
+            return mapReduceNonNull(ok -> (Route)ok.route, Route::union, invalidateOks);
         }
 
         public static InvalidateReply max(List<InvalidateReply> invalidateReplies)
diff --git a/accord-core/src/main/java/accord/messages/BeginRecovery.java b/accord-core/src/main/java/accord/messages/BeginRecovery.java
index add5fd3..19cb9f1 100644
--- a/accord-core/src/main/java/accord/messages/BeginRecovery.java
+++ b/accord-core/src/main/java/accord/messages/BeginRecovery.java
@@ -18,10 +18,9 @@
 
 package accord.messages;
 
-import accord.api.Key;
-import accord.local.*;
 import accord.api.Result;
 import accord.local.CommandsForKey.TxnIdWithExecuteAt;
+import accord.local.SafeCommandStore;
 import accord.local.Status.Phase;
 import accord.primitives.*;
 import accord.topology.Topologies;
@@ -29,11 +28,14 @@ import accord.topology.Topologies;
 import java.util.List;
 import java.util.stream.Stream;
 
+import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import accord.local.Node.Id;
+import accord.local.Command;
+import accord.local.Status;
 
 import java.util.Collections;
 
@@ -48,7 +50,7 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
 {
     public static class SerializationSupport
     {
-        public static BeginRecovery create(TxnId txnId, PartialRoute scope, long waitForEpoch, PartialTxn partialTxn, Ballot ballot, @Nullable Route route)
+        public static BeginRecovery create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, PartialTxn partialTxn, Ballot ballot, @Nullable FullRoute<?> route)
         {
             return new BeginRecovery(txnId, scope, waitForEpoch, partialTxn, ballot, route);
         }
@@ -56,17 +58,17 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
 
     public final PartialTxn partialTxn;
     public final Ballot ballot;
-    public final @Nullable Route route;
+    public final @Nullable FullRoute<?> route;
 
-    public BeginRecovery(Id to, Topologies topologies, TxnId txnId, Txn txn, Route route, Ballot ballot)
+    public BeginRecovery(Id to, Topologies topologies, TxnId txnId, Txn txn, FullRoute<?> route, Ballot ballot)
     {
         super(to, topologies, route, txnId);
-        this.partialTxn = txn.slice(scope.covering, scope.contains(scope.homeKey));
+        this.partialTxn = txn.slice(scope.covering(), scope.contains(scope.homeKey()));
         this.ballot = ballot;
-        this.route = scope.contains(scope.homeKey) ? route : null;
+        this.route = scope.contains(scope.homeKey()) ? route : null;
     }
 
-    private BeginRecovery(TxnId txnId, PartialRoute scope, long waitForEpoch, PartialTxn partialTxn, Ballot ballot, @Nullable Route route)
+    private BeginRecovery(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, PartialTxn partialTxn, Ballot ballot, @Nullable FullRoute<?> route)
     {
         super(txnId, scope, waitForEpoch);
         this.partialTxn = partialTxn;
@@ -116,16 +118,17 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
         }
         else
         {
-            rejectsFastPath = acceptedStartedAfterWithoutWitnessing(safeStore, txnId, partialTxn.keys()).anyMatch(ignore -> true);
+            Ranges ranges = safeStore.ranges().at(txnId.epoch);
+            rejectsFastPath = acceptedStartedAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys()).anyMatch(ignore -> true);
             if (!rejectsFastPath)
-                rejectsFastPath = committedExecutesAfterAndDidNotWitness(safeStore, txnId, partialTxn.keys()).anyMatch(ignore -> true);
+                rejectsFastPath = committedExecutesAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys()).anyMatch(ignore -> true);
 
             // TODO: introduce some good unit tests for verifying these two functions in a real repair scenario
             // committed txns with an earlier txnid and have our txnid as a dependency
-            earlierCommittedWitness = committedStartedBeforeAndDidWitness(safeStore, txnId, partialTxn.keys());
+            earlierCommittedWitness = committedStartedBeforeAndWitnessed(safeStore, txnId, ranges, partialTxn.keys());
 
             // accepted txns with an earlier txnid that don't have our txnid as a dependency
-            earlierAcceptedNoWitness = acceptedStartedBeforeAndDidNotWitness(safeStore, txnId, partialTxn.keys());
+            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());
     }
@@ -176,7 +179,7 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
         return partialTxn.keys();
     }
@@ -197,18 +200,18 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
                '}';
     }
 
-    public interface RecoverReply extends Reply
+    public static abstract class RecoverReply implements Reply
     {
         @Override
-        default MessageType type()
+        public MessageType type()
         {
             return MessageType.BEGIN_RECOVER_RSP;
         }
 
-        boolean isOk();
+        public abstract boolean isOk();
     }
 
-    public static class RecoverOk implements RecoverReply
+    public static class RecoverOk extends RecoverReply
     {
         public final TxnId txnId; // TODO for debugging?
         public final Status status;
@@ -221,14 +224,13 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
         public final Writes writes;
         public final Result result;
 
-        public RecoverOk(TxnId txnId, Status status, Ballot accepted, Timestamp executeAt, PartialDeps deps, Deps earlierCommittedWitness, Deps earlierAcceptedNoWitness, boolean rejectsFastPath, Writes writes, Result result)
+        public RecoverOk(TxnId txnId, Status status, Ballot accepted, Timestamp executeAt, @Nonnull PartialDeps deps, Deps earlierCommittedWitness, Deps earlierAcceptedNoWitness, boolean rejectsFastPath, Writes writes, Result result)
         {
-            Preconditions.checkNotNull(deps);
             this.txnId = txnId;
             this.accepted = accepted;
             this.executeAt = executeAt;
             this.status = status;
-            this.deps = deps;
+            this.deps = Invariants.nonNull(deps);
             this.earlierCommittedWitness = earlierCommittedWitness;
             this.earlierAcceptedNoWitness = earlierAcceptedNoWitness;
             this.rejectsFastPath = rejectsFastPath;
@@ -270,7 +272,7 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
         }
     }
 
-    public static class RecoverNack implements RecoverReply
+    public static class RecoverNack extends RecoverReply
     {
         public final Ballot supersededBy;
         public RecoverNack(Ballot supersededBy)
@@ -293,17 +295,12 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
         }
     }
 
-    private static Deps acceptedStartedBeforeAndDidNotWitness(SafeCommandStore commandStore, TxnId txnId, Keys keys)
+    private static Deps acceptedStartedBeforeWithoutWitnessing(SafeCommandStore commandStore, TxnId txnId, Ranges ranges, Seekables<?, ?> keys)
     {
-        try (Deps.OrderedBuilder builder = Deps.orderedBuilder(true);)
+        try (Deps.OrderedBuilder builder = Deps.orderedBuilder(true))
         {
-            keys.forEach(key -> {
-                CommandsForKey forKey = commandStore.maybeCommandsForKey(key);
-                if (forKey == null)
-                    return;
-
+            commandStore.forEach(keys, ranges, forKey -> {
                 // accepted txns with an earlier txnid that do not have our txnid as a dependency
-                builder.nextKey(key);
                 /**
                  * The idea here is to discover those transactions that have been Accepted without witnessing us
                  * and whom may not have adopted us as dependencies as responses to the Accept. Once we have
@@ -315,6 +312,7 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
                  * Which is to say, we expect the previously proposed dependencies (if any) to be used to evaluate this
                  * condition.
                  */
+                builder.nextKey(forKey.key());
                 forKey.uncommitted().before(txnId, RorWs, WITHOUT, txnId, HAS_BEEN, Accepted).forEach((command) -> {
                     if (command.executeAt.compareTo(txnId) > 0)
                         builder.add(command.txnId);
@@ -324,20 +322,17 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
         }
     }
 
-    private static Deps committedStartedBeforeAndDidWitness(SafeCommandStore commandStore, TxnId txnId, Keys keys)
+    private static Deps committedStartedBeforeAndWitnessed(SafeCommandStore commandStore, TxnId txnId, Ranges ranges, Seekables<?, ?> keys)
     {
-        try (Deps.OrderedBuilder builder = Deps.orderedBuilder(true);)
+        try (Deps.OrderedBuilder builder = Deps.orderedBuilder(true))
         {
-            keys.forEach(key -> {
-                CommandsForKey forKey = commandStore.maybeCommandsForKey(key);
-                if (forKey == null)
-                    return;
+            commandStore.forEach(keys, ranges, forKey -> {
                 /**
                  * The idea here is to discover those transactions that have been Committed and DID witness us
                  * so that we can remove these from the set of acceptedStartedBeforeAndDidNotWitness
                  * on other nodes, to minimise the number of transactions we try to wait for on recovery
                  */
-                builder.nextKey(key);
+                builder.nextKey(forKey.key());
                 forKey.committedById().before(txnId, RorWs, WITH, txnId, ANY_STATUS, null)
                         .forEach(builder::add);
             });
@@ -345,47 +340,30 @@ public class BeginRecovery extends TxnRequest<BeginRecovery.RecoverReply>
         }
     }
 
-    private static Stream<TxnIdWithExecuteAt> acceptedStartedAfterWithoutWitnessing(SafeCommandStore commandStore, TxnId startedAfter, Keys keys)
+    private static Stream<? extends TxnIdWithExecuteAt> acceptedStartedAfterWithoutWitnessing(SafeCommandStore commandStore, TxnId startedAfter, Ranges ranges, Seekables<?, ?> keys)
     {
-        return keys.stream().flatMap(key -> {
-            CommandsForKey forKey = commandStore.maybeCommandsForKey(key);
-            if (forKey == null)
-                return Stream.of();
-
-            /**
-             * The idea here is to discover those transactions that were started after us and have been Accepted
-             * and did not witness us as part of their pre-accept round, as this means that we CANNOT have taken
-             * the fast path. This is central to safe recovery, as if every transaction that executes later has
-             * witnessed us we are safe to propose the pre-accept timestamp regardless, whereas if any transaction
-             * has not witnessed us we can safely invalidate it.
-             */
-
-            // AcceptedInvalidate, if successful, cannot take a dependency on us as it is a no-op; if another
-            // Accepted is successful then its status here is irrelevant, as it will either be informed by other
-            // replicas, or else will have to perform a new round that will adopt us as a dependency
-            return forKey.uncommitted().after(startedAfter, RorWs, WITHOUT, startedAfter, HAS_BEEN, Accepted);
-        });
+        /**
+         * The idea here is to discover those transactions that were started after us and have been Accepted
+         * and did not witness us as part of their pre-accept round, as this means that we CANNOT have taken
+         * the fast path. This is central to safe recovery, as if every transaction that executes later has
+         * witnessed us we are safe to propose the pre-accept timestamp regardless, whereas if any transaction
+         * has not witnessed us we can safely invalidate it.
+         */
+        return commandStore.mapReduce(keys, ranges, forKey ->
+            forKey.uncommitted().after(startedAfter, RorWs, WITHOUT, startedAfter, HAS_BEEN, Accepted)
+        , Stream::concat, Stream.empty());
     }
 
-    private static Stream<TxnId> committedExecutesAfterAndDidNotWitness(SafeCommandStore commandStore, TxnId startedAfter, Keys keys)
+    private static Stream<TxnId> committedExecutesAfterWithoutWitnessing(SafeCommandStore commandStore, TxnId startedAfter, Ranges ranges, Seekables<?, ?> keys)
     {
-        return keys.stream().flatMap(key -> {
-            CommandsForKey forKey = commandStore.maybeCommandsForKey(key);
-            if (forKey == null)
-                return Stream.of();
-
-            /**
-             * The idea here is to discover those transactions that have been decided to execute after us
-             * and did not witness us as part of their pre-accept or accept round, as this means that we CANNOT have
-             * taken the fast path. This is central to safe recovery, as if every transaction that executes later has
-             * witnessed us we are safe to propose the pre-accept timestamp regardless, whereas if any transaction
-             * has not witnessed us we can safely invalidate it.
-             */
-
-            // We declare HAS_BEEN Committed here only for clarity, as it is currently redundant, but if in future
-            // we revisit the statuses we store it is worth making clear that we require the full decided dependencies
-            // here in order for this evaluation to be valid.
-            return forKey.committedByExecuteAt().after(startedAfter, RorWs, WITHOUT, startedAfter, HAS_BEEN, Committed);
-        });
+        /**
+         * The idea here is to discover those transactions that have been decided to execute after us
+         * and did not witness us as part of their pre-accept or accept round, as this means that we CANNOT have
+         * taken the fast path. This is central to safe recovery, as if every transaction that executes later has
+         * witnessed us we are safe to propose the pre-accept timestamp regardless, whereas if any transaction
+         * has not witnessed us we can safely invalidate it.
+         */
+        return commandStore.mapReduce(keys, ranges, forKey -> forKey.committedByExecuteAt().after(startedAfter, RorWs, WITHOUT, startedAfter, ANY_STATUS, null),
+                Stream::concat, Stream.empty());
     }
 }
diff --git a/accord-core/src/main/java/accord/messages/CheckStatus.java b/accord-core/src/main/java/accord/messages/CheckStatus.java
index 40b3a27..d3242dd 100644
--- a/accord-core/src/main/java/accord/messages/CheckStatus.java
+++ b/accord-core/src/main/java/accord/messages/CheckStatus.java
@@ -20,7 +20,6 @@ package accord.messages;
 
 import javax.annotation.Nullable;
 
-import accord.api.Key;
 import accord.api.Result;
 import accord.api.RoutingKey;
 import accord.local.*;
@@ -40,19 +39,19 @@ import static accord.local.Status.Durability.NotDurable;
 import static accord.messages.TxnRequest.computeScope;
 
 public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
-        implements EpochRequest, PreLoadContext, MapReduceConsume<SafeCommandStore, CheckStatus.CheckStatusOk>
+        implements Request, PreLoadContext, MapReduceConsume<SafeCommandStore, CheckStatus.CheckStatusOk>
 {
     public static class SerializationSupport
     {
         public static CheckStatusOk createOk(SaveStatus status, Ballot promised, Ballot accepted, @Nullable Timestamp executeAt,
                                              boolean isCoordinating, Durability durability,
-                                             @Nullable AbstractRoute route, @Nullable RoutingKey homeKey)
+                                             @Nullable Route<?> route, @Nullable RoutingKey homeKey)
         {
             return new CheckStatusOk(status, promised, accepted, executeAt, isCoordinating, durability, route, homeKey);
         }
         public static CheckStatusOk createOk(SaveStatus status, Ballot promised, Ballot accepted, @Nullable Timestamp executeAt,
                                              boolean isCoordinating, Durability durability,
-                                             @Nullable AbstractRoute route, @Nullable RoutingKey homeKey,
+                                             @Nullable Route<?> route, @Nullable RoutingKey homeKey,
                                              PartialTxn partialTxn, PartialDeps committedDeps, Writes writes, Result result)
         {
             return new CheckStatusOkFull(status, promised, accepted, executeAt, isCoordinating, durability, route, homeKey,
@@ -66,15 +65,15 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
         No, Route, All
     }
 
-    public final RoutingKeys someKeys;
+    public final Unseekables<?, ?> query;
     public final long startEpoch;
     public final long endEpoch;
     public final IncludeInfo includeInfo;
 
-    public CheckStatus(TxnId txnId, RoutingKeys someKeys, long startEpoch, long endEpoch, IncludeInfo includeInfo)
+    public CheckStatus(TxnId txnId, Unseekables<?, ?> query, long startEpoch, long endEpoch, IncludeInfo includeInfo)
     {
         super(txnId);
-        this.someKeys = someKeys;
+        this.query = query;
         this.startEpoch = startEpoch;
         this.endEpoch = endEpoch;
         this.includeInfo = includeInfo;
@@ -87,19 +86,15 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
-        return Collections.emptyList();
+        return Keys.EMPTY;
     }
 
-    public CheckStatus(Id to, Topologies topologies, TxnId txnId, RoutingKeys someKeys, IncludeInfo includeInfo)
+    public CheckStatus(Id to, Topologies topologies, TxnId txnId, Unseekables<?, ?> query, IncludeInfo includeInfo)
     {
         super(txnId);
-        // TODO (now): cleanup use of instanceof to avoid hotspot cache thrashing
-        if (someKeys instanceof AbstractRoute)
-            this.someKeys = computeScope(to, topologies, (AbstractRoute) someKeys, 0, AbstractRoute::sliceStrict, PartialRoute::union);
-        else
-            this.someKeys = computeScope(to, topologies, someKeys, 0, RoutingKeys::slice, RoutingKeys::union);
+        this.query = computeScope(to, topologies, (Unseekables) query, 0, Unseekables::slice, Unseekables::union);
         this.startEpoch = topologies.oldestEpoch();
         this.endEpoch = topologies.currentEpoch();
         this.includeInfo = includeInfo;
@@ -107,7 +102,7 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
 
     public void process()
     {
-        node.mapReduceConsumeLocal(this, someKeys, startEpoch, endEpoch, this);
+        node.mapReduceConsumeLocal(this, query, startEpoch, endEpoch, this);
     }
 
     @Override
@@ -139,12 +134,12 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
         else node.reply(replyTo, replyContext, ok);
     }
 
-    public interface CheckStatusReply extends Reply
+    public static abstract class CheckStatusReply implements Reply
     {
-        boolean isOk();
+        abstract public boolean isOk();
     }
 
-    public static class CheckStatusOk implements CheckStatusReply
+    public static class CheckStatusOk extends CheckStatusReply
     {
         public final SaveStatus saveStatus;
         public final Ballot promised;
@@ -152,7 +147,7 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
         public final @Nullable Timestamp executeAt; // not set if invalidating or invalidated
         public final boolean isCoordinating;
         public final Durability durability; // i.e. on all shards
-        public final @Nullable AbstractRoute route;
+        public final @Nullable Route<?> route;
         public final @Nullable RoutingKey homeKey;
 
         public CheckStatusOk(Node node, Command command)
@@ -162,8 +157,8 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
         }
 
         private CheckStatusOk(SaveStatus saveStatus, Ballot promised, Ballot accepted, @Nullable Timestamp executeAt,
-                              boolean isCoordinating, Durability durability,
-                              @Nullable AbstractRoute route, @Nullable RoutingKey homeKey)
+                      boolean isCoordinating, Durability durability,
+                      @Nullable Route<?> route, @Nullable RoutingKey homeKey)
         {
             this.saveStatus = saveStatus;
             this.promised = promised;
@@ -216,7 +211,7 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
             CheckStatusOk maxPromised = prefer.promised.compareTo(defer.promised) >= 0 ? prefer : defer;
             CheckStatusOk maxDurability = prefer.durability.compareTo(defer.durability) >= 0 ? prefer : defer;
             CheckStatusOk maxHomeKey = prefer.homeKey != null || defer.homeKey == null ? prefer : defer;
-            AbstractRoute mergedRoute = AbstractRoute.merge(prefer.route, defer.route);
+            Route<?> mergedRoute = Route.merge(prefer.route, (Route)defer.route);
 
             // if the maximum (or preferred equal) is the same on all dimensions, return it
             if (mergeStatus == maxStatus.saveStatus && maxStatus == maxPromised && maxStatus == maxDurability
@@ -257,7 +252,7 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
         }
 
         protected CheckStatusOkFull(SaveStatus status, Ballot promised, Ballot accepted, Timestamp executeAt,
-                                  boolean isCoordinating, Durability durability, AbstractRoute route,
+                                  boolean isCoordinating, Durability durability, Route<?> route,
                                   RoutingKey homeKey, PartialTxn partialTxn, PartialDeps committedDeps, Writes writes, Result result)
         {
             super(status, promised, accepted, executeAt, isCoordinating, durability, route, homeKey);
@@ -304,19 +299,19 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
                                          max.homeKey, partialTxn, committedDeps, fullMax.writes, fullMax.result);
         }
 
-        public Known sufficientFor(AbstractKeys<?, ?> forKeys)
+        public Known sufficientFor(Unseekables<?, ?> unseekables)
         {
-            return sufficientFor(forKeys, saveStatus, partialTxn, committedDeps, writes, result);
+            return sufficientFor(unseekables, saveStatus, partialTxn, committedDeps, writes, result);
         }
 
-        private static Known sufficientFor(AbstractKeys<?, ?> forKeys, SaveStatus maxStatus, PartialTxn partialTxn, PartialDeps committedDeps, Writes writes, Result result)
+        private static Known sufficientFor(Unseekables<?, ?> unseekables, SaveStatus maxStatus, PartialTxn partialTxn, PartialDeps committedDeps, Writes writes, Result result)
         {
             Status.Definition definition = maxStatus.known.definition;
             switch (definition)
             {
                 default: throw new AssertionError();
                 case DefinitionKnown:
-                    if (partialTxn != null && partialTxn.covers(forKeys))
+                    if (partialTxn != null && partialTxn.covers(unseekables))
                         break;
                     definition = Definition.DefinitionUnknown;
                 case DefinitionUnknown:
@@ -329,7 +324,7 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
             {
                 default: throw new AssertionError();
                 case DepsKnown:
-                    if (committedDeps != null && committedDeps.covers(forKeys))
+                    if (committedDeps != null && committedDeps.covers(unseekables))
                         break;
                     deps = KnownDeps.DepsUnknown;
                 case NoDeps:
@@ -371,7 +366,7 @@ public class CheckStatus extends AbstractEpochRequest<CheckStatus.CheckStatusOk>
         }
     }
 
-    public static class CheckStatusNack implements CheckStatusReply
+    public static class CheckStatusNack extends CheckStatusReply
     {
         private static final CheckStatusNack instance = new CheckStatusNack();
 
diff --git a/accord-core/src/main/java/accord/messages/Commit.java b/accord-core/src/main/java/accord/messages/Commit.java
index 2db1717..74521d0 100644
--- a/accord-core/src/main/java/accord/messages/Commit.java
+++ b/accord-core/src/main/java/accord/messages/Commit.java
@@ -21,8 +21,8 @@ package accord.messages;
 import java.util.Collections;
 import java.util.Set;
 
-import accord.api.Key;
 import accord.local.*;
+import accord.local.PreLoadContext;
 import accord.messages.ReadData.ReadNack;
 import accord.messages.ReadData.ReadReply;
 import accord.primitives.*;
@@ -30,7 +30,7 @@ import accord.local.Node.Id;
 import accord.topology.Topologies;
 import javax.annotation.Nullable;
 
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import accord.topology.Topology;
 
@@ -42,7 +42,7 @@ public class Commit extends TxnRequest<ReadNack>
 {
     public static class SerializerSupport
     {
-        public static Commit create(TxnId txnId, PartialRoute scope, long waitForEpoch, Timestamp executeAt, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable Route fullRoute, @Nullable ReadData read)
+        public static Commit create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Timestamp executeAt, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData read)
         {
             return new Commit(txnId, scope, waitForEpoch, executeAt, partialTxn, partialDeps, fullRoute, read);
         }
@@ -51,7 +51,7 @@ public class Commit extends TxnRequest<ReadNack>
     public final Timestamp executeAt;
     public final @Nullable PartialTxn partialTxn;
     public final PartialDeps partialDeps;
-    public final @Nullable Route route;
+    public final @Nullable FullRoute<?> route;
     public final ReadData read;
 
     @SuppressWarnings("MismatchedQueryAndUpdateOfCollection")
@@ -61,36 +61,36 @@ public class Commit extends TxnRequest<ReadNack>
 
     // TODO: cleanup passing of topologies here - maybe fetch them afresh from Node? Or perhaps introduce well-named
     //       classes to represent different topology combinations
-    public Commit(Kind kind, Id to, Topology coordinateTopology, Topologies topologies, TxnId txnId, Txn txn, Route route, @Nullable Keys readScope, Timestamp executeAt, Deps deps, boolean read)
+    public Commit(Kind kind, Id to, Topology coordinateTopology, Topologies topologies, TxnId txnId, Txn txn, FullRoute<?> route, @Nullable Seekables<?, ?> readScope, Timestamp executeAt, Deps deps, boolean read)
     {
         super(to, topologies, route, txnId);
 
-        Route sendRoute = null;
+        FullRoute<?> sendRoute = null;
         PartialTxn partialTxn = null;
         if (kind == Kind.Maximal)
         {
-            boolean isHome = coordinateTopology.rangesForNode(to).contains(route.homeKey);
-            partialTxn = txn.slice(scope.covering, isHome);
+            boolean isHome = coordinateTopology.rangesForNode(to).contains(route.homeKey());
+            partialTxn = txn.slice(scope.covering(), isHome);
             if (isHome)
                 sendRoute = route;
         }
         else if (executeAt.epoch != txnId.epoch)
         {
-            KeyRanges coordinateRanges = coordinateTopology.rangesForNode(to);
-            KeyRanges executeRanges = topologies.computeRangesForNode(to);
-            KeyRanges extraRanges = executeRanges.difference(coordinateRanges);
+            Ranges coordinateRanges = coordinateTopology.rangesForNode(to);
+            Ranges executeRanges = topologies.computeRangesForNode(to);
+            Ranges extraRanges = executeRanges.difference(coordinateRanges);
             if (!extraRanges.isEmpty())
-                partialTxn = txn.slice(extraRanges, coordinateRanges.contains(route.homeKey));
+                partialTxn = txn.slice(extraRanges, coordinateRanges.contains(route.homeKey()));
         }
 
         this.executeAt = executeAt;
         this.partialTxn = partialTxn;
-        this.partialDeps = deps.slice(scope.covering);
+        this.partialDeps = deps.slice(scope.covering());
         this.route = sendRoute;
         this.read = read ? new ReadData(to, topologies, txnId, readScope, executeAt) : null;
     }
 
-    Commit(TxnId txnId, PartialRoute scope, long waitForEpoch, Timestamp executeAt, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable Route fullRoute, @Nullable ReadData read)
+    Commit(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Timestamp executeAt, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData read)
     {
         super(txnId, scope, waitForEpoch);
         this.executeAt = executeAt;
@@ -102,7 +102,7 @@ public class Commit extends TxnRequest<ReadNack>
 
     // TODO (soon): accept Topology not Topologies
     // TODO: do not commit if we're already ready to execute (requires extra info in Accept responses)
-    public static void commitMinimalAndRead(Node node, Topologies executeTopologies, TxnId txnId, Txn txn, Route route, Keys readScope, Timestamp executeAt, Deps deps, Set<Id> readSet, Callback<ReadReply> callback)
+    public static void commitMinimalAndRead(Node node, Topologies executeTopologies, TxnId txnId, Txn txn, FullRoute<?> route, Seekables<?, ?> readScope, Timestamp executeAt, Deps deps, Set<Id> readSet, Callback<ReadReply> callback)
     {
         Topologies allTopologies = executeTopologies;
         if (txnId.epoch != executeAt.epoch)
@@ -134,9 +134,9 @@ public class Commit extends TxnRequest<ReadNack>
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
-        return Collections.emptyList();
+        return Keys.EMPTY;
     }
 
     public void process()
@@ -157,7 +157,7 @@ public class Commit extends TxnRequest<ReadNack>
                 return null;
 
             case Insufficient:
-                Preconditions.checkState(!command.hasBeenWitnessed());
+                Invariants.checkState(!command.hasBeenWitnessed());
                 if (defer == null)
                     defer = new Defer(DefinitionOnly, Committed.minKnown, Commit.this);
                 defer.add(command, safeStore.commandStore());
@@ -196,54 +196,54 @@ public class Commit extends TxnRequest<ReadNack>
                '}';
     }
 
-    public static class Invalidate implements EpochRequest, PreLoadContext
+    public static class Invalidate implements Request, PreLoadContext
     {
         public static class SerializerSupport
         {
-            public static Invalidate create(TxnId txnId, RoutingKeys scope, long waitForEpoch, long invalidateUntilEpoch)
+            public static Invalidate create(TxnId txnId, Unseekables<?, ?> scope, long waitForEpoch, long invalidateUntilEpoch)
             {
                 return new Invalidate(txnId, scope, waitForEpoch, invalidateUntilEpoch);
             }
         }
 
-        public static void commitInvalidate(Node node, TxnId txnId, RoutingKeys someKeys, Timestamp until)
+        public static void commitInvalidate(Node node, TxnId txnId, Unseekables<?, ?> inform, Timestamp until)
         {
-            commitInvalidate(node, txnId, someKeys, until.epoch);
+            commitInvalidate(node, txnId, inform, until.epoch);
         }
 
-        public static void commitInvalidate(Node node, TxnId txnId, RoutingKeys someKeys, long untilEpoch)
+        public static void commitInvalidate(Node node, TxnId txnId, Unseekables<?, ?> inform, long untilEpoch)
         {
             // TODO: this kind of check needs to be inserted in all equivalent methods
-            Preconditions.checkState(untilEpoch >= txnId.epoch);
-            Preconditions.checkState(node.topology().hasEpoch(untilEpoch));
-            Topologies commitTo = node.topology().preciseEpochs(someKeys, txnId.epoch, untilEpoch);
-            commitInvalidate(node, commitTo, txnId, someKeys);
+            Invariants.checkState(untilEpoch >= txnId.epoch);
+            Invariants.checkState(node.topology().hasEpoch(untilEpoch));
+            Topologies commitTo = node.topology().preciseEpochs(inform, txnId.epoch, untilEpoch);
+            commitInvalidate(node, commitTo, txnId, inform);
         }
 
-        public static void commitInvalidate(Node node, Topologies commitTo, TxnId txnId, RoutingKeys someKeys)
+        public static void commitInvalidate(Node node, Topologies commitTo, TxnId txnId, Unseekables<?, ?> inform)
         {
             for (Node.Id to : commitTo.nodes())
             {
-                Invalidate send = new Invalidate(to, commitTo, txnId, someKeys);
+                Invalidate send = new Invalidate(to, commitTo, txnId, inform);
                 node.send(to, send);
             }
         }
 
         public final TxnId txnId;
-        public final RoutingKeys scope;
+        public final Unseekables<?, ?> scope;
         public final long waitForEpoch;
         public final long invalidateUntilEpoch;
 
-        Invalidate(Id to, Topologies topologies, TxnId txnId, RoutingKeys someKeys)
+        Invalidate(Id to, Topologies topologies, TxnId txnId, Unseekables<?, ?> scope)
         {
             this.txnId = txnId;
-            int latestRelevantIndex = latestRelevantEpochIndex(to, topologies, someKeys);
-            this.scope = computeScope(to, topologies, someKeys, latestRelevantIndex, RoutingKeys::slice, RoutingKeys::union);
+            int latestRelevantIndex = latestRelevantEpochIndex(to, topologies, scope);
+            this.scope = computeScope(to, topologies, (Unseekables)scope, latestRelevantIndex, Unseekables::slice, Unseekables::union);
             this.waitForEpoch = computeWaitForEpoch(to, topologies, latestRelevantIndex);
             this.invalidateUntilEpoch = topologies.currentEpoch();
         }
 
-        Invalidate(TxnId txnId, RoutingKeys scope, long waitForEpoch, long invalidateUntilEpoch)
+        Invalidate(TxnId txnId, Unseekables<?, ?> scope, long waitForEpoch, long invalidateUntilEpoch)
         {
             this.txnId = txnId;
             this.scope = scope;
@@ -258,9 +258,9 @@ public class Commit extends TxnRequest<ReadNack>
         }
 
         @Override
-        public Iterable<Key> keys()
+        public Seekables<?, ?> keys()
         {
-            return Collections.emptyList();
+            return Keys.EMPTY;
         }
 
         @Override
diff --git a/accord-core/src/main/java/accord/messages/EpochRequest.java b/accord-core/src/main/java/accord/messages/EpochRequest.java
deleted file mode 100644
index 8aaa115..0000000
--- a/accord-core/src/main/java/accord/messages/EpochRequest.java
+++ /dev/null
@@ -1,24 +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.messages;
-
-public interface EpochRequest extends Request
-{
-    long waitForEpoch();
-}
diff --git a/accord-core/src/main/java/accord/messages/GetDeps.java b/accord-core/src/main/java/accord/messages/GetDeps.java
index 9bdc9dd..fd62a91 100644
--- a/accord-core/src/main/java/accord/messages/GetDeps.java
+++ b/accord-core/src/main/java/accord/messages/GetDeps.java
@@ -1,20 +1,13 @@
 package accord.messages;
 
-import accord.api.Key;
 import accord.local.SafeCommandStore;
-import com.google.common.base.Preconditions;
+import accord.primitives.*;
+import accord.utils.Invariants;
 
 import accord.local.Node.Id;
-import accord.primitives.KeyRanges;
-import accord.primitives.Keys;
-import accord.primitives.PartialDeps;
-import accord.primitives.PartialRoute;
-import accord.primitives.Route;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
 import accord.topology.Topologies;
 
+import javax.annotation.Nonnull;
 import java.util.Collections;
 
 import static accord.messages.PreAccept.calculatePartialDeps;
@@ -23,25 +16,25 @@ public class GetDeps extends TxnRequest.WithUnsynced<PartialDeps>
 {
     public static final class SerializationSupport
     {
-        public static GetDeps create(TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, Keys keys, Timestamp executeAt, Txn.Kind kind)
+        public static GetDeps create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, Seekables<?, ?> keys, Timestamp executeAt, Txn.Kind kind)
         {
             return new GetDeps(txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey, keys, executeAt, kind);
         }
     }
 
-    public final Keys keys;
+    public final Seekables<?, ?> keys;
     public final Timestamp executeAt;
     public final Txn.Kind kind;
 
-    public GetDeps(Id to, Topologies topologies, Route route, TxnId txnId, Txn txn, Timestamp executeAt)
+    public GetDeps(Id to, Topologies topologies, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt)
     {
         super(to, topologies, txnId, route);
-        this.keys = txn.keys().slice(scope.covering);
+        this.keys = txn.keys().slice(scope.covering());
         this.executeAt = executeAt;
         this.kind = txn.kind();
     }
 
-    protected GetDeps(TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, Keys keys, Timestamp executeAt, Txn.Kind kind)
+    protected GetDeps(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, Seekables<?, ?> keys, Timestamp executeAt, Txn.Kind kind)
     {
         super(txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey);
         this.keys = keys;
@@ -58,7 +51,7 @@ public class GetDeps extends TxnRequest.WithUnsynced<PartialDeps>
     public PartialDeps apply(SafeCommandStore instance)
     {
         // TODO: shrink ranges to those that intersect key
-        KeyRanges ranges = instance.ranges().between(minEpoch, executeAt.epoch);
+        Ranges ranges = instance.ranges().between(minEpoch, executeAt.epoch);
         return calculatePartialDeps(instance, txnId, keys, kind, executeAt, ranges);
     }
 
@@ -97,7 +90,7 @@ public class GetDeps extends TxnRequest.WithUnsynced<PartialDeps>
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
         return keys;
     }
@@ -106,10 +99,9 @@ public class GetDeps extends TxnRequest.WithUnsynced<PartialDeps>
     {
         public final PartialDeps deps;
 
-        public GetDepsOk(PartialDeps deps)
+        public GetDepsOk(@Nonnull PartialDeps deps)
         {
-            Preconditions.checkNotNull(deps);
-            this.deps = deps;
+            this.deps = Invariants.nonNull(deps);
         }
 
         @Override
diff --git a/accord-core/src/main/java/accord/messages/InformDurable.java b/accord-core/src/main/java/accord/messages/InformDurable.java
index 355a91f..66da6aa 100644
--- a/accord-core/src/main/java/accord/messages/InformDurable.java
+++ b/accord-core/src/main/java/accord/messages/InformDurable.java
@@ -1,6 +1,5 @@
 package accord.messages;
 
-import accord.api.Key;
 import accord.api.ProgressLog.ProgressShard;
 import accord.local.Command;
 import accord.local.Node.Id;
@@ -22,7 +21,7 @@ public class InformDurable extends TxnRequest<Reply> implements PreLoadContext
 {
     public static class SerializationSupport
     {
-        public static InformDurable create(TxnId txnId, PartialRoute scope, long waitForEpoch, Timestamp executeAt, Durability durability)
+        public static InformDurable create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Timestamp executeAt, Durability durability)
         {
             return new InformDurable(txnId, scope, waitForEpoch, executeAt, durability);
         }
@@ -32,14 +31,14 @@ public class InformDurable extends TxnRequest<Reply> implements PreLoadContext
     public final Durability durability;
     private transient ProgressShard shard;
 
-    public InformDurable(Id to, Topologies topologies, AbstractRoute route, TxnId txnId, Timestamp executeAt, Durability durability)
+    public InformDurable(Id to, Topologies topologies, Route<?> route, TxnId txnId, Timestamp executeAt, Durability durability)
     {
         super(to, topologies, route, txnId);
         this.executeAt = executeAt;
         this.durability = durability;
     }
 
-    private InformDurable(TxnId txnId, PartialRoute scope, long waitForEpoch, Timestamp executeAt, Durability durability)
+    private InformDurable(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Timestamp executeAt, Durability durability)
     {
         super(txnId, scope, waitForEpoch);
         this.executeAt = executeAt;
@@ -54,12 +53,12 @@ public class InformDurable extends TxnRequest<Reply> implements PreLoadContext
             // we need to pick a progress log, but this node might not have participated in the coordination epoch
             // in this rare circumstance we simply pick a key to select some progress log to coordinate this
             at = executeAt;
-            progressKey = node.selectProgressKey(executeAt.epoch, scope, scope.homeKey);
+            progressKey = node.selectProgressKey(executeAt.epoch, scope, scope.homeKey());
             shard = Adhoc;
         }
         else
         {
-            shard = scope.homeKey.equals(progressKey) ? Home : Local;
+            shard = scope.homeKey().equals(progressKey) ? Home : Local;
         }
 
         // TODO (soon): do not load from disk to perform this update
@@ -70,7 +69,7 @@ public class InformDurable extends TxnRequest<Reply> implements PreLoadContext
     public Reply apply(SafeCommandStore safeStore)
     {
         Command command = safeStore.command(txnId);
-        command.setDurability(safeStore, durability, scope.homeKey, executeAt);
+        command.setDurability(safeStore, durability, scope.homeKey(), executeAt);
         safeStore.progressLog().durable(txnId, scope, shard);
         return Ok;
     }
@@ -109,8 +108,8 @@ public class InformDurable extends TxnRequest<Reply> implements PreLoadContext
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
-        return Collections.emptyList();
+        return Keys.EMPTY;
     }
 }
diff --git a/accord-core/src/main/java/accord/messages/InformOfTxnId.java b/accord-core/src/main/java/accord/messages/InformOfTxnId.java
index 8fea71c..bfaea3b 100644
--- a/accord-core/src/main/java/accord/messages/InformOfTxnId.java
+++ b/accord-core/src/main/java/accord/messages/InformOfTxnId.java
@@ -2,6 +2,8 @@ package accord.messages;
 
 import accord.api.RoutingKey;
 import accord.local.*;
+import accord.primitives.Keys;
+import accord.primitives.Seekables;
 import accord.primitives.TxnId;
 
 import java.util.Collections;
@@ -10,7 +12,7 @@ import static accord.api.ProgressLog.ProgressShard.Home;
 import static accord.messages.SimpleReply.Nack;
 import static accord.messages.SimpleReply.Ok;
 
-public class InformOfTxnId extends AbstractEpochRequest<Reply> implements EpochRequest, PreLoadContext
+public class InformOfTxnId extends AbstractEpochRequest<Reply> implements Request, PreLoadContext
 {
     public final RoutingKey homeKey;
 
diff --git a/accord-core/src/main/java/accord/messages/PreAccept.java b/accord-core/src/main/java/accord/messages/PreAccept.java
index eb60120..0a694b8 100644
--- a/accord-core/src/main/java/accord/messages/PreAccept.java
+++ b/accord-core/src/main/java/accord/messages/PreAccept.java
@@ -21,14 +21,12 @@ package accord.messages;
 import java.util.Collections;
 import java.util.Objects;
 
-import accord.api.Key;
 import accord.local.*;
 import accord.local.CommandsForKey.CommandTimeseries.TestKind;
 
 import accord.local.Node.Id;
 import accord.messages.TxnRequest.WithUnsynced;
 import accord.topology.Topologies;
-import accord.primitives.Keys;
 import accord.primitives.Timestamp;
 import javax.annotation.Nullable;
 
@@ -46,25 +44,25 @@ public class PreAccept extends WithUnsynced<PreAccept.PreAcceptReply>
 {
     public static class SerializerSupport
     {
-        public static PreAccept create(TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, long maxEpoch, PartialTxn partialTxn, @Nullable Route fullRoute)
+        public static PreAccept create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, long maxEpoch, PartialTxn partialTxn, @Nullable FullRoute<?> fullRoute)
         {
             return new PreAccept(txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey, maxEpoch, partialTxn, fullRoute);
         }
     }
 
     public final PartialTxn partialTxn;
-    public final @Nullable Route route; // ordinarily only set on home shard
+    public final @Nullable FullRoute<?> route; // ordinarily only set on home shard
     public final long maxEpoch;
 
-    public PreAccept(Id to, Topologies topologies, TxnId txnId, Txn partialTxn, Route route)
+    public PreAccept(Id to, Topologies topologies, TxnId txnId, Txn txn, FullRoute<?> route)
     {
         super(to, topologies, txnId, route);
-        this.partialTxn = partialTxn.slice(scope.covering, route.contains(route.homeKey));
+        this.partialTxn = txn.slice(scope.covering(), route.contains(route.homeKey()));
         this.maxEpoch = topologies.currentEpoch();
-        this.route = scope.contains(scope.homeKey) ? route : null;
+        this.route = scope.contains(scope.homeKey()) ? route : null;
     }
 
-    private PreAccept(TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, long maxEpoch, PartialTxn partialTxn, @Nullable Route fullRoute)
+    private PreAccept(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, long maxEpoch, PartialTxn partialTxn, @Nullable FullRoute<?> fullRoute)
     {
         super(txnId, scope, waitForEpoch, minEpoch, doNotComputeProgressKey);
         this.partialTxn = partialTxn;
@@ -79,7 +77,7 @@ public class PreAccept extends WithUnsynced<PreAccept.PreAcceptReply>
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
         return partialTxn.keys();
     }
@@ -135,18 +133,18 @@ public class PreAccept extends WithUnsynced<PreAccept.PreAcceptReply>
         return MessageType.PREACCEPT_REQ;
     }
 
-    public interface PreAcceptReply extends Reply
+    public static abstract class PreAcceptReply implements Reply
     {
         @Override
-        default MessageType type()
+        public MessageType type()
         {
             return MessageType.PREACCEPT_RSP;
         }
 
-        boolean isOk();
+        public abstract boolean isOk();
     }
 
-    public static class PreAcceptOk implements PreAcceptReply
+    public static class PreAcceptOk extends PreAcceptReply
     {
         public final TxnId txnId;
         public final Timestamp witnessedAt;
@@ -191,7 +189,7 @@ public class PreAccept extends WithUnsynced<PreAccept.PreAcceptReply>
         }
     }
 
-    public static class PreAcceptNack implements PreAcceptReply
+    public static class PreAcceptNack extends PreAcceptReply
     {
         public static final PreAcceptNack INSTANCE = new PreAcceptNack();
 
@@ -210,7 +208,7 @@ public class PreAccept extends WithUnsynced<PreAccept.PreAcceptReply>
         }
     }
 
-    static PartialDeps calculatePartialDeps(SafeCommandStore commandStore, TxnId txnId, Keys keys, Txn.Kind kindOfTxn, Timestamp executeAt, KeyRanges ranges)
+    static PartialDeps calculatePartialDeps(SafeCommandStore commandStore, TxnId txnId, Seekables<?, ?> keys, Txn.Kind kindOfTxn, Timestamp executeAt, Ranges ranges)
     {
         try (PartialDeps.OrderedBuilder builder = PartialDeps.orderedBuilder(ranges, false))
         {
@@ -218,15 +216,11 @@ public class PreAccept extends WithUnsynced<PreAccept.PreAcceptReply>
         }
     }
 
-    private static <T extends Deps> T calculateDeps(SafeCommandStore commandStore, TxnId txnId, Keys keys, Txn.Kind kindOfTxn, Timestamp executeAt, KeyRanges ranges, Deps.AbstractOrderedBuilder<T> builder)
+    private static <T extends Deps> T calculateDeps(SafeCommandStore commandStore, TxnId txnId, Seekables<?, ?> keys, Txn.Kind kindOfTxn, Timestamp executeAt, Ranges ranges, Deps.AbstractOrderedBuilder<T> builder)
     {
-        keys.foldl(ranges, (i, key, ignore) -> {
-            CommandsForKey forKey = commandStore.maybeCommandsForKey(key);
-            if (forKey == null)
-                return null;
-
-            builder.nextKey(key);
-            TestKind testKind = kindOfTxn.isWrite() ? RorWs : Ws;
+        TestKind testKind = kindOfTxn.isWrite() ? RorWs : Ws;
+        commandStore.forEach(keys, ranges, forKey -> {
+            builder.nextKey(forKey.key());
             forKey.uncommitted().before(executeAt, testKind, ANY_DEPS, null, ANY_STATUS, null)
                     .forEach(info -> {
                         if (!info.txnId.equals(txnId)) builder.add(info.txnId);
@@ -235,8 +229,7 @@ public class PreAccept extends WithUnsynced<PreAccept.PreAcceptReply>
                     .forEach(id -> {
                         if (!id.equals(txnId)) builder.add(id);
                     });
-            return null;
-        }, null);
+        });
 
         return builder.build();
     }
diff --git a/accord-core/src/main/java/accord/messages/ReadData.java b/accord-core/src/main/java/accord/messages/ReadData.java
index 48276b4..f6faab8 100644
--- a/accord-core/src/main/java/accord/messages/ReadData.java
+++ b/accord-core/src/main/java/accord/messages/ReadData.java
@@ -23,13 +23,13 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 
-import accord.api.Key;
 import accord.primitives.*;
 
 import accord.local.*;
 import accord.api.Data;
 import accord.topology.Topologies;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,30 +49,30 @@ public class ReadData extends AbstractEpochRequest<ReadData.ReadNack> implements
 
     public static class SerializerSupport
     {
-        public static ReadData create(TxnId txnId, Keys scope, long executeAtEpoch, long waitForEpoch)
+        public static ReadData create(TxnId txnId, Seekables<?, ?> scope, long executeAtEpoch, long waitForEpoch)
         {
             return new ReadData(txnId, scope, executeAtEpoch, waitForEpoch);
         }
     }
 
     public final long executeAtEpoch;
-    public final Keys readScope; // TODO: this should be RoutingKeys as we have the Keys locally - but for simplicity for now we use Keys to implement keys()
+    public final Seekables<?, ?> readScope; // TODO: this should be RoutingKeys as we have the Keys locally - but for simplicity for now we use Keys to implement keys()
     private final long waitForEpoch;
     private Data data;
     private transient boolean isObsolete; // TODO: respond with the Executed result we have stored?
     private transient BitSet waitingOn;
     private transient int waitingOnCount;
 
-    public ReadData(Node.Id to, Topologies topologies, TxnId txnId, Keys readScope, Timestamp executeAt)
+    public ReadData(Node.Id to, Topologies topologies, TxnId txnId, Seekables<?, ?> readScope, Timestamp executeAt)
     {
         super(txnId);
         this.executeAtEpoch = executeAt.epoch;
         int startIndex = latestRelevantEpochIndex(to, topologies, readScope);
-        this.readScope = computeScope(to, topologies, readScope, startIndex, Keys::slice, Keys::union);
+        this.readScope = computeScope(to, topologies, (Seekables)readScope, startIndex, Seekables::slice, Seekables::union);
         this.waitForEpoch = computeWaitForEpoch(to, topologies, startIndex);
     }
 
-    ReadData(TxnId txnId, Keys readScope, long executeAtEpoch, long waitForEpoch)
+    ReadData(TxnId txnId, Seekables<?, ?> readScope, long executeAtEpoch, long waitForEpoch)
     {
         super(txnId);
         this.executeAtEpoch = executeAtEpoch;
@@ -146,7 +146,7 @@ public class ReadData extends AbstractEpochRequest<ReadData.ReadNack> implements
             default:
                 throw new AssertionError();
             case Committed:
-                if (command.partialTxn().keys().none(safeStore.commandStore()::hashIntersects))
+                if (!Iterables.any(command.partialTxn().keys(), safeStore.commandStore()::hashIntersects))
                     throw new IllegalStateException();
             case NotWitnessed:
             case PreAccepted:
@@ -160,7 +160,7 @@ public class ReadData extends AbstractEpochRequest<ReadData.ReadNack> implements
                 if (status == Committed)
                     return null;
 
-                safeStore.progressLog().waiting(txnId, Committed.minKnown, readScope.toRoutingKeys());
+                safeStore.progressLog().waiting(txnId, Committed.minKnown, readScope.toUnseekables());
                 return NotCommitted;
 
             case ReadyToExecute:
@@ -256,7 +256,7 @@ public class ReadData extends AbstractEpochRequest<ReadData.ReadNack> implements
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
         return readScope;
     }
diff --git a/accord-core/src/main/java/accord/messages/Request.java b/accord-core/src/main/java/accord/messages/Request.java
index b48d8c4..683a260 100644
--- a/accord-core/src/main/java/accord/messages/Request.java
+++ b/accord-core/src/main/java/accord/messages/Request.java
@@ -24,4 +24,5 @@ import accord.local.Node.Id;
 public interface Request extends Message
 {
     void process(Node on, Id from, ReplyContext replyContext);
+    default long waitForEpoch() { return 0; }
 }
diff --git a/accord-core/src/main/java/accord/messages/TxnRequest.java b/accord-core/src/main/java/accord/messages/TxnRequest.java
index e516368..c34fa05 100644
--- a/accord-core/src/main/java/accord/messages/TxnRequest.java
+++ b/accord-core/src/main/java/accord/messages/TxnRequest.java
@@ -22,53 +22,52 @@ import java.util.function.BiFunction;
 
 import accord.local.SafeCommandStore;
 import accord.utils.MapReduceConsume;
-import com.google.common.base.Preconditions;
+import accord.primitives.*;
+import accord.utils.Invariants;
 
 import accord.api.RoutingKey;
 import accord.local.Node;
 import accord.local.Node.Id;
-import accord.primitives.AbstractKeys;
-import accord.primitives.AbstractRoute;
-import accord.primitives.KeyRanges;
 import accord.local.PreLoadContext;
+import accord.primitives.Ranges;
 import accord.primitives.PartialRoute;
 import accord.primitives.Route;
+import accord.primitives.FullRoute;
 import accord.topology.Topologies;
 import accord.topology.Topology;
-import accord.primitives.TxnId;
 
 import static java.lang.Long.min;
 
-public abstract class TxnRequest<R> implements EpochRequest, PreLoadContext, MapReduceConsume<SafeCommandStore, R>
+public abstract class TxnRequest<R> implements Request, PreLoadContext, MapReduceConsume<SafeCommandStore, R>
 {
     public static abstract class WithUnsynced<R> extends TxnRequest<R>
     {
         public final long minEpoch; // TODO: can this just always be TxnId.epoch?
         public final boolean doNotComputeProgressKey;
 
-        protected WithUnsynced(Id to, Topologies topologies, TxnId txnId, Route route)
+        public WithUnsynced(Id to, Topologies topologies, TxnId txnId, FullRoute<?> route)
         {
             this(to, topologies, txnId, route, latestRelevantEpochIndex(to, topologies, route));
         }
 
-        protected WithUnsynced(Id to, Topologies topologies, TxnId txnId, Route route, int startIndex)
+        private WithUnsynced(Id to, Topologies topologies, TxnId txnId, FullRoute<?> route, int startIndex)
         {
             super(to, topologies, route, txnId, startIndex);
             this.minEpoch = topologies.oldestEpoch();
             this.doNotComputeProgressKey = doNotComputeProgressKey(topologies, startIndex, txnId, waitForEpoch());
 
-            // TODO (soon): alongside Invariants class, introduce PARANOID mode for checking extra invariants
-            KeyRanges ranges = topologies.forEpoch(txnId.epoch).rangesForNode(to);
+            // TODO (now): alongside Invariants class, introduce PARANOID mode for checking extra invariants
+            Ranges ranges = topologies.forEpoch(txnId.epoch).rangesForNode(to);
             if (doNotComputeProgressKey)
             {
-                Preconditions.checkState(!ranges.intersects(route)); // confirm dest is not a replica on txnId.epoch
+                Invariants.checkState(!route.intersects(ranges)); // confirm dest is not a replica on txnId.epoch
             }
             else
             {
-                boolean intersects = ranges.intersects(route);
+                boolean intersects = route.intersects(ranges);
                 long progressEpoch = Math.min(waitForEpoch(), txnId.epoch);
-                KeyRanges computesRangesOn = topologies.forEpoch(progressEpoch).rangesForNode(to);
-                boolean check = computesRangesOn != null && computesRangesOn.intersects(route);
+                Ranges computesRangesOn = topologies.forEpoch(progressEpoch).rangesForNode(to);
+                boolean check = computesRangesOn != null && route.intersects(computesRangesOn);
                 if (check != intersects)
                     throw new IllegalStateException();
             }
@@ -91,26 +90,26 @@ public abstract class TxnRequest<R> implements EpochRequest, PreLoadContext, Map
     }
 
     public final TxnId txnId;
-    public final PartialRoute scope;
+    public final PartialRoute<?> scope;
     public final long waitForEpoch;
     protected transient RoutingKey progressKey;
     protected transient Node node;
     protected transient Id replyTo;
     protected transient ReplyContext replyContext;
 
-    protected TxnRequest(Id to, Topologies topologies, AbstractRoute route, TxnId txnId)
+    public TxnRequest(Node.Id to, Topologies topologies, Route<?> route, TxnId txnId)
     {
         this(to, topologies, route, txnId, latestRelevantEpochIndex(to, topologies, route));
     }
 
-    protected TxnRequest(Id to, Topologies topologies, AbstractRoute route, TxnId txnId, int startIndex)
+    public TxnRequest(Node.Id to, Topologies topologies, Route<?> route, TxnId txnId, int startIndex)
     {
         this(txnId, computeScope(to, topologies, route, startIndex), computeWaitForEpoch(to, topologies, startIndex));
     }
 
-    protected TxnRequest(TxnId txnId, PartialRoute scope, long waitForEpoch)
+    public TxnRequest(TxnId txnId, PartialRoute<?> scope, long waitForEpoch)
     {
-        Preconditions.checkState(!scope.isEmpty());
+        Invariants.checkState(!scope.isEmpty());
         this.txnId = txnId;
         this.scope = scope;
         this.waitForEpoch = waitForEpoch;
@@ -120,7 +119,7 @@ public abstract class TxnRequest<R> implements EpochRequest, PreLoadContext, Map
      * The portion of the complete Route that this TxnRequest applies to. Should represent the complete
      * range owned by the target node for the involved epochs.
      */
-    public PartialRoute scope()
+    public PartialRoute<?> scope()
     {
         return scope;
     }
@@ -149,17 +148,17 @@ public abstract class TxnRequest<R> implements EpochRequest, PreLoadContext, Map
     {
         // if waitForEpoch < txnId.epoch, then this replica's ownership is unchanged
         long progressEpoch = min(waitForEpoch(), txnId.epoch);
-        return node.trySelectProgressKey(progressEpoch, scope, scope.homeKey);
+        return node.trySelectProgressKey(progressEpoch, scope, scope.homeKey());
     }
 
     protected abstract void process();
 
     // finds the first topology index that intersects with the node
-    protected static int latestRelevantEpochIndex(Node.Id node, Topologies topologies, AbstractKeys<?, ?> keys)
+    protected static int latestRelevantEpochIndex(Node.Id node, Topologies topologies, Routables<?, ?> route)
     {
-        KeyRanges latest = topologies.current().rangesForNode(node);
+        Ranges latest = topologies.current().rangesForNode(node);
 
-        if (latest.intersects(keys))
+        if (route.intersects(latest))
             return 0;
 
         int i = 0;
@@ -174,16 +173,16 @@ public abstract class TxnRequest<R> implements EpochRequest, PreLoadContext, Map
             latest = topologies.get(i).rangesForNode(node);
         }
 
-        if (latest.intersects(keys))
+        if (route.intersects(latest))
             return i;
 
         // find first non-empty intersection for node
         while (++i < mi)
         {
-            KeyRanges next = topologies.get(i).rangesForNode(node);
+            Ranges next = topologies.get(i).rangesForNode(node);
             if (!next.equals(latest))
             {
-                if (next.intersects(keys))
+                if (route.intersects(next))
                     return i;
                 latest = next;
             }
@@ -198,9 +197,9 @@ public abstract class TxnRequest<R> implements EpochRequest, PreLoadContext, Map
      * on the assumption that this might also mean some local shard rearrangement
      * (ignoring the case where the latest epochs do not intersect the keys at all)
      */
-    public static long computeWaitForEpoch(Node.Id node, Topologies topologies, AbstractKeys<?, ?> keys)
+    public static long computeWaitForEpoch(Node.Id node, Topologies topologies, Unseekables<?, ?> scope)
     {
-        return computeWaitForEpoch(node, topologies, latestRelevantEpochIndex(node, topologies, keys));
+        return computeWaitForEpoch(node, topologies, latestRelevantEpochIndex(node, topologies, scope));
     }
 
     public static long computeWaitForEpoch(Node.Id node, Topologies topologies, int startIndex)
@@ -210,11 +209,11 @@ public abstract class TxnRequest<R> implements EpochRequest, PreLoadContext, Map
         if (i == mi)
             return topologies.oldestEpoch();
 
-        KeyRanges latest = topologies.get(i - 1).rangesForNode(node);
+        Ranges latest = topologies.get(i - 1).rangesForNode(node);
         while (i < mi)
         {
             Topology topology = topologies.get(i);
-            KeyRanges ranges = topology.rangesForNode(node);
+            Ranges ranges = topology.rangesForNode(node);
             if (!ranges.equals(latest))
                 break;
             ++i;
@@ -222,18 +221,18 @@ public abstract class TxnRequest<R> implements EpochRequest, PreLoadContext, Map
         return topologies.get(i - 1).epoch();
     }
 
-    public static PartialRoute computeScope(Node.Id node, Topologies topologies, Route keys)
+    public static PartialRoute<?> computeScope(Node.Id node, Topologies topologies, FullRoute<?> fullRoute)
     {
-        return computeScope(node, topologies, keys, latestRelevantEpochIndex(node, topologies, keys));
+        return computeScope(node, topologies, fullRoute, latestRelevantEpochIndex(node, topologies, fullRoute));
     }
 
-    public static PartialRoute computeScope(Node.Id node, Topologies topologies, AbstractRoute route, int startIndex)
+    public static PartialRoute<?> computeScope(Node.Id node, Topologies topologies, Route<?> route, int startIndex)
     {
-        return computeScope(node, topologies, route, startIndex, AbstractRoute::slice, PartialRoute::union);
+        return computeScope(node, topologies, route, startIndex, Route::slice, PartialRoute::union);
     }
 
     // TODO: move to Topologies
-    public static <I extends AbstractKeys<?, ?>, O extends AbstractKeys<?, ?>> O computeScope(Node.Id node, Topologies topologies, I keys, int startIndex, BiFunction<I, KeyRanges, O> slice, BiFunction<O, O, O> merge)
+    public static <I, O> O computeScope(Node.Id node, Topologies topologies, I keys, int startIndex, BiFunction<I, Ranges, O> slice, BiFunction<O, O, O> merge)
     {
         O scope = computeScopeInternal(node, topologies, keys, startIndex, slice, merge);
         if (scope == null)
@@ -241,14 +240,14 @@ public abstract class TxnRequest<R> implements EpochRequest, PreLoadContext, Map
         return scope;
     }
 
-    private static <I, O> O computeScopeInternal(Node.Id node, Topologies topologies, I keys, int startIndex, BiFunction<I, KeyRanges, O> slice, BiFunction<O, O, O> merge)
+    private static <I, O> O computeScopeInternal(Node.Id node, Topologies topologies, I keys, int startIndex, BiFunction<I, Ranges, O> slice, BiFunction<O, O, O> merge)
     {
-        KeyRanges last = null;
+        Ranges last = null;
         O scope = null;
         for (int i = startIndex, mi = topologies.size() ; i < mi ; ++i)
         {
             Topology topology = topologies.get(i);
-            KeyRanges ranges = topology.rangesForNode(node);
+            Ranges ranges = topology.rangesForNode(node);
             if (ranges != last && !ranges.equals(last))
             {
                 O add = slice.apply(keys, ranges);
diff --git a/accord-core/src/main/java/accord/messages/WaitOnCommit.java b/accord-core/src/main/java/accord/messages/WaitOnCommit.java
index 1cee8bf..3a77406 100644
--- a/accord-core/src/main/java/accord/messages/WaitOnCommit.java
+++ b/accord-core/src/main/java/accord/messages/WaitOnCommit.java
@@ -21,12 +21,9 @@ package accord.messages;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
-import accord.api.Key;
 import accord.local.*;
 import accord.local.Node.Id;
-import accord.local.Status.Known;
-import accord.primitives.RoutingKeys;
-import accord.primitives.TxnId;
+import accord.primitives.*;
 import accord.utils.MapReduceConsume;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,20 +32,20 @@ import static accord.local.Status.Committed;
 import static accord.utils.Utils.listOf;
 import accord.topology.Topology;
 
-public class WaitOnCommit implements EpochRequest, MapReduceConsume<SafeCommandStore, Void>, PreLoadContext, CommandListener
+public class WaitOnCommit implements Request, MapReduceConsume<SafeCommandStore, Void>, PreLoadContext, CommandListener
 {
     private static final Logger logger = LoggerFactory.getLogger(WaitOnCommit.class);
 
     public static class SerializerSupport
     {
-        public static WaitOnCommit create(TxnId txnId, RoutingKeys scope)
+        public static WaitOnCommit create(TxnId txnId, Unseekables<?, ?> scope)
         {
             return new WaitOnCommit(txnId, scope);
         }
     }
 
     public final TxnId txnId;
-    public final RoutingKeys scope;
+    public final Unseekables<?, ?> scope;
 
     private transient Node node;
     private transient Id replyTo;
@@ -56,13 +53,13 @@ public class WaitOnCommit implements EpochRequest, MapReduceConsume<SafeCommandS
     private transient volatile int waitingOn;
     private static final AtomicIntegerFieldUpdater<WaitOnCommit> waitingOnUpdater = AtomicIntegerFieldUpdater.newUpdater(WaitOnCommit.class, "waitingOn");
 
-    public WaitOnCommit(Id to, Topology topologies, TxnId txnId, RoutingKeys someKeys)
+    public WaitOnCommit(Id to, Topology topologies, TxnId txnId, Unseekables<?, ?> unseekables)
     {
         this.txnId = txnId;
-        this.scope = someKeys.slice(topologies.rangesForNode(to));
+        this.scope = unseekables.slice(topologies.rangesForNode(to));
     }
 
-    public WaitOnCommit(TxnId txnId, RoutingKeys scope)
+    public WaitOnCommit(TxnId txnId, Unseekables<?, ?> scope)
     {
         this.txnId = txnId;
         this.scope = scope;
@@ -159,9 +156,9 @@ public class WaitOnCommit implements EpochRequest, MapReduceConsume<SafeCommandS
     }
 
     @Override
-    public Iterable<Key> keys()
+    public Seekables<?, ?> keys()
     {
-        return Collections.emptyList();
+        return Keys.EMPTY;
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/primitives/AbstractKeys.java b/accord-core/src/main/java/accord/primitives/AbstractKeys.java
index e41e267..47725fe 100644
--- a/accord-core/src/main/java/accord/primitives/AbstractKeys.java
+++ b/accord-core/src/main/java/accord/primitives/AbstractKeys.java
@@ -2,24 +2,22 @@ package accord.primitives;
 
 import java.util.Arrays;
 import java.util.Iterator;
-import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
 import java.util.function.IntFunction;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import accord.api.RoutingKey;
-import accord.utils.IndexedFold;
-import accord.utils.IndexedFoldIntersectToLong;
-import accord.utils.IndexedFoldToLong;
-import accord.utils.IndexedPredicate;
-import accord.utils.IndexedRangeFoldToLong;
-import accord.utils.SortedArrays;
+import accord.utils.*;
 import net.nicoulaj.compilecommand.annotations.Inline;
 
+import static accord.primitives.Routable.Kind.Key;
+
 @SuppressWarnings("rawtypes")
 // TODO: check that foldl call-sites are inlined and optimised by HotSpot
-public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys<K, KS>> implements Iterable<K>
+public abstract class AbstractKeys<K extends RoutableKey, KS extends Routables<K, ?>> implements Iterable<K>, Routables<K, KS>
 {
     final K[] keys;
 
@@ -37,58 +35,82 @@ public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys
         return Arrays.equals(keys, that.keys);
     }
 
+    public final int indexOf(K key)
+    {
+        return Arrays.binarySearch(keys, key);
+    }
+
+    public final K get(int indexOf)
+    {
+        return keys[indexOf];
+    }
+
     @Override
-    public int hashCode()
+    public final Unseekable.Kind kindOfContents()
     {
-        return Arrays.hashCode(keys);
+        return Key;
     }
 
-    public int indexOf(K key)
+    public final boolean isEmpty()
     {
-        return Arrays.binarySearch(keys, key);
+        return keys.length == 0;
     }
 
-    public boolean contains(K key)
+    public final int size()
     {
-        return indexOf(key) >= 0;
+        return keys.length;
     }
 
-    public K get(int indexOf)
+    public final boolean contains(RoutableKey key)
     {
-        return keys[indexOf];
+        return Arrays.binarySearch(keys, key) >= 0;
     }
 
-    /**
-     * return true if this keys collection contains all keys found in the given keys
-     */
-    public boolean containsAll(AbstractKeys<K, ?> that)
+    public final boolean containsAll(Routables<?, ?> keysOrRanges)
     {
-        if (that.isEmpty())
-            return true;
+        return keysOrRanges.size() == Routables.foldl(keysOrRanges, this, (i, k, p, v) -> v + 1, 0, 0, 0);
+    }
+
+    @Override
+    public boolean intersects(AbstractRanges<?> ranges)
+    {
+        // TODO (now): make this final
+        return ranges.intersects(this);
+    }
 
-        return foldlIntersect(that, (li, ri, k, p, v) -> v + 1, 0, 0, 0) == that.size();
+    public final int findNext(K key, int startIndex)
+    {
+        return SortedArrays.exponentialSearch(keys, startIndex, keys.length, key);
     }
 
+    @Override
+    public final int findNext(int thisIndex, Range find, SortedArrays.Search search)
+    {
+        return SortedArrays.exponentialSearch(keys, thisIndex, size(), find, Range::compareTo, search);
+    }
 
-    public boolean isEmpty()
+    @Override
+    public final int findNext(int thisIndex, K find, SortedArrays.Search search)
     {
-        return keys.length == 0;
+        return SortedArrays.exponentialSearch(keys, thisIndex, size(), find, RoutableKey::compareTo, search);
     }
 
-    public int size()
+    @Override
+    public final long findNextIntersection(int thisIdx, AbstractRanges<?> that, int thatIdx)
     {
-        return keys.length;
+        return SortedArrays.findNextIntersectionWithMultipleMatches(this.keys, thisIdx, that.ranges, thatIdx, (RoutableKey k, Range r) -> -r.compareTo(k), Range::compareTo);
     }
 
-    public int findNext(K key, int startIndex)
+    @Override
+    public final long findNextIntersection(int thisIdx, AbstractKeys<?, ?> that, int thatIdx)
     {
-        return SortedArrays.exponentialSearch(keys, startIndex, keys.length, key);
+        return SortedArrays.findNextIntersection(this.keys, thisIdx, that.keys, thatIdx, RoutableKey::compareTo);
     }
 
-    // returns thisIdx in top 32 bits, thatIdx in bottom
-    public long findNextIntersection(int thisIdx, AbstractKeys<K, ?> that, int thatIdx)
+    @Override
+    public final long findNextIntersection(int thisIndex, Routables<K, ?> with, int withIndex)
     {
-        return SortedArrays.findNextIntersection(this.keys, thisIdx, that.keys, thatIdx);
+        return findNextIntersection(thisIndex, (AbstractKeys<?, ?>) with, withIndex);
     }
 
     public Stream<K> stream()
@@ -122,14 +144,13 @@ public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys
         return stream().map(Object::toString).collect(Collectors.joining(",", "[", "]"));
     }
 
-
-
-    protected K[] slice(KeyRanges ranges, IntFunction<K[]> factory)
+    // TODO (now): accept cached buffers
+    protected K[] slice(Ranges ranges, IntFunction<K[]> factory)
     {
-        return SortedArrays.sliceWithMultipleMatches(keys, ranges.ranges, factory, (k, r) -> -r.compareTo(k), KeyRange::compareTo);
+        return SortedArrays.sliceWithMultipleMatches(keys, ranges.ranges, factory, (k, r) -> -r.compareTo(k), Range::compareTo);
     }
 
-    public boolean any(KeyRanges ranges, Predicate<? super K> predicate)
+    public boolean any(Ranges ranges, Predicate<? super K> predicate)
     {
         return 1 == foldl(ranges, (i1, key, i2, i3) -> predicate.test(key) ? 1 : 0, 0, 0, 1);
     }
@@ -149,27 +170,9 @@ public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys
      * If terminateAfter is greater than 0, the method will return once terminateAfter matches are encountered
      */
     @Inline
-    public <V> V foldl(KeyRanges rs, IndexedFold<? super K, V> fold, V accumulator)
+    public final <V> V foldl(Ranges rs, IndexedFold<? super K, V> fold, V accumulator)
     {
-        int ki = 0, ri = 0;
-        while (true)
-        {
-            long rki = rs.findNextIntersection(ri, this, ki);
-            if (rki < 0)
-                break;
-
-            ri = (int)(rki >>> 32);
-            ki = (int)(rki);
-            KeyRange range = rs.get(ri);
-            int nextai = range.nextHigherKeyIndex(this, ki + 1);
-            while (ki < nextai)
-            {
-                accumulator = fold.apply(ki, get(ki), accumulator);
-                ++ki;
-            }
-        }
-
-        return accumulator;
+        return Routables.foldl(this, rs, fold, accumulator);
     }
 
     /**
@@ -177,67 +180,19 @@ public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys
      * If terminateAfter is greater than 0, the method will return once terminateAfter matches are encountered
      */
     @Inline
-    public <V> V foldl(IndexedFold<? super K, V> fold, V accumulator)
-    {
-        for (int ki = 0; ki < size() ; ++ki)
-            accumulator = fold.apply(ki, get(ki), accumulator);
-        return accumulator;
-    }
-
-    @Inline
-    public long foldl(KeyRanges rs, IndexedFoldToLong<? super K> fold, long param, long initialValue, long terminalValue)
+    public final void forEach(Ranges rs, Consumer<? super K> forEach)
     {
-        int ki = 0, ri = 0;
-        done: while (true)
-        {
-            long rki = rs.findNextIntersection(ri, this, ki);
-            if (rki < 0)
-                break;
-
-            ri = (int)(rki >>> 32);
-            ki = (int)(rki);
-            KeyRange range = rs.get(ri);
-            int nextai = range.nextHigherKeyIndex(this, ki + 1);
-            while (ki < nextai)
-            {
-                initialValue = fold.apply(ki, get(ki), param, initialValue);
-                if (initialValue == terminalValue)
-                    break done;
-                ++ki;
-            }
-        }
-
-        return initialValue;
+        Routables.foldl(this, rs, (i, k, consumer) -> { consumer.accept(k); return consumer; }, forEach);
     }
 
-    /**
-     * A fold variation permitting more efficient operation over indices only, by providing ranges of matching indices
-     */
     @Inline
-    public long rangeFoldl(KeyRanges rs, IndexedRangeFoldToLong fold, long param, long initialValue, long terminalValue)
+    public final long foldl(Ranges rs, IndexedFoldToLong<? super K> fold, long param, long initialValue, long terminalValue)
     {
-        int ki = 0, ri = 0;
-        while (true)
-        {
-            long rki = rs.findNextIntersection(ri, this, ki);
-            if (rki < 0)
-                break;
-
-            ri = (int)(rki >>> 32);
-            ki = (int)(rki);
-            KeyRange range = rs.get(ri);
-            int nextai = range.nextHigherKeyIndex(this, ki + 1);
-            initialValue = fold.apply(ki, nextai, param, initialValue);
-            if (initialValue == terminalValue)
-                break;
-            ki = nextai;
-        }
-
-        return initialValue;
+        return Routables.foldl(this, rs, fold, param, initialValue, terminalValue);
     }
 
     @Inline
-    public long foldl(IndexedFoldToLong<? super K> fold, long param, long initialValue, long terminalValue)
+    public final long foldl(IndexedFoldToLong<? super K> fold, long param, long initialValue, long terminalValue)
     {
         for (int i = 0; i < keys.length; i++)
         {
@@ -248,55 +203,25 @@ public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys
         return initialValue;
     }
 
-    /**
-     * A fold variation that intersects two key sets, invoking the fold function only on those
-     * items that are members of both sets (with their corresponding indices).
-     */
-    @Inline
-    public long foldlIntersect(AbstractKeys<K, ?> intersect, IndexedFoldIntersectToLong<? super K> fold, long param, long initialValue, long terminalValue)
-    {
-        return SortedArrays.foldlIntersection(this.keys, intersect.keys, fold, param, initialValue, terminalValue);
-    }
-
-    /**
-     * A fold variation that invokes the fold function only on those items that are members of this set
-     * and NOT the provided set.
-     */
-    @Inline
-    public long foldlDifference(AbstractKeys<K, ?> subtract, IndexedFoldToLong<? super K> fold, long param, long initialValue, long terminalValue)
-    {
-        return SortedArrays.foldlDifference(keys, subtract.keys, fold, param, initialValue, terminalValue);
-    }
-
-    public Route toRoute(RoutingKey homeKey)
+    public final FullKeyRoute toRoute(RoutingKey homeKey)
     {
         if (isEmpty())
-            return new Route(homeKey, new RoutingKey[] { homeKey });
+            return new FullKeyRoute(homeKey, new RoutingKey[] { homeKey });
 
         RoutingKey[] result = toRoutingKeysArray(homeKey);
         int pos = Arrays.binarySearch(result, homeKey);
-        return new Route(result[pos], result);
+        return new FullKeyRoute(result[pos], result);
     }
 
-    public PartialRoute toPartialRoute(KeyRanges ranges, RoutingKey homeKey)
-    {
-        if (isEmpty())
-            return new PartialRoute(ranges, homeKey, new RoutingKey[] { homeKey });
-
-        RoutingKey[] result = toRoutingKeysArray(homeKey);
-        int pos = Arrays.binarySearch(result, homeKey);
-        return new PartialRoute(ranges, result[pos], result);
-    }
-
-    private RoutingKey[] toRoutingKeysArray(RoutingKey homeKey)
+    protected RoutingKey[] toRoutingKeysArray(RoutingKey withKey)
     {
         RoutingKey[] result;
         int resultCount;
-        int insertPos = Arrays.binarySearch(keys, homeKey);
+        int insertPos = Arrays.binarySearch(keys, withKey);
         if (insertPos < 0)
             insertPos = -1 - insertPos;
 
-        if (insertPos < keys.length && keys[insertPos].toRoutingKey().equals(homeKey))
+        if (insertPos < keys.length && keys[insertPos].toUnseekable().equals(withKey))
         {
             result = new RoutingKey[keys.length];
             resultCount = copyToRoutingKeys(keys, 0, result, 0, keys.length);
@@ -305,8 +230,8 @@ public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys
         {
             result = new RoutingKey[1 + keys.length];
             resultCount = copyToRoutingKeys(keys, 0, result, 0, insertPos);
-            if (resultCount == 0 || !homeKey.equals(result[resultCount - 1]))
-                result[resultCount++] = homeKey;
+            if (resultCount == 0 || !withKey.equals(result[resultCount - 1]))
+                result[resultCount++] = withKey;
             resultCount += copyToRoutingKeys(keys, insertPos, result, resultCount, keys.length - insertPos);
         }
 
@@ -316,19 +241,24 @@ public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys
         return result;
     }
 
-    public RoutingKeys toRoutingKeys()
+    public final RoutingKeys toUnseekables()
+    {
+        return toUnseekables(array -> array.length == 0 ? RoutingKeys.EMPTY : new RoutingKeys(array));
+    }
+
+    private <R> R toUnseekables(Function<RoutingKey[], R> constructor)
     {
         if (isEmpty())
-            return RoutingKeys.EMPTY;
+            constructor.apply(RoutingKeys.EMPTY.keys);
 
         RoutingKey[] result = new RoutingKey[keys.length];
         int resultCount = copyToRoutingKeys(keys, 0, result, 0, keys.length);
         if (resultCount < result.length)
             result = Arrays.copyOf(result, resultCount);
-        return new RoutingKeys(result);
+        return constructor.apply(result);
     }
 
-    private static <K extends RoutingKey> int copyToRoutingKeys(K[] src, int srcPos, RoutingKey[] trg, int trgPos, int count)
+    private static <K extends RoutableKey> int copyToRoutingKeys(K[] src, int srcPos, RoutingKey[] trg, int trgPos, int count)
     {
         if (count == 0)
             return 0;
@@ -336,11 +266,11 @@ public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys
         int srcEnd = srcPos + count;
         int trgStart = trgPos;
         if (trgPos == 0)
-            trg[trgPos++] = src[srcPos++].toRoutingKey();
+            trg[trgPos++] = src[srcPos++].toUnseekable();
 
         while (srcPos < srcEnd)
         {
-            RoutingKey next = src[srcPos++].toRoutingKey();
+            RoutingKey next = src[srcPos++].toUnseekable();
             if (!next.equals(trg[trgPos - 1]))
                 trg[trgPos++] = next;
         }
@@ -348,4 +278,16 @@ public abstract class AbstractKeys<K extends RoutingKey, KS extends AbstractKeys
         return trgPos - trgStart;
     }
 
+    static <T extends RoutableKey> T[] sort(T[] keys)
+    {
+        Arrays.sort(keys);
+        return keys;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Arrays.hashCode(keys);
+    }
+
 }
diff --git a/accord-core/src/main/java/accord/primitives/AbstractRanges.java b/accord-core/src/main/java/accord/primitives/AbstractRanges.java
new file mode 100644
index 0000000..7b5a24c
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/AbstractRanges.java
@@ -0,0 +1,531 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.primitives;
+
+import accord.api.RoutingKey;
+import accord.utils.ArrayBuffers.ObjectBuffers;
+import accord.utils.Invariants;
+import accord.utils.SortedArrays;
+import com.google.common.collect.Iterators;
+
+import javax.annotation.Nonnull;
+import java.util.*;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.cachedRanges;
+import static accord.utils.SortedArrays.Search.FAST;
+import static accord.utils.SortedArrays.swapHighLow32b;
+
+public abstract class AbstractRanges<RS extends Routables<Range, ?>> implements Iterable<Range>, Routables<Range, RS>
+{
+    static final Range[] NO_RANGES = new Range[0];
+
+    final Range[] ranges;
+
+    AbstractRanges(@Nonnull Range[] ranges)
+    {
+        this.ranges = Invariants.nonNull(ranges);
+    }
+
+    public int indexOf(RoutableKey key)
+    {
+        return SortedArrays.binarySearch(ranges, 0, ranges.length, key, (k, r) -> -r.compareTo(k), FAST);
+    }
+
+    public int indexOf(Range find)
+    {
+        return SortedArrays.binarySearch(ranges, 0, ranges.length, find, Range::compareIntersecting, FAST);
+    }
+
+    public boolean contains(RoutableKey key)
+    {
+        return indexOf(key) >= 0;
+    }
+
+    public boolean containsAll(Routables<?, ?> that)
+    {
+        switch (that.kindOfContents())
+        {
+            default: throw new AssertionError();
+            case Key: return containsAll((AbstractKeys<?, ?>) that);
+            case Range: return containsAll((AbstractRanges<?>) that);
+        }
+    }
+
+    /**
+     * @return true iff {@code that} is fully contained within {@code this}
+     */
+    public boolean containsAll(AbstractKeys<?, ?> that)
+    {
+        if (this.isEmpty()) return that.isEmpty();
+        if (that.isEmpty()) return true;
+        return Routables.rangeFoldl(that, this, (from, to, p, v) -> v + (to - from), 0, 0, 0) == that.size();
+    }
+
+    /**
+     * @return true iff {@code that} is a subset of {@code this}
+     */
+    public boolean containsAll(AbstractRanges<?> that)
+    {
+        if (this.isEmpty()) return that.isEmpty();
+        if (that.isEmpty()) return true;
+        return ((int) supersetLinearMerge(this.ranges, that.ranges)) == that.size();
+    }
+
+    public int size()
+    {
+        return ranges.length;
+    }
+
+    @Override
+    public final Unseekable.Kind kindOfContents()
+    {
+        return Unseekable.Kind.Range;
+    }
+
+    @Override
+    public final Range get(int i)
+    {
+        return ranges[i];
+    }
+
+    @Override
+    public final boolean isEmpty()
+    {
+        return size() == 0;
+    }
+
+    public final boolean intersects(AbstractKeys<?, ?> keys)
+    {
+        return findNextIntersection(0, keys, 0) >= 0;
+    }
+
+    public final <K extends RoutableKey> boolean intersects(AbstractKeys<K, ?> keys, Predicate<? super K> matches)
+    {
+        int ri = 0, ki = 0;
+        while (true)
+        {
+            long rki = findNextIntersection(ri, keys, ki);
+            if (rki < 0)
+                return false;
+
+            ri = (int) (rki);
+            ki = (int) (rki >>> 32);
+
+            if (matches.test(keys.get(ki)))
+                return true;
+
+            ki++;
+        }
+    }
+
+    public boolean intersects(AbstractRanges<?> that)
+    {
+        return SortedArrays.findNextIntersection(this.ranges, 0, that.ranges, 0, Range::compareIntersecting) >= 0;
+    }
+
+    public boolean intersects(Range that)
+    {
+        return SortedArrays.binarySearch(ranges, 0, ranges.length, that, Range::compareIntersecting, SortedArrays.Search.FAST) >= 0;
+    }
+
+    // returns ri in low 32 bits, ki in top, or -1 if no match found
+    public long findNextIntersection(int ri, AbstractKeys<?, ?> keys, int ki)
+    {
+        return swapHighLow32b(SortedArrays.findNextIntersectionWithMultipleMatches(keys.keys, ki, ranges, ri));
+    }
+
+    // returns ki in bottom 32 bits, ri in top, or -1 if no match found
+    public long findNextIntersection(int thisi, AbstractRanges<?> that, int thati)
+    {
+        return SortedArrays.findNextIntersectionWithMultipleMatches(ranges, thisi, that.ranges, thati, Range::compareIntersecting, Range::compareIntersecting);
+    }
+
+    @Override
+    public final long findNextIntersection(int thisIndex, Routables<Range, ?> with, int withIndex)
+    {
+        return findNextIntersection(thisIndex, (AbstractRanges<?>) with, withIndex);
+    }
+
+    @Override
+    public int findNext(int thisIndex, Range find, SortedArrays.Search search)
+    {
+        return SortedArrays.exponentialSearch(ranges, thisIndex, size(), find, Range::compareIntersecting, search);
+    }
+
+    /**
+     * Returns the ranges that intersect with any of the members of the parameter.
+     * DOES NOT MODIFY THE RANGES.
+     */
+    static <RS extends AbstractRanges<?>, P> RS intersect(RS input, Unseekables<?, ?> keysOrRanges, P param, BiFunction<P, Range[], RS> constructor)
+    {
+        switch (keysOrRanges.kindOfContents())
+        {
+            default: throw new AssertionError();
+            case Range:
+            {
+                AbstractRanges<?> that = (AbstractRanges<?>) keysOrRanges;
+                Range[] result = SortedArrays.linearIntersection(input.ranges, input.ranges.length, that.ranges, that.ranges.length, Range::compareIntersecting, cachedRanges());
+                return result == input.ranges ? input : constructor.apply(param, result);
+            }
+            case Key:
+            {
+                AbstractKeys<?, ?> that = (AbstractKeys<?, ?>) keysOrRanges;
+                Range[] result = SortedArrays.linearIntersection(input.ranges, input.ranges.length, that.keys, that.keys.length, cachedRanges());
+                return result == input.ranges ? input : constructor.apply(param, result);
+            }
+        }
+    }
+
+    interface SliceConstructor<P, RS extends AbstractRanges<?>>
+    {
+        RS construct(Ranges covering, P param, Range[] ranges);
+    }
+
+    static <RS extends AbstractRanges<?>, P> RS slice(Ranges covering, AbstractRanges<?> input, P param, SliceConstructor<P, RS> constructor)
+    {
+        ObjectBuffers<Range> cachedRanges = cachedRanges();
+
+        Range[] buffer = cachedRanges.get(covering.ranges.length + input.ranges.length);
+        int bufferCount = 0;
+        try
+        {
+            int li = 0, ri = 0;
+            while (true)
+            {
+                long lri = covering.findNextIntersection(li, input, ri);
+                if (lri < 0)
+                    break;
+
+                li = (int) (lri);
+                ri = (int) (lri >>> 32);
+
+                Range l = covering.ranges[li], r = input.ranges[ri];
+                buffer[bufferCount++] = Range.slice(l, r);
+                if (l.end().compareTo(r.end()) >= 0) ri++;
+                else li++;
+            }
+            Range[] result = cachedRanges.complete(buffer, bufferCount);
+            cachedRanges.discard(buffer, bufferCount);
+            return constructor.construct(covering, param, result);
+        }
+        catch (Throwable t)
+        {
+            cachedRanges.forceDiscard(buffer, bufferCount);
+            throw t;
+        }
+    }
+
+    /**
+     * attempts a linear merge where {@code as} is expected to be a superset of {@code bs},
+     * terminating at the first indexes where this ceases to be true
+     * @return index of {@code as} in upper 32bits, {@code bs} in lower 32bits
+     *
+     * TODO: better support for merging runs of overlapping or adjacent ranges
+     */
+    static long supersetLinearMerge(Range[] as, Range[] bs)
+    {
+        int ai = 0, bi = 0;
+        out: while (ai < as.length && bi < bs.length)
+        {
+            Range a = as[ai];
+            Range b = bs[bi];
+
+            int c = a.compareIntersecting(b);
+            if (c < 0)
+            {
+                ai++;
+            }
+            else if (c > 0)
+            {
+                break;
+            }
+            else if (b.start().compareTo(a.start()) < 0)
+            {
+                break;
+            }
+            else if ((c = b.end().compareTo(a.end())) <= 0)
+            {
+                bi++;
+                if (c == 0) ai++;
+            }
+            else
+            {
+                // use a temporary counter, so that if we don't find a run of ranges that enforce the superset
+                // condition we exit at the start of the mismatch run (and permit it to be merged)
+                // TODO: use exponentialSearch
+                int tmpai = ai;
+                do
+                {
+                    if (++tmpai == as.length || !a.end().equals(as[tmpai].start()))
+                        break out;
+                    a = as[tmpai];
+                }
+                while (a.end().compareTo(b.end()) < 0);
+                bi++;
+                ai = tmpai;
+            }
+        }
+
+        return ((long)ai << 32) | bi;
+    }
+
+    interface UnionConstructor<P1, P2, RS extends AbstractRanges<?>>
+    {
+        RS construct(P1 param1, P2 param2, Range[] ranges);
+    }
+
+    public enum UnionMode { MERGE_ADJACENT, MERGE_OVERLAPPING }
+
+    /**
+     * @return the union of {@code left} and {@code right}, returning one of the two inputs if possible
+     */
+    static <P1, P2, RS extends AbstractRanges<?>> RS union(UnionMode mode, AbstractRanges<?> left, AbstractRanges<?> right, P1 param1, P2 param2, UnionConstructor<P1, P2, RS> constructor)
+    {
+        if (left == right || right.isEmpty()) return constructor.construct(param1, param2, left.ranges);
+        if (left.isEmpty()) return constructor.construct(param1, param2, right.ranges);
+
+        Range[] as = left.ranges, bs = right.ranges;
+        {
+            // make sure as/ai represent the ranges right might fully contain the other
+            int c = as[0].start().compareTo(bs[0].start());
+            if (c > 0 || c == 0 && as[as.length - 1].end().compareTo(bs[bs.length - 1].end()) < 0)
+            {
+                Range[] tmp = as; as = bs; bs = tmp;
+            }
+        }
+
+        int ai, bi; {
+            long tmp = supersetLinearMerge(as, bs);
+            ai = (int)(tmp >>> 32);
+            bi = (int)tmp;
+        }
+
+        if (bi == bs.length)
+            return constructor.construct(param1, param2, (as == left.ranges ? left : right).ranges);
+
+        // TODO (now): caching
+        Range[] result = new Range[as.length + (bs.length - bi)];
+        int resultCount;
+        switch (mode)
+        {
+            default: throw new AssertionError();
+            case MERGE_ADJACENT:
+                resultCount = copyAndMergeTouching(as, 0, result, 0, ai);
+                break;
+            case MERGE_OVERLAPPING:
+                System.arraycopy(as, 0, result, 0, ai);
+                resultCount = ai;
+        }
+
+        while (ai < as.length && bi < bs.length)
+        {
+            Range a = as[ai];
+            Range b = bs[bi];
+
+            int c = a.compareIntersecting(b);
+            if (c < 0)
+            {
+                result[resultCount++] = a;
+                ai++;
+            }
+            else if (c > 0)
+            {
+                result[resultCount++] = b;
+                bi++;
+            }
+            else
+            {
+                // TODO: we don't seem to currently merge adjacent (but non-overlapping)
+                RoutingKey start = a.start().compareTo(b.start()) <= 0 ? a.start() : b.start();
+                RoutingKey end = a.end().compareTo(b.end()) >= 0 ? a.end() : b.end();
+                ai++;
+                bi++;
+                while (ai < as.length || bi < bs.length)
+                {
+                    Range min;
+                    if (ai == as.length) min = bs[bi];
+                    else if (bi == bs.length) min = a = as[ai];
+                    else min = as[ai].start().compareTo(bs[bi].start()) < 0 ? a = as[ai] : bs[bi];
+                    if (min.start().compareTo(end) > 0)
+                        break;
+                    if (min.end().compareTo(end) > 0)
+                        end = min.end();
+                    if (a == min) ai++;
+                    else bi++;
+                }
+                result[resultCount++] = a.subRange(start, end);
+            }
+        }
+
+        while (ai < as.length)
+            result[resultCount++] = as[ai++];
+
+        while (bi < bs.length)
+            result[resultCount++] = bs[bi++];
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+
+        return constructor.construct(param1, param2, result);
+    }
+
+    @Override
+    public String toString()
+    {
+        return Arrays.toString(ranges);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Arrays.hashCode(ranges);
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        if (that == null || this.getClass() != that.getClass())
+            return false;
+        return Arrays.equals(this.ranges, ((AbstractRanges<?>) that).ranges);
+    }
+
+    @Override
+    public Iterator<Range> iterator()
+    {
+        return Iterators.forArray(ranges);
+    }
+
+    static <RS extends AbstractRanges<?>> RS mergeTouching(RS input, Function<Range[], RS> constructor)
+    {
+        Range[] ranges = input.ranges;
+        if (ranges.length == 0)
+            return input;
+
+        // TODO: use cache
+        ObjectBuffers<Range> cachedKeyRanges = cachedRanges();
+        Range[] buffer = cachedKeyRanges.get(ranges.length);
+        try
+        {
+            int count = copyAndMergeTouching(ranges, 0, buffer, 0, ranges.length);
+            if (count == buffer.length)
+                return input;
+            Range[] result = cachedKeyRanges.complete(buffer, count);
+            cachedKeyRanges.discard(buffer, count);
+            return constructor.apply(result);
+        }
+        catch (Throwable t)
+        {
+            cachedKeyRanges.forceDiscard(buffer, ranges.length);
+            throw t;
+        }
+    }
+
+    static int copyAndMergeTouching(Range[] src, int srcPosition, Range[] trg, int trgPosition, int srcCount)
+    {
+        if (srcCount == 0)
+            return 0;
+
+        int count = 0;
+        Range prev = src[srcPosition];
+        RoutingKey end = prev.end();
+        for (int i = 1 ; i < srcCount ; ++i)
+        {
+            Range next = src[srcPosition + i];
+            if (!end.equals(next.start()))
+            {
+                trg[trgPosition + count++] = maybeUpdateEnd(prev, end);
+                prev = next;
+            }
+            end = next.end();
+        }
+        trg[trgPosition + count++] = maybeUpdateEnd(prev, end);
+        return count;
+    }
+
+    static Range maybeUpdateEnd(Range range, RoutingKey withEnd)
+    {
+        return withEnd == range.end() ? range : range.subRange(range.start(), withEnd);
+    }
+
+    static <RS extends AbstractRanges<?>> RS of(Function<Range[], RS> constructor, Range... ranges)
+    {
+        if (ranges.length == 0)
+            return constructor.apply(NO_RANGES);
+
+        return sortAndDeoverlap(constructor, ranges, ranges.length);
+    }
+
+    static <RS extends AbstractRanges<?>> RS sortAndDeoverlap(Function<Range[], RS> constructor, Range[] ranges, int count)
+    {
+        if (count == 0)
+            return constructor.apply(NO_RANGES);
+
+        if (count == 1)
+        {
+            if (ranges.length == 1)
+                return constructor.apply(ranges);
+
+            return constructor.apply(Arrays.copyOf(ranges, count));
+        }
+
+        Arrays.sort(ranges, 0, count, Comparator.comparing(Range::start));
+        Range prev = ranges[0];
+        int removed = 0;
+        for (int i = 1 ; i < count ; ++i)
+        {
+            Range next = ranges[i];
+            if (prev.end().compareTo(next.start()) > 0)
+            {
+                prev = prev.subRange(prev.start(), next.start());
+                if (prev.end().compareTo(next.end()) >= 0)
+                {
+                    removed++;
+                }
+                else if (removed > 0)
+                {
+                    ranges[i - removed] = prev = next.subRange(prev.end(), next.end());
+                }
+            }
+            else if (removed > 0)
+            {
+                ranges[i - removed] = prev = next;
+            }
+        }
+
+        count -= removed;
+        if (count != ranges.length)
+            ranges = Arrays.copyOf(ranges, count);
+
+        return constructor.apply(ranges);
+    }
+
+    static <RS extends AbstractRanges<?>> RS ofSortedAndDeoverlapped(Function<Range[], RS> constructor, Range... ranges)
+    {
+        for (int i = 1 ; i < ranges.length ; ++i)
+        {
+            if (ranges[i - 1].end().compareTo(ranges[i].start()) > 0)
+                throw new IllegalArgumentException(Arrays.toString(ranges) + " is not correctly sorted or deoverlapped");
+        }
+
+        return constructor.apply(ranges);
+    }
+}
\ No newline at end of file
diff --git a/accord-core/src/main/java/accord/primitives/AbstractRoutableKeys.java b/accord-core/src/main/java/accord/primitives/AbstractRoutableKeys.java
new file mode 100644
index 0000000..a7ae455
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/AbstractRoutableKeys.java
@@ -0,0 +1,11 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+
+public abstract class AbstractRoutableKeys<KS extends Unseekables<RoutingKey, ?>> extends AbstractKeys<RoutingKey, KS> implements Iterable<RoutingKey>, Unseekables<RoutingKey, KS>
+{
+    AbstractRoutableKeys(RoutingKey[] keys)
+    {
+        super(keys);
+    }
+}
diff --git a/accord-core/src/main/java/accord/primitives/AbstractRoute.java b/accord-core/src/main/java/accord/primitives/AbstractRoute.java
deleted file mode 100644
index 7989fb4..0000000
--- a/accord-core/src/main/java/accord/primitives/AbstractRoute.java
+++ /dev/null
@@ -1,37 +0,0 @@
-package accord.primitives;
-
-import javax.annotation.Nullable;
-
-import com.google.common.base.Preconditions;
-
-import accord.api.RoutingKey;
-
-public abstract class AbstractRoute extends RoutingKeys
-{
-    public final RoutingKey homeKey;
-
-    public AbstractRoute(RoutingKey[] keys, RoutingKey homeKey)
-    {
-        super(keys);
-        Preconditions.checkNotNull(homeKey);
-        this.homeKey = homeKey;
-    }
-
-    public abstract AbstractRoute union(AbstractRoute that);
-
-    public abstract boolean covers(KeyRanges ranges);
-
-    public abstract PartialRoute slice(KeyRanges ranges);
-
-    /**
-     * Requires that the ranges are fully covered by this collection
-     */
-    public abstract PartialRoute sliceStrict(KeyRanges ranges);
-
-    public static AbstractRoute merge(@Nullable AbstractRoute prefer, @Nullable AbstractRoute defer)
-    {
-        if (defer == null) return prefer;
-        if (prefer == null) return defer;
-        return prefer.union(defer);
-    }
-}
diff --git a/accord-core/src/main/java/accord/primitives/Deps.java b/accord-core/src/main/java/accord/primitives/Deps.java
index e563162..fef69b9 100644
--- a/accord-core/src/main/java/accord/primitives/Deps.java
+++ b/accord-core/src/main/java/accord/primitives/Deps.java
@@ -27,12 +27,14 @@ import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 import accord.api.Key;
+import accord.utils.ArrayBuffers;
 import accord.api.RoutingKey;
 import accord.utils.SortedArrays;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 import static accord.utils.ArrayBuffers.*;
 import static accord.utils.SortedArrays.*;
+import static accord.utils.SortedArrays.Search.FAST;
 import static accord.utils.Utils.listOf;
 
 /**
@@ -83,8 +85,13 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         return new OrderedBuilder(hasOrderedTxnId);
     }
 
+    // TODO: cache this object to reduce setup/teardown and allocation
     public static abstract class AbstractOrderedBuilder<T extends Deps> implements AutoCloseable
     {
+        final ObjectBuffers<TxnId> cachedTxnIds = cachedTxnIds();
+        final ObjectBuffers<Key> cachedKeys = cachedKeys();
+        final IntBuffers cachedInts = cachedInts();
+
         final boolean hasOrderedTxnId;
         Key[] keys;
         int[] keyLimits;
@@ -96,10 +103,10 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
 
         public AbstractOrderedBuilder(boolean hasOrderedTxnId)
         {
-            this.keys = cachedKeys().get(16);
-            this.keyLimits = cachedInts().getInts(keys.length);
+            this.keys = cachedKeys.get(16);
+            this.keyLimits = cachedInts.getInts(keys.length);
             this.hasOrderedTxnId = hasOrderedTxnId;
-            this.keyToTxnId = cachedTxnIds().get(16);
+            this.keyToTxnId = cachedTxnIds.get(16);
         }
 
         public boolean isEmpty()
@@ -124,13 +131,13 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
 
             if (keyCount == keys.length)
             {
-                Key[] newKeys = cachedKeys().get(keyCount * 2);
+                Key[] newKeys = cachedKeys.get(keyCount * 2);
                 System.arraycopy(keys, 0, newKeys, 0, keyCount);
-                cachedKeys().forceDiscard(keys, keyCount);
+                cachedKeys.forceDiscard(keys, keyCount);
                 keys = newKeys;
-                int[] newKeyLimits = cachedInts().getInts(keyCount * 2);
+                int[] newKeyLimits = cachedInts.getInts(keyCount * 2);
                 System.arraycopy(keyLimits, 0, newKeyLimits, 0, keyCount);
-                cachedInts().forceDiscard(keyLimits);
+                cachedInts.forceDiscard(keyLimits, keyCount);
                 keyLimits = newKeyLimits;
             }
             keys[keyCount++] = key;
@@ -175,9 +182,9 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
 
             if (totalCount >= keyToTxnId.length)
             {
-                TxnId[] newTxnIds = cachedTxnIds().get(keyToTxnId.length * 2);
+                TxnId[] newTxnIds = cachedTxnIds.get(keyToTxnId.length * 2);
                 System.arraycopy(keyToTxnId, 0, newTxnIds, 0, totalCount);
-                cachedTxnIds().forceDiscard(keyToTxnId, totalCount);
+                cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
                 keyToTxnId = newTxnIds;
             }
 
@@ -191,7 +198,7 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
 
             finishKey();
 
-            TxnId[] uniqueTxnId = cachedTxnIds().get(totalCount);
+            TxnId[] uniqueTxnId = cachedTxnIds.get(totalCount);
             System.arraycopy(keyToTxnId, 0, uniqueTxnId, 0, totalCount);
             Arrays.sort(uniqueTxnId, 0, totalCount);
             int txnIdCount = 1;
@@ -201,8 +208,8 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
                     uniqueTxnId[txnIdCount++] = uniqueTxnId[i];
             }
 
-            TxnId[] txnIds = cachedTxnIds().complete(uniqueTxnId, txnIdCount);
-            cachedTxnIds().discard(uniqueTxnId, totalCount);
+            TxnId[] txnIds = cachedTxnIds.complete(uniqueTxnId, txnIdCount);
+            cachedTxnIds.discard(uniqueTxnId, totalCount);
 
             int[] result = new int[keyCount + totalCount];
             int offset = keyCount;
@@ -217,7 +224,7 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
                 }, keyCount, offset, -1);
             }
 
-            return build(Keys.ofSortedUnchecked(cachedKeys().complete(keys, keyCount)), txnIds, result);
+            return build(Keys.ofSortedUnchecked(cachedKeys.complete(keys, keyCount)), txnIds, result);
         }
 
         abstract T build(Keys keys, TxnId[] txnIds, int[] keyToTxnId);
@@ -225,9 +232,9 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         @Override
         public void close()
         {
-            cachedKeys().discard(keys, keyCount);
-            cachedInts().forceDiscard(keyLimits);
-            cachedTxnIds().forceDiscard(keyToTxnId, totalCount);
+            cachedKeys.discard(keys, keyCount);
+            cachedInts.forceDiscard(keyLimits, keyCount);
+            cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
         }
     }
 
@@ -296,9 +303,9 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
             }
             else
             {
-                Preconditions.checkState(keys == bufKeys && keysLength == bufKeysLength);
-                Preconditions.checkState(txnIds == bufTxnIds && txnIdsLength == bufTxnIdsLength);
-                Preconditions.checkState(outLength == bufLength);
+                Invariants.checkState(keys == bufKeys && keysLength == bufKeysLength);
+                Invariants.checkState(txnIds == bufTxnIds && txnIdsLength == bufTxnIdsLength);
+                Invariants.checkState(outLength == bufLength);
             }
             return null;
         }
@@ -324,9 +331,9 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
             );
             if (buf == deps.keyToTxnId)
             {
-                Preconditions.checkState(deps.keys.keys == bufKeys && deps.keys.keys.length == bufKeysLength);
-                Preconditions.checkState(deps.txnIds == bufTxnIds && deps.txnIds.length == bufTxnIdsLength);
-                Preconditions.checkState(deps.keyToTxnId.length == bufLength);
+                Invariants.checkState(deps.keys.keys == bufKeys && deps.keys.keys.length == bufKeysLength);
+                Invariants.checkState(deps.txnIds == bufTxnIds && deps.txnIds.length == bufTxnIdsLength);
+                Invariants.checkState(deps.keyToTxnId.length == bufLength);
                 from = deps;
             }
         }
@@ -439,7 +446,7 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
     }
 
     // TODO: offer option of computing the maximal KeyRanges that covers the same set of keys as covered by the parameter
-    public PartialDeps slice(KeyRanges ranges)
+    public PartialDeps slice(Ranges ranges)
     {
         if (isEmpty())
             return new PartialDeps(ranges, keys, txnIds, keyToTxnId);
@@ -456,7 +463,7 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         int offset = select.size();
         for (int j = 0 ; j < select.size() ; ++j)
         {
-            int findi = keys.findNext(select.get(j), i);
+            int findi = keys.findNext(i, select.get(j), FAST);
             if (findi < 0)
                 continue;
 
@@ -471,7 +478,7 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         offset = select.size();
         for (int j = 0 ; j < select.size() ; ++j)
         {
-            int findi = keys.findNext(select.get(j), i);
+            int findi = keys.findNext(i, select.get(j), FAST);
             if (findi >= 0)
             {
                 i = findi;
@@ -497,33 +504,45 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
      */
     private static TxnId[] trimUnusedTxnId(Keys keys, TxnId[] txnIds, int[] keysToTxnId)
     {
-        int[] remapTxnId = new int[txnIds.length];
-        // on init all values got set to 0, so use 1 to define that the id exists in the index
-        for (int i = keys.size() ; i < keysToTxnId.length ; ++i)
-            remapTxnId[keysToTxnId[i]] = 1;
-
-        int offset = 0;
-        for (int i = 0 ; i < remapTxnId.length ; ++i)
+        IntBuffers cache = ArrayBuffers.cachedInts();
+        // we use remapTxnId twice:
+        //  - first we use the end to store a bitmap of those TxnId we are actually using
+        //  - then we use it to store the remap index (incrementally replacing the bitmap)
+        int bitMapOffset = txnIds.length + 1 - (txnIds.length+31)/32;
+        int[] remapTxnId = cache.getInts(txnIds.length + 1);
+        try
         {
-            if (remapTxnId[i] == 1) remapTxnId[i] = offset++;
-            else remapTxnId[i] = -1;
-        }
+            Arrays.fill(remapTxnId, bitMapOffset, txnIds.length + 1, 0);
+            for (int i = keys.size() ; i < keysToTxnId.length ; ++i)
+                setBit(remapTxnId, bitMapOffset, keysToTxnId[i]);
 
-        TxnId[] result = txnIds;
-        if (offset < remapTxnId.length)
-        {
-            result = new TxnId[offset];
+            int offset = 0;
             for (int i = 0 ; i < txnIds.length ; ++i)
             {
-                if (remapTxnId[i]>= 0)
-                    result[remapTxnId[i]] = txnIds[i];
+                if (hasSetBit(remapTxnId, bitMapOffset, i)) remapTxnId[i] = offset++;
+                else remapTxnId[i] = -1;
             }
-            // Update keysToTxnId to point to the new remapped TxnId offsets
-            for (int i = keys.size() ; i < keysToTxnId.length ; ++i)
-                keysToTxnId[i] = remapTxnId[keysToTxnId[i]];
-        }
 
-        return result;
+            TxnId[] result = txnIds;
+            if (offset < txnIds.length)
+            {
+                result = new TxnId[offset];
+                for (int i = 0 ; i < txnIds.length ; ++i)
+                {
+                    if (remapTxnId[i] >= 0)
+                        result[remapTxnId[i]] = txnIds[i];
+                }
+                // Update keysToTxnId to point to the new remapped TxnId offsets
+                for (int i = keys.size() ; i < keysToTxnId.length ; ++i)
+                    keysToTxnId[i] = remapTxnId[keysToTxnId[i]];
+            }
+
+            return result;
+        }
+        finally
+        {
+            cache.forceDiscard(remapTxnId, txnIds.length);
+        }
     }
 
     public Deps with(Deps that)
@@ -828,9 +847,9 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
             if (out != null)
                 intBuffers.discard(out, outLength);
             if (remapLeft != null)
-                intBuffers.forceDiscard(remapLeft);
+                intBuffers.forceDiscard(remapLeft, leftValuesLength);
             if (remapRight != null)
-                intBuffers.forceDiscard(remapRight);
+                intBuffers.forceDiscard(remapRight, rightValuesLength);
         }
     }
 
@@ -865,48 +884,64 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         if (isEmpty())
             return this;
 
-        int[] remapTxnIds = new int[txnIds.length];
-        TxnId[] txnIds; {
-            int count = 0;
-            for (int i = 0 ; i < this.txnIds.length ; ++i)
-            {
-                if (remove.test(this.txnIds[i])) remapTxnIds[i] = -1;
-                else remapTxnIds[i] = count++;
-            }
+        IntBuffers cache = ArrayBuffers.cachedInts();
+        int[] remapTxnIds = cache.getInts(txnIds.length);
+        int[] keyToTxnId = null;
+        int o = 0;
+        try
+        {
+            TxnId[] txnIds; {
+                int count = 0;
+                for (int i = 0 ; i < this.txnIds.length ; ++i)
+                {
+                    if (remove.test(this.txnIds[i])) remapTxnIds[i] = -1;
+                    else remapTxnIds[i] = count++;
+                }
 
-            if (count == remapTxnIds.length)
-                return this;
+                if (count == this.txnIds.length)
+                    return this;
 
-            if (count == 0)
-                return NONE;
+                if (count == 0)
+                    return NONE;
 
-            txnIds = new TxnId[count];
-            for (int i = 0 ; i < this.txnIds.length ; ++i)
+                txnIds = new TxnId[count];
+                for (int i = 0 ; i < this.txnIds.length ; ++i)
+                {
+                    if (remapTxnIds[i] >= 0)
+                        txnIds[remapTxnIds[i]] = this.txnIds[i];
+                }
+            }
+
+            keyToTxnId = cache.getInts(this.keyToTxnId.length);
+            int k = 0, i = keys.size();
+            o = i;
+            while (i < this.keyToTxnId.length)
             {
-                if (remapTxnIds[i] >= 0)
-                    txnIds[remapTxnIds[i]] = this.txnIds[i];
+                while (this.keyToTxnId[k] == i)
+                    keyToTxnId[k++] = o;
+
+                int remapped = remapTxnIds[this.keyToTxnId[i]];
+                if (remapped >= 0)
+                    keyToTxnId[o++] = remapped;
+                ++i;
             }
-        }
 
-        int[] keyToTxnId = new int[this.keyToTxnId.length];
-        int k = 0, i = keys.size(), o = i;
-        while (i < this.keyToTxnId.length)
-        {
-            while (this.keyToTxnId[k] == i)
+            while (k < keys.size())
                 keyToTxnId[k++] = o;
 
-            int remapped = remapTxnIds[this.keyToTxnId[i]];
-            if (remapped >= 0)
-                keyToTxnId[o++] = remapped;
-            ++i;
+            int[] result = cache.complete(keyToTxnId, o);
+            cache.discard(keyToTxnId, o);
+            return new Deps(keys, txnIds, result);
+        }
+        catch (Throwable t)
+        {
+            cache.forceDiscard(keyToTxnId, o);
+            throw t;
+        }
+        finally
+        {
+            cache.forceDiscard(remapTxnIds, txnIds.length);
         }
-
-        while (k < keys.size())
-            keyToTxnId[k++] = o;
-
-        keyToTxnId = Arrays.copyOf(keyToTxnId, o);
-
-        return new Deps(keys, txnIds, keyToTxnId);
     }
 
     public boolean contains(TxnId txnId)
@@ -940,11 +975,20 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         return Keys.of(result);
     }
 
-    public RoutingKeys someRoutingKeys(TxnId txnId)
+    public Unseekables<RoutingKey, ?> someRoutables(TxnId txnId)
+    {
+        return toUnseekables(txnId, array -> {
+            if (array.length == 0)
+                throw new IllegalStateException("Cannot create a RouteFragment without any keys");
+            return new RoutingKeys(array);
+        });
+    }
+
+    private <R> R toUnseekables(TxnId txnId, Function<RoutingKey[], R> constructor)
     {
         int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
         if (txnIdIndex < 0)
-            return RoutingKeys.EMPTY;
+            constructor.apply(RoutingKeys.EMPTY.keys);
 
         ensureTxnIdToKey();
 
@@ -952,20 +996,20 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         int end = txnIdToKey[txnIdIndex];
         RoutingKey[] result = new RoutingKey[end - start];
         if (start == end)
-            return RoutingKeys.EMPTY;
+            constructor.apply(RoutingKeys.EMPTY.keys);
 
-        result[0] = keys.get(txnIdToKey[start]).toRoutingKey();
+        result[0] = keys.get(txnIdToKey[start]).toUnseekable();
         int resultCount = 1;
         for (int i = start + 1 ; i < end ; ++i)
         {
-            RoutingKey next = keys.get(txnIdToKey[i]).toRoutingKey();
+            RoutingKey next = keys.get(txnIdToKey[i]).toUnseekable();
             if (!next.equals(result[resultCount - 1]))
                 result[resultCount++] = next;
         }
 
         if (resultCount < result.length)
             result = Arrays.copyOf(result, resultCount);
-        return new RoutingKeys(result);
+        return constructor.apply(result);
     }
 
     void ensureTxnIdToKey()
@@ -1008,9 +1052,9 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         return trg;
     }
 
-    public void forEachOn(KeyRanges ranges, Predicate<Key> include, BiConsumer<Key, TxnId> forEach)
+    public void forEachOn(Ranges ranges, Predicate<Key> include, BiConsumer<Key, TxnId> forEach)
     {
-        keys.foldl(ranges, (index, key, value) -> {
+        Routables.foldl(keys, ranges, (index, key, value) -> {
             if (!include.test(key))
                 return null;
 
@@ -1024,12 +1068,12 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
     }
 
     /**
-     * For each {@link TxnId} that references a key within the {@link KeyRanges}; the {@link TxnId} will be seen exactly once.
+     * For each {@link TxnId} that references a key within the {@link Ranges}; the {@link TxnId} will be seen exactly once.
      * @param ranges to match on
      * @param include function to say if a key should be used or not
      * @param forEach function to call on each unique {@link TxnId}
      */
-    public void forEachOn(KeyRanges ranges, Predicate<Key> include, Consumer<TxnId> forEach)
+    public void forEachOn(Ranges ranges, Predicate<? super Key> include, Consumer<TxnId> forEach)
     {
         // Find all keys within the ranges, but record existence within an int64 bitset.  Since the bitset is limited
         // to 64, this search must be called multiple times searching for different TxnIds in txnIds; this also has
@@ -1038,7 +1082,7 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         // does not rely on this ordering.
         for (int offset = 0 ; offset < txnIds.length ; offset += 64)
         {
-            long bitset = keys.foldl(ranges, (keyIndex, key, off, value) -> {
+            long bitset = Routables.foldl(keys, ranges, (keyIndex, key, off, value) -> {
                 if (!include.test(key))
                     return value;
 
@@ -1278,4 +1322,14 @@ public class Deps implements Iterable<Map.Entry<Key, TxnId>>
         }
     }
 
+    private static void setBit(int[] array, int offset, int index)
+    {
+        array[offset + index / 32] |= (1 << (index & 31));
+    }
+
+    private static boolean hasSetBit(int[] array, int offset, int index)
+    {
+        return (array[offset + index / 32] & (1 << (index & 31))) != 0;
+    }
+
 }
diff --git a/accord-core/src/main/java/accord/primitives/FullKeyRoute.java b/accord-core/src/main/java/accord/primitives/FullKeyRoute.java
new file mode 100644
index 0000000..c6abd45
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/FullKeyRoute.java
@@ -0,0 +1,71 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+import accord.utils.Invariants;
+
+public class FullKeyRoute extends KeyRoute implements FullRoute<RoutingKey>
+{
+    public static class SerializationSupport
+    {
+        public static FullKeyRoute create(RoutingKey homeKey, RoutingKey[] keys)
+        {
+            return new FullKeyRoute(homeKey, keys);
+        }
+    }
+
+    public FullKeyRoute(RoutingKey homeKey, RoutingKey[] keys)
+    {
+        super(homeKey, keys);
+    }
+
+    @Override
+    public UnseekablesKind kind()
+    {
+        return UnseekablesKind.FullKeyRoute;
+    }
+
+    @Override
+    public boolean covers(Ranges ranges)
+    {
+        return true;
+    }
+
+    @Override
+    public boolean intersects(AbstractRanges<?> ranges)
+    {
+        // TODO (now): remove this in favour of parent implementation - ambiguous at present
+        return true;
+    }
+
+    @Override
+    public FullKeyRoute with(RoutingKey withKey)
+    {
+        Invariants.checkArgument(contains(withKey));
+        // TODO (now): remove this in favour of parent implementation - ambiguous at present
+        return this;
+    }
+
+    @Override
+    public PartialKeyRoute slice(Ranges ranges)
+    {
+        return new PartialKeyRoute(ranges, homeKey, slice(ranges, RoutingKey[]::new));
+    }
+
+    @Override
+    public PartialKeyRoute sliceStrict(Ranges ranges)
+    {
+        return slice(ranges);
+    }
+
+    public FullKeyRoute toMaximalUnseekables()
+    {
+        return this;
+    }
+
+    @Override
+    public String toString()
+    {
+        return "{homeKey:" + homeKey + ',' + super.toString() + '}';
+    }
+
+}
diff --git a/accord-core/src/main/java/accord/primitives/FullRangeRoute.java b/accord-core/src/main/java/accord/primitives/FullRangeRoute.java
new file mode 100644
index 0000000..7e39fd7
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/FullRangeRoute.java
@@ -0,0 +1,70 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+import accord.utils.Invariants;
+
+public class FullRangeRoute extends RangeRoute implements FullRoute<Range>
+{
+    public static class SerializationSupport
+    {
+        public static FullRangeRoute create(RoutingKey homeKey, Range[] ranges)
+        {
+            return new FullRangeRoute(homeKey, ranges);
+        }
+    }
+
+    public FullRangeRoute(RoutingKey homeKey, Range[] ranges)
+    {
+        super(homeKey, ranges);
+    }
+
+    @Override
+    public UnseekablesKind kind()
+    {
+        return UnseekablesKind.FullRangeRoute;
+    }
+
+    @Override
+    public boolean covers(Ranges ranges)
+    {
+        return true;
+    }
+
+    @Override
+    public boolean intersects(AbstractRanges<?> ranges)
+    {
+        return true;
+    }
+
+    @Override
+    public FullRangeRoute with(RoutingKey withKey)
+    {
+        Invariants.checkArgument(contains(withKey));
+        // TODO (now): remove this in favour of parent implementation - ambiguous at present
+        return this;
+    }
+
+    @Override
+    public PartialRangeRoute slice(Ranges ranges)
+    {
+        return slice(ranges, this, homeKey, PartialRangeRoute::new);
+    }
+
+    @Override
+    public PartialRangeRoute sliceStrict(Ranges ranges)
+    {
+        return slice(ranges);
+    }
+
+    public FullRangeRoute toMaximalUnseekables()
+    {
+        return this;
+    }
+
+    @Override
+    public String toString()
+    {
+        return "{homeKey:" + homeKey + ',' + super.toString() + '}';
+    }
+
+}
diff --git a/accord-core/src/main/java/accord/primitives/FullRoute.java b/accord-core/src/main/java/accord/primitives/FullRoute.java
new file mode 100644
index 0000000..c4d0c4e
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/FullRoute.java
@@ -0,0 +1,6 @@
+package accord.primitives;
+
+public interface FullRoute<T extends Unseekable> extends Route<T>, Unseekables<T, Route<T>>
+{
+    @Override default FullRoute<T> union(Route<T> route) { return this; }
+}
diff --git a/accord-core/src/main/java/accord/primitives/KeyRanges.java b/accord-core/src/main/java/accord/primitives/KeyRanges.java
deleted file mode 100644
index 0d3de3d..0000000
--- a/accord-core/src/main/java/accord/primitives/KeyRanges.java
+++ /dev/null
@@ -1,494 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.primitives;
-
-import accord.api.RoutingKey;
-import accord.utils.SortedArrays;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterators;
-
-import java.util.*;
-import java.util.function.Predicate;
-
-import static accord.utils.SortedArrays.Search.FAST;
-import static accord.utils.SortedArrays.swapHighLow32b;
-import static accord.utils.Utils.toArray;
-
-public class KeyRanges implements Iterable<KeyRange>
-{
-    public static final KeyRanges EMPTY = ofSortedAndDeoverlappedUnchecked(new KeyRange[0]);
-
-    final KeyRange[] ranges;
-
-    private KeyRanges(KeyRange[] ranges)
-    {
-        Preconditions.checkNotNull(ranges);
-        this.ranges = ranges;
-    }
-
-    public KeyRanges(List<KeyRange> ranges)
-    {
-        this(toArray(ranges, KeyRange[]::new));
-    }
-
-    @Override
-    public String toString()
-    {
-        return Arrays.toString(ranges);
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        KeyRanges ranges1 = (KeyRanges) o;
-        return Arrays.equals(ranges, ranges1.ranges);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return Arrays.hashCode(ranges);
-    }
-
-    @Override
-    public Iterator<KeyRange> iterator()
-    {
-        return Iterators.forArray(ranges);
-    }
-
-    // TODO: reconsider users of this method, in light of newer facilities like foldl, findNext etc
-    public int rangeIndexForKey(int lowerBound, int upperBound, RoutingKey key)
-    {
-        return SortedArrays.binarySearch(ranges, lowerBound, upperBound, key, (k, r) -> r.compareKey(k), FAST);
-    }
-
-    public int rangeIndexForKey(RoutingKey key)
-    {
-        return rangeIndexForKey(0, ranges.length, key);
-    }
-
-    public boolean contains(RoutingKey key)
-    {
-        return rangeIndexForKey(key) >= 0;
-    }
-
-    public boolean containsAll(AbstractKeys<?, ?> keys)
-    {
-        return keys.rangeFoldl(this, (from, to, p, v) -> v + (to - from), 0, 0, 0) == keys.size();
-    }
-
-    public int size()
-    {
-        return ranges.length;
-    }
-
-    public KeyRange get(int i)
-    {
-        return ranges[i];
-    }
-
-    public boolean isEmpty()
-    {
-        return size() == 0;
-    }
-
-    public KeyRanges select(int[] indexes)
-    {
-        KeyRange[] selection = new KeyRange[indexes.length];
-        for (int i=0; i<indexes.length; i++)
-            selection[i] = ranges[indexes[i]];
-        return ofSortedAndDeoverlapped(selection);
-    }
-
-    public boolean intersects(AbstractKeys<?, ?> keys)
-    {
-        return findNextIntersection(0, keys, 0) >= 0;
-    }
-
-    public <K extends RoutingKey> boolean intersects(AbstractKeys<K, ?> keys, Predicate<? super K> matches)
-    {
-        int ri = 0, ki = 0;
-        while (true)
-        {
-            long rki = findNextIntersection(ri, keys, ki);
-            if (rki < 0)
-                return false;
-
-            ri = (int) (rki >>> 32);
-            ki = (int) (rki);
-
-            if (matches.test(keys.get(ki)))
-                return true;
-
-            ki++;
-        }
-    }
-
-    public boolean intersects(KeyRanges that)
-    {
-        return SortedArrays.findNextIntersection(this.ranges, 0, that.ranges, 0, KeyRange::compareIntersecting) >= 0;
-    }
-
-    public int findFirstKey(AbstractKeys<?, ?> keys)
-    {
-        return findNextKey(0, keys, 0);
-    }
-
-    public int findNextKey(int ri, AbstractKeys<?, ?> keys, int ki)
-    {
-        return (int) findNextIntersection(ri, keys, ki);
-    }
-
-    // returns ri in top 32 bits, ki in bottom, or -1 if no match found
-    // TODO (now): inconsistent bits order vs SortedArrays
-    public long findNextIntersection(int ri, AbstractKeys<?, ?> keys, int ki)
-    {
-        return swapHighLow32b(SortedArrays.findNextIntersectionWithMultipleMatches(keys.keys, ki, ranges, ri));
-    }
-
-    public int findFirstKey(RoutingKey[] keys)
-    {
-        return findNextKey(0, keys, 0);
-    }
-
-    public int findNextKey(int ri, RoutingKey[] keys, int ki)
-    {
-        return (int) (findNextIntersection(ri, keys, ki));
-    }
-
-    // returns ri in top 32 bits, ki in bottom, or -1 if no match found
-    public long findNextIntersection(int ri, RoutingKey[] keys, int ki)
-    {
-        return SortedArrays.findNextIntersectionWithMultipleMatches(keys, ki, ranges, ri);
-    }
-
-    /**
-     * Subtracts the given set of key ranges from this
-     * @param that
-     * @return
-     */
-    public KeyRanges difference(KeyRanges that)
-    {
-        if (that == this)
-            return KeyRanges.EMPTY;
-
-        List<KeyRange> result = new ArrayList<>(this.size() + that.size());
-        int thatIdx = 0;
-
-        for (int thisIdx=0; thisIdx<this.size(); thisIdx++)
-        {
-            KeyRange thisRange = this.ranges[thisIdx];
-            while (thatIdx < that.size())
-            {
-                KeyRange thatRange = that.ranges[thatIdx];
-
-                int cmp = thisRange.compareIntersecting(thatRange);
-                if (cmp > 0)
-                {
-                    thatIdx++;
-                    continue;
-                }
-                if (cmp < 0) break;
-
-                int scmp = thisRange.start().compareTo(thatRange.start());
-                int ecmp = thisRange.end().compareTo(thatRange.end());
-
-                if (scmp < 0)
-                    result.add(thisRange.subRange(thisRange.start(), thatRange.start()));
-
-                if (ecmp <= 0)
-                {
-                    thisRange = null;
-                    break;
-                }
-                else
-                {
-                    thisRange = thisRange.subRange(thatRange.end(), thisRange.end());
-                    thatIdx++;
-                }
-            }
-            if (thisRange != null)
-                result.add(thisRange);
-        }
-        return new KeyRanges(toArray(result, KeyRange[]::new));
-    }
-
-    /**
-     * attempts a linear merge where {@code as} is expected to be a superset of {@code bs},
-     * terminating at the first indexes where this ceases to be true
-     * @return index of {@code as} in upper 32bits, {@code bs} in lower 32bits
-     *
-     * TODO: better support for merging runs of overlapping or adjacent ranges
-     */
-    private static long supersetLinearMerge(KeyRange[] as, KeyRange[] bs)
-    {
-        int ai = 0, bi = 0;
-        out: while (ai < as.length && bi < bs.length)
-        {
-            KeyRange a = as[ai];
-            KeyRange b = bs[bi];
-
-            int c = a.compareIntersecting(b);
-            if (c < 0)
-            {
-                ai++;
-            }
-            else if (c > 0)
-            {
-                break;
-            }
-            else if (b.start().compareTo(a.start()) < 0)
-            {
-                break;
-            }
-            else if ((c = b.end().compareTo(a.end())) <= 0)
-            {
-                bi++;
-                if (c == 0) ai++;
-            }
-            else
-            {
-                // use a temporary counter, so that if we don't find a run of ranges that enforce the superset
-                // condition we exit at the start of the mismatch run (and permit it to be merged)
-                // TODO: use exponentialSearch
-                int tmpai = ai;
-                do
-                {
-                    if (++tmpai == as.length || !a.end().equals(as[tmpai].start()))
-                        break out;
-                    a = as[tmpai];
-                }
-                while (a.end().compareTo(b.end()) < 0);
-                bi++;
-                ai = tmpai;
-            }
-        }
-
-        return ((long)ai << 32) | bi;
-    }
-
-    /**
-     * @return true iff {@code that} is a subset of {@code this}
-     */
-    public boolean contains(KeyRanges that)
-    {
-        if (this.isEmpty()) return that.isEmpty();
-        if (that.isEmpty()) return true;
-
-        return ((int) supersetLinearMerge(this.ranges, that.ranges)) == that.size();
-    }
-
-    /**
-     * @return the union of {@code this} and {@code that}, returning one of the two inputs if possible
-     */
-    public KeyRanges union(KeyRanges that)
-    {
-        if (this == that) return this;
-        if (this.isEmpty()) return that;
-        if (that.isEmpty()) return this;
-
-        KeyRange[] as = this.ranges, bs = that.ranges;
-        {
-            // make sure as/ai represent the ranges that might fully contain the other
-            int c = as[0].start().compareTo(bs[0].start());
-            if (c > 0 || c == 0 && as[as.length - 1].end().compareTo(bs[bs.length - 1].end()) < 0)
-            {
-                KeyRange[] tmp = as; as = bs; bs = tmp;
-            }
-        }
-
-        int ai, bi; {
-            long tmp = supersetLinearMerge(as, bs);
-            ai = (int)(tmp >>> 32);
-            bi = (int)tmp;
-        }
-
-        if (bi == bs.length)
-            return as == this.ranges ? this : that;
-
-        KeyRange[] result = new KeyRange[as.length + (bs.length - bi)];
-        int resultCount = copyAndMergeTouching(as, 0, result, 0, ai);
-
-        while (ai < as.length && bi < bs.length)
-        {
-            KeyRange a = as[ai];
-            KeyRange b = bs[bi];
-
-            int c = a.compareIntersecting(b);
-            if (c < 0)
-            {
-                result[resultCount++] = a;
-                ai++;
-            }
-            else if (c > 0)
-            {
-                result[resultCount++] = b;
-                bi++;
-            }
-            else
-            {
-                RoutingKey start = a.start().compareTo(b.start()) <= 0 ? a.start() : b.start();
-                RoutingKey end = a.end().compareTo(b.end()) >= 0 ? a.end() : b.end();
-                ai++;
-                bi++;
-                while (ai < as.length || bi < bs.length)
-                {
-                    KeyRange min;
-                    if (ai == as.length) min = bs[bi];
-                    else if (bi == bs.length) min = a = as[ai];
-                    else min = as[ai].start().compareTo(bs[bi].start()) < 0 ? a = as[ai] : bs[bi];
-                    if (min.start().compareTo(end) > 0)
-                        break;
-                    if (min.end().compareTo(end) > 0)
-                        end = min.end();
-                    if (a == min) ai++;
-                    else bi++;
-                }
-                result[resultCount++] = a.subRange(start, end);
-            }
-        }
-
-        while (ai < as.length)
-            result[resultCount++] = as[ai++];
-
-        while (bi < bs.length)
-            result[resultCount++] = bs[bi++];
-
-        if (resultCount < result.length)
-            result = Arrays.copyOf(result, resultCount);
-
-        return new KeyRanges(result);
-    }
-
-    public KeyRanges mergeTouching()
-    {
-        if (ranges.length == 0)
-            return this;
-
-        KeyRange[] result = new KeyRange[ranges.length];
-        int count = copyAndMergeTouching(ranges, 0, result, 0, ranges.length);
-        if (count == result.length)
-            return this;
-        result = Arrays.copyOf(result, count);
-        return new KeyRanges(result);
-    }
-
-    private static int copyAndMergeTouching(KeyRange[] src, int srcPosition, KeyRange[] trg, int trgPosition, int srcCount)
-    {
-        if (srcCount == 0)
-            return 0;
-
-        int count = 0;
-        KeyRange prev = src[srcPosition];
-        RoutingKey end = prev.end();
-        for (int i = 1 ; i < srcCount ; ++i)
-        {
-            KeyRange next = src[srcPosition + i];
-            if (!end.equals(next.start()))
-            {
-                trg[trgPosition + count++] = maybeUpdateEnd(prev, end);
-                prev = next;
-            }
-            end = next.end();
-        }
-        trg[trgPosition + count++] = maybeUpdateEnd(prev, end);
-        return count;
-    }
-
-    private static KeyRange maybeUpdateEnd(KeyRange range, RoutingKey withEnd)
-    {
-        return withEnd == range.end() ? range : range.subRange(range.start(), withEnd);
-    }
-
-    public static KeyRanges of(KeyRange ... ranges)
-    {
-        if (ranges.length == 0)
-            return EMPTY;
-
-        return sortAndDeoverlap(ranges, ranges.length);
-    }
-
-    private static KeyRanges sortAndDeoverlap(KeyRange[] ranges, int count)
-    {
-        if (count == 0)
-            return EMPTY;
-
-        if (count == 1)
-        {
-            if (ranges.length == 1)
-                return new KeyRanges(ranges);
-
-            return new KeyRanges(Arrays.copyOf(ranges, count));
-        }
-
-        Arrays.sort(ranges, 0, count, Comparator.comparing(KeyRange::start));
-        KeyRange prev = ranges[0];
-        int removed = 0;
-        for (int i = 1 ; i < count ; ++i)
-        {
-            KeyRange next = ranges[i];
-            if (prev.end().compareTo(next.start()) > 0)
-            {
-                prev = prev.subRange(prev.start(), next.start());
-                if (prev.end().compareTo(next.end()) >= 0)
-                {
-                    removed++;
-                }
-                else if (removed > 0)
-                {
-                    ranges[i - removed] = prev = next.subRange(prev.end(), next.end());
-                }
-            }
-            else if (removed > 0)
-            {
-                ranges[i - removed] = prev = next;
-            }
-        }
-
-        count -= removed;
-        if (count != ranges.length)
-            ranges = Arrays.copyOf(ranges, count);
-
-        return new KeyRanges(ranges);
-    }
-
-    public static KeyRanges ofSortedAndDeoverlapped(KeyRange ... ranges)
-    {
-        for (int i = 1 ; i < ranges.length ; ++i)
-        {
-            if (ranges[i - 1].end().compareTo(ranges[i].start()) > 0)
-                throw new IllegalArgumentException(Arrays.toString(ranges) + " is not correctly sorted or deoverlapped");
-        }
-
-        return new KeyRanges(ranges);
-    }
-
-    static KeyRanges ofSortedAndDeoverlappedUnchecked(KeyRange ... ranges)
-    {
-        return new KeyRanges(ranges);
-    }
-
-    public static KeyRanges single(KeyRange range)
-    {
-        return new KeyRanges(new KeyRange[]{range});
-    }
-}
diff --git a/accord-core/src/main/java/accord/primitives/KeyRoute.java b/accord-core/src/main/java/accord/primitives/KeyRoute.java
new file mode 100644
index 0000000..ccb2a9f
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/KeyRoute.java
@@ -0,0 +1,34 @@
+package accord.primitives;
+
+import accord.utils.Invariants;
+
+import accord.api.RoutingKey;
+import accord.utils.SortedArrays;
+
+import javax.annotation.Nonnull;
+
+public abstract class KeyRoute extends AbstractRoutableKeys<Route<RoutingKey>> implements Route<RoutingKey>
+{
+    public final RoutingKey homeKey;
+
+    KeyRoute(@Nonnull RoutingKey homeKey, RoutingKey[] keys)
+    {
+        super(keys);
+        this.homeKey = Invariants.nonNull(homeKey);
+    }
+
+    @Override
+    public Unseekables<RoutingKey, ?> toMaximalUnseekables()
+    {
+        return new RoutingKeys(SortedArrays.insert(keys, homeKey, RoutingKey[]::new));
+    }
+
+    @Override
+    public RoutingKey homeKey()
+    {
+        return homeKey;
+    }
+
+    @Override
+    public abstract PartialKeyRoute slice(Ranges ranges);
+}
diff --git a/accord-core/src/main/java/accord/primitives/Keys.java b/accord-core/src/main/java/accord/primitives/Keys.java
index 9202444..0e075ba 100644
--- a/accord-core/src/main/java/accord/primitives/Keys.java
+++ b/accord-core/src/main/java/accord/primitives/Keys.java
@@ -27,10 +27,9 @@ import accord.utils.ArrayBuffers.ObjectBuffers;
 
 import static accord.utils.ArrayBuffers.cachedKeys;
 
-@SuppressWarnings("rawtypes")
 // TODO: this should probably be a BTree
 // TODO: check that foldl call-sites are inlined and optimised by HotSpot
-public class Keys extends AbstractKeys<Key, Keys>
+public class Keys extends AbstractKeys<Key, Keys> implements Seekables<Key, Keys>
 {
     public static class SerializationSupport
     {
@@ -67,60 +66,15 @@ public class Keys extends AbstractKeys<Key, Keys>
     }
 
     @Override
-    public int hashCode()
-    {
-        return Arrays.hashCode(keys);
-    }
-
-    public int indexOf(Key key)
-    {
-        return Arrays.binarySearch(keys, key);
-    }
-
-    public boolean contains(Key key)
-    {
-        return indexOf(key) >= 0;
-    }
-
-    public Key get(int indexOf)
-    {
-        return keys[indexOf];
-    }
-
-    public boolean isEmpty()
-    {
-        return keys.length == 0;
-    }
-
-    public int size()
-    {
-        return keys.length;
-    }
-
-    /**
-     * return true if this keys collection contains all keys found in the given keys
-     */
-    public boolean containsAll(Keys that)
-    {
-        if (that.isEmpty())
-            return true;
-
-        return foldlIntersect(that, (li, ri, k, p, v) -> v + 1, 0, 0, 0) == that.size();
-    }
-
     public Keys union(Keys that)
     {
         return wrap(SortedArrays.linearUnion(keys, that.keys, cachedKeys()), that);
     }
 
-    public Keys slice(KeyRanges ranges)
-    {
-        return wrap(SortedArrays.sliceWithMultipleMatches(keys, ranges.ranges, Key[]::new, (k, r) -> -r.compareTo(k), KeyRange::compareTo));
-    }
-
-    public int findNext(Key key, int startIndex)
+    @Override
+    public Keys slice(Ranges ranges)
     {
-        return SortedArrays.exponentialSearch(keys, startIndex, keys.length, key);
+        return wrap(SortedArrays.sliceWithMultipleMatches(keys, ranges.ranges, Key[]::new, (k, r) -> -r.compareTo(k), Range::compareTo));
     }
 
     public Keys with(Key key)
@@ -276,19 +230,13 @@ public class Keys extends AbstractKeys<Key, Keys>
         return new Keys(keys);
     }
 
-    private Keys wrap(Key[] wrap, Keys that)
+    private Keys wrap(Key[] wrap, AbstractKeys<Key, ?> that)
     {
-        return wrap == keys ? this : wrap == that.keys ? that : new Keys(wrap);
+        return wrap == keys ? this : wrap == that.keys && that instanceof Keys ? (Keys)that : new Keys(wrap);
     }
 
     private Keys wrap(Key[] wrap)
     {
         return wrap == keys ? this : new Keys(wrap);
     }
-
-    private static Key[] sort(Key[] array)
-    {
-        Arrays.sort(array);
-        return array;
-    }
 }
diff --git a/accord-core/src/main/java/accord/primitives/PartialDeps.java b/accord-core/src/main/java/accord/primitives/PartialDeps.java
index 14db2ab..addf11e 100644
--- a/accord-core/src/main/java/accord/primitives/PartialDeps.java
+++ b/accord-core/src/main/java/accord/primitives/PartialDeps.java
@@ -4,13 +4,13 @@ import com.google.common.base.Preconditions;
 
 public class PartialDeps extends Deps
 {
-    public static final PartialDeps NONE = new PartialDeps(KeyRanges.EMPTY, Deps.NONE.keys, Deps.NONE.txnIds, Deps.NONE.keyToTxnId);
+    public static final PartialDeps NONE = new PartialDeps(Ranges.EMPTY, Deps.NONE.keys, Deps.NONE.txnIds, Deps.NONE.keyToTxnId);
 
     public static class SerializerSupport
     {
         private SerializerSupport() {}
 
-        public static PartialDeps create(KeyRanges covering, Keys keys, TxnId[] txnIds, int[] keyToTxnId)
+        public static PartialDeps create(Ranges covering, Keys keys, TxnId[] txnIds, int[] keyToTxnId)
         {
             return new PartialDeps(covering, keys, txnIds, keyToTxnId);
         }
@@ -18,8 +18,8 @@ public class PartialDeps extends Deps
 
     public static class OrderedBuilder extends AbstractOrderedBuilder<PartialDeps>
     {
-        final KeyRanges covering;
-        public OrderedBuilder(KeyRanges covering, boolean hasOrderedTxnId)
+        final Ranges covering;
+        public OrderedBuilder(Ranges covering, boolean hasOrderedTxnId)
         {
             super(hasOrderedTxnId);
             this.covering = covering;
@@ -32,28 +32,23 @@ public class PartialDeps extends Deps
         }
     }
 
-    public static OrderedBuilder orderedBuilder(KeyRanges ranges, boolean hasOrderedTxnId)
+    public static OrderedBuilder orderedBuilder(Ranges ranges, boolean hasOrderedTxnId)
     {
         return new OrderedBuilder(ranges, hasOrderedTxnId);
     }
 
-    public final KeyRanges covering;
+    public final Ranges covering;
 
-    PartialDeps(KeyRanges covering, Keys keys, TxnId[] txnIds, int[] keyToTxnId)
+    PartialDeps(Ranges covering, Keys keys, TxnId[] txnIds, int[] keyToTxnId)
     {
         super(keys, txnIds, keyToTxnId);
         this.covering = covering;
         Preconditions.checkState(covering.containsAll(keys));
     }
 
-    public boolean covers(KeyRanges ranges)
+    public boolean covers(Unseekables<?, ?> keysOrRanges)
     {
-        return covering.contains(ranges);
-    }
-
-    public boolean covers(AbstractKeys<?, ?> keys)
-    {
-        return covering.containsAll(keys);
+        return covering.containsAll(keysOrRanges);
     }
 
     public PartialDeps with(PartialDeps that)
@@ -62,23 +57,23 @@ public class PartialDeps extends Deps
         return new PartialDeps(covering.union(that.covering), merged.keys, merged.txnIds, merged.keyToTxnId);
     }
 
-    public Deps reconstitute(Route route)
+    public Deps reconstitute(FullRoute<?> route)
     {
         if (!covers(route))
             throw new IllegalArgumentException();
         return new Deps(keys, txnIds, keyToTxnId);
     }
 
-    // PartialRoute might cover a wider set of ranges, some of which may have no involved keys
-    public PartialDeps reconstitutePartial(PartialRoute route)
+    // PartialRoute<?>might cover a wider set of ranges, some of which may have no involved keys
+    public PartialDeps reconstitutePartial(PartialRoute<?> route)
     {
         if (!covers(route))
             throw new IllegalArgumentException();
 
-        if (covers(route.covering))
+        if (covers(route.covering()))
             return this;
 
-        return new PartialDeps(route.covering, keys, txnIds, keyToTxnId);
+        return new PartialDeps(route.covering(), keys, txnIds, keyToTxnId);
     }
 
     @Override
diff --git a/accord-core/src/main/java/accord/primitives/PartialKeyRoute.java b/accord-core/src/main/java/accord/primitives/PartialKeyRoute.java
new file mode 100644
index 0000000..21c1ec0
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/PartialKeyRoute.java
@@ -0,0 +1,103 @@
+package accord.primitives;
+
+import accord.utils.Invariants;
+
+import accord.api.RoutingKey;
+import accord.utils.SortedArrays;
+
+/**
+ * A slice of a Route that covers
+ */
+public class PartialKeyRoute extends KeyRoute implements PartialRoute<RoutingKey>
+{
+    public static class SerializationSupport
+    {
+        public static PartialKeyRoute create(Ranges covering, RoutingKey homeKey, RoutingKey[] keys)
+        {
+            return new PartialKeyRoute(covering, homeKey, keys);
+        }
+    }
+
+    public final Ranges covering;
+
+    public PartialKeyRoute(Ranges covering, RoutingKey homeKey, RoutingKey[] keys)
+    {
+        super(homeKey, keys);
+        this.covering = covering;
+    }
+
+    public PartialKeyRoute sliceStrict(Ranges newRange)
+    {
+        if (!covering.containsAll(newRange))
+            throw new IllegalArgumentException("Not covered");
+
+        RoutingKey[] keys = slice(newRange, RoutingKey[]::new);
+        return new PartialKeyRoute(newRange, homeKey, keys);
+    }
+
+    @Override
+    public Route<RoutingKey> union(Route<RoutingKey> that)
+    {
+        if (that.kind().isFullRoute()) return that;
+        return union((PartialKeyRoute) that);
+    }
+
+    @Override
+    public UnseekablesKind kind()
+    {
+        return UnseekablesKind.PartialKeyRoute;
+    }
+
+    @Override
+    public boolean covers(Ranges ranges)
+    {
+        return covering.containsAll(ranges);
+    }
+
+    @Override
+    public boolean intersects(AbstractRanges<?> ranges)
+    {
+        // TODO (now): remove this in favour of parent implementation - ambiguous at present
+        return ranges.intersects(covering);
+    }
+
+    @Override
+    public AbstractRoutableKeys<?> with(RoutingKey withKey)
+    {
+        if (contains(withKey))
+            return this;
+
+        return new RoutingKeys(toRoutingKeysArray(withKey));
+    }
+
+    public PartialKeyRoute slice(Ranges newRange)
+    {
+        if (newRange.containsAll(covering))
+            return this;
+
+        RoutingKey[] keys = slice(covering, RoutingKey[]::new);
+        return new PartialKeyRoute(covering, homeKey, keys);
+    }
+
+    @Override
+    public Ranges covering()
+    {
+        return covering;
+    }
+
+    public PartialKeyRoute union(PartialRoute<RoutingKey> with)
+    {
+        if (!(with instanceof PartialKeyRoute))
+            throw new IllegalArgumentException();
+
+        PartialKeyRoute that = (PartialKeyRoute) with;
+        Invariants.checkState(homeKey.equals(that.homeKey));
+        RoutingKey[] keys = SortedArrays.linearUnion(this.keys, that.keys, RoutingKey[]::new);
+        Ranges covering = this.covering.union(that.covering);
+        if (covering == this.covering && keys == this.keys)
+            return this;
+        if (covering == that.covering && keys == that.keys)
+            return that;
+        return new PartialKeyRoute(covering, homeKey, keys);
+    }
+}
diff --git a/accord-core/src/main/java/accord/primitives/PartialRangeRoute.java b/accord-core/src/main/java/accord/primitives/PartialRangeRoute.java
new file mode 100644
index 0000000..c28239a
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/PartialRangeRoute.java
@@ -0,0 +1,108 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+import accord.utils.Invariants;
+
+import static accord.primitives.AbstractRanges.UnionMode.MERGE_OVERLAPPING;
+
+/**
+ * A slice of a Route that covers
+ */
+public class PartialRangeRoute extends RangeRoute implements PartialRoute<Range>
+{
+    public static class SerializationSupport
+    {
+        public static PartialRangeRoute create(Ranges covering, RoutingKey homeKey, Range[] ranges)
+        {
+            return new PartialRangeRoute(covering, homeKey, ranges);
+        }
+    }
+
+    public final Ranges covering;
+
+    public PartialRangeRoute(Ranges covering, RoutingKey homeKey, Range[] ranges)
+    {
+        super(homeKey, ranges);
+        this.covering = covering;
+    }
+
+    @Override
+    public UnseekablesKind kind()
+    {
+        return UnseekablesKind.PartialRangeRoute;
+    }
+
+    @Override
+    public Ranges covering()
+    {
+        return covering;
+    }
+
+    @Override
+    public boolean covers(Ranges ranges)
+    {
+        return covering.containsAll(ranges);
+    }
+
+    @Override
+    public boolean intersects(AbstractRanges<?> ranges)
+    {
+        return ranges.intersects(covering);
+    }
+
+    public PartialRangeRoute sliceStrict(Ranges newRange)
+    {
+        if (!covering.containsAll(newRange))
+            throw new IllegalArgumentException("Not covered");
+
+        return slice(newRange, this, homeKey, PartialRangeRoute::new);
+    }
+
+    @Override
+    public Unseekables<Range, ?> toMaximalUnseekables()
+    {
+        // TODO (now)
+        throw new UnsupportedOperationException();
+    }
+
+    public PartialRangeRoute slice(Ranges newRange)
+    {
+        if (newRange.containsAll(covering))
+            return this;
+
+        return slice(newRange, this, homeKey, PartialRangeRoute::new);
+    }
+
+    public Unseekables<Range, ?> with(RoutingKey withKey)
+    {
+        // TODO: this is left unimplemented until we actually have a range transaction to decide how best to address it
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Route<Range> union(Route<Range> that)
+    {
+        if (Route.isFullRoute(that)) return that;
+        return union((PartialRangeRoute) that);
+    }
+
+    public PartialRangeRoute union(PartialRoute<Range> with)
+    {
+        if (!(with instanceof PartialRangeRoute))
+            throw new IllegalArgumentException();
+
+        PartialRangeRoute that = (PartialRangeRoute) with;
+        Invariants.checkState(homeKey.equals(that.homeKey));
+        Ranges covering = this.covering.union(that.covering);
+        if (covering == this.covering) return this;
+        else if (covering == that.covering) return that;
+
+        return union(MERGE_OVERLAPPING, this, that, covering, homeKey, PartialRangeRoute::new);
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return super.equals(that) && covering.equals(((PartialRangeRoute)that).covering);
+    }
+}
diff --git a/accord-core/src/main/java/accord/primitives/PartialRoute.java b/accord-core/src/main/java/accord/primitives/PartialRoute.java
index 5f4ed67..cb93004 100644
--- a/accord-core/src/main/java/accord/primitives/PartialRoute.java
+++ b/accord-core/src/main/java/accord/primitives/PartialRoute.java
@@ -1,78 +1,12 @@
 package accord.primitives;
 
-import java.util.function.IntFunction;
-
-import com.google.common.base.Preconditions;
-
-import accord.api.RoutingKey;
-import accord.utils.SortedArrays;
-
-/**
- * A slice of a Route that covers
- */
-public class PartialRoute extends AbstractRoute
+public interface PartialRoute<T extends Unseekable> extends Route<T>
 {
-    public static class SerializationSupport
-    {
-        public static PartialRoute create(KeyRanges covering, RoutingKey homeKey, RoutingKey[] keys)
-        {
-            return new PartialRoute(covering, homeKey, keys);
-        }
-    }
-
-    public final KeyRanges covering;
-
-    public PartialRoute(KeyRanges covering, RoutingKey homeKey, RoutingKey[] keys)
-    {
-        super(keys, homeKey);
-        this.covering = covering;
-    }
-
-    public PartialRoute sliceStrict(KeyRanges newRange)
-    {
-        if (!covering.contains(newRange))
-            throw new IllegalArgumentException("Not covered");
-
-        RoutingKey[] keys = slice(newRange, RoutingKey[]::new);
-        return new PartialRoute(newRange, homeKey, keys);
-    }
-
-    @Override
-    public AbstractRoute union(AbstractRoute that)
-    {
-        if (that instanceof Route) return that;
-        return union((PartialRoute) that);
-    }
-
-    @Override
-    public boolean covers(KeyRanges ranges)
-    {
-        return covering.contains(ranges);
-    }
-
-    public PartialRoute slice(KeyRanges newRange)
-    {
-        if (newRange.contains(covering))
-            return this;
-
-        RoutingKey[] keys = slice(covering, RoutingKey[]::new);
-        return new PartialRoute(covering, homeKey, keys);
-    }
-
-    public PartialRoute union(PartialRoute that)
-    {
-        Preconditions.checkState(homeKey.equals(that.homeKey));
-        RoutingKey[] keys = SortedArrays.linearUnion(this.keys, that.keys, factory());
-        KeyRanges covering = this.covering.union(that.covering);
-        if (covering == this.covering && keys == this.keys)
-            return this;
-        if (covering == that.covering && keys == that.keys)
-            return that;
-        return new PartialRoute(covering, homeKey, keys);
-    }
+    boolean isEmpty();
+    Ranges covering();
 
-    private static IntFunction<RoutingKey[]> factory()
-    {
-        return RoutingKey[]::new;
-    }
+    /**
+     * Expected to be compatible PartialRoute type, i.e. both split from the same FullRoute
+     */
+    PartialRoute<T> union(PartialRoute<T> route);
 }
diff --git a/accord-core/src/main/java/accord/primitives/PartialTxn.java b/accord-core/src/main/java/accord/primitives/PartialTxn.java
index e7a7d47..3da4715 100644
--- a/accord-core/src/main/java/accord/primitives/PartialTxn.java
+++ b/accord-core/src/main/java/accord/primitives/PartialTxn.java
@@ -8,31 +8,24 @@ import accord.api.Update;
 
 public interface PartialTxn extends Txn
 {
-    KeyRanges covering();
+    Ranges covering();
     // TODO: merge efficient merge when more than one input
     PartialTxn with(PartialTxn add);
-    Txn reconstitute(Route route);
-    PartialTxn reconstitutePartial(PartialRoute route);
+    Txn reconstitute(FullRoute<?> route);
+    PartialTxn reconstitutePartial(PartialRoute<?> route);
 
-    default boolean covers(KeyRanges ranges)
+    default boolean covers(Unseekables<?, ?> unseekables)
     {
-        return covering().contains(ranges);
-    }
-
-    default boolean covers(AbstractKeys<?, ?> keys)
-    {
-        // TODO: this distinction seems brittle
-        if (keys instanceof AbstractRoute)
-            return covers((AbstractRoute)keys);
-
-        return covering().containsAll(keys);
-    }
-
-    default boolean covers(AbstractRoute route)
-    {
-        if (query() == null && route.contains(route.homeKey))
-            return false;
-        return covering().containsAll(route);
+        if (query() == null)
+        {
+            // The home shard is expected to store the query contents
+            // So if the query is null, and we are being asked if we
+            // cover a range that includes a home shard, we should say no
+            Route<?> asRoute = Route.tryCastToRoute(unseekables);
+            if (asRoute != null && asRoute.contains(asRoute.homeKey()))
+                return false;
+        }
+        return covering().containsAll(unseekables);
     }
 
     // TODO: override toString
@@ -43,28 +36,28 @@ public interface PartialTxn extends Txn
 
     class InMemory extends Txn.InMemory implements PartialTxn
     {
-        public final KeyRanges covering;
+        public final Ranges covering;
 
-        public InMemory(KeyRanges covering, Kind kind, Keys keys, Read read, Query query, Update update)
+        public InMemory(Ranges covering, Kind kind, Seekables<?, ?> keys, Read read, Query query, Update update)
         {
             super(kind, keys, read, query, update);
             this.covering = covering;
         }
 
         @Override
-        public KeyRanges covering()
+        public Ranges covering()
         {
             return covering;
         }
 
         // TODO: merge efficient merge when more than one input
-        public accord.primitives.PartialTxn with(accord.primitives.PartialTxn add)
+        public PartialTxn with(PartialTxn add)
         {
             if (!add.kind().equals(kind()))
                 throw new IllegalArgumentException();
 
-            KeyRanges covering = this.covering.union(add.covering());
-            Keys keys = this.keys().union(add.keys());
+            Ranges covering = this.covering.union(add.covering());
+            Seekables<?, ?> keys = ((Seekables)this.keys()).union(add.keys());
             Read read = this.read().merge(add.read());
             Query query = this.query() == null ? add.query() : this.query();
             Update update = this.update() == null ? null : this.update().merge(add.update());
@@ -83,7 +76,12 @@ public interface PartialTxn extends Txn
 
         // TODO: override toString
 
-        public Txn reconstitute(Route route)
+        public boolean covers(Ranges ranges)
+        {
+            return covering.containsAll(ranges);
+        }
+
+        public Txn reconstitute(FullRoute<?> route)
         {
             if (!covers(route) || query() == null)
                 throw new IllegalStateException("Incomplete PartialTxn: " + this + ", route: " + route);
@@ -91,15 +89,15 @@ public interface PartialTxn extends Txn
             return new Txn.InMemory(kind(), keys(), read(), query(), update());
         }
 
-        public accord.primitives.PartialTxn reconstitutePartial(PartialRoute route)
+        public PartialTxn reconstitutePartial(PartialRoute<?> route)
         {
             if (!covers(route))
                 throw new IllegalStateException("Incomplete PartialTxn: " + this + ", route: " + route);
 
-            if (covering.contains(route.covering))
+            if (covering.containsAll(route.covering()))
                 return this;
 
-            return new PartialTxn.InMemory(route.covering, kind(), keys(), read(), query(), update());
+            return new PartialTxn.InMemory(route.covering(), kind(), keys(), read(), query(), update());
         }
     }
 
diff --git a/accord-core/src/main/java/accord/primitives/KeyRange.java b/accord-core/src/main/java/accord/primitives/Range.java
similarity index 73%
rename from accord-core/src/main/java/accord/primitives/KeyRange.java
rename to accord-core/src/main/java/accord/primitives/Range.java
index e44e12a..c69894e 100644
--- a/accord-core/src/main/java/accord/primitives/KeyRange.java
+++ b/accord-core/src/main/java/accord/primitives/Range.java
@@ -19,11 +19,10 @@
 package accord.primitives;
 
 import accord.api.RoutingKey;
+import accord.utils.Invariants;
 import accord.utils.SortedArrays;
 import accord.utils.SortedArrays.Search;
 
-import com.google.common.base.Preconditions;
-
 import java.util.Objects;
 
 import static accord.utils.SortedArrays.Search.CEIL;
@@ -32,9 +31,9 @@ import static accord.utils.SortedArrays.Search.FAST;
 /**
  * A range of keys
  */
-public abstract class KeyRange implements Comparable<RoutingKey>
+public abstract class Range implements Comparable<RoutableKey>, Unseekable, Seekable
 {
-    public static class EndInclusive extends KeyRange
+    public static class EndInclusive extends Range
     {
         public EndInclusive(RoutingKey start, RoutingKey end)
         {
@@ -42,7 +41,7 @@ public abstract class KeyRange implements Comparable<RoutingKey>
         }
 
         @Override
-        public int compareTo(RoutingKey key)
+        public int compareTo(RoutableKey key)
         {
             if (key.compareTo(start()) <= 0)
                 return 1;
@@ -64,13 +63,13 @@ public abstract class KeyRange implements Comparable<RoutingKey>
         }
 
         @Override
-        public KeyRange subRange(RoutingKey start, RoutingKey end)
+        public Range subRange(RoutingKey start, RoutingKey end)
         {
             return new EndInclusive(start, end);
         }
     }
 
-    public static class StartInclusive extends KeyRange
+    public static class StartInclusive extends Range
     {
         public StartInclusive(RoutingKey start, RoutingKey end)
         {
@@ -78,7 +77,7 @@ public abstract class KeyRange implements Comparable<RoutingKey>
         }
 
         @Override
-        public int compareTo(RoutingKey key)
+        public int compareTo(RoutableKey key)
         {
             if (key.compareTo(start()) < 0)
                 return 1;
@@ -100,15 +99,15 @@ public abstract class KeyRange implements Comparable<RoutingKey>
         }
 
         @Override
-        public KeyRange subRange(RoutingKey start, RoutingKey end)
+        public Range subRange(RoutingKey start, RoutingKey end)
         {
             return new StartInclusive(start, end);
         }
     }
 
-    public static KeyRange range(RoutingKey start, RoutingKey end, boolean startInclusive, boolean endInclusive)
+    public static Range range(RoutingKey start, RoutingKey end, boolean startInclusive, boolean endInclusive)
     {
-        return new KeyRange(start, end) {
+        return new Range(start, end) {
 
             @Override
             public boolean startInclusive()
@@ -123,13 +122,13 @@ public abstract class KeyRange implements Comparable<RoutingKey>
             }
 
             @Override
-            public KeyRange subRange(RoutingKey start, RoutingKey end)
+            public Range subRange(RoutingKey start, RoutingKey end)
             {
                 throw new UnsupportedOperationException("subRange");
             }
 
             @Override
-            public int compareTo(RoutingKey key)
+            public int compareTo(RoutableKey key)
             {
                 if (startInclusive)
                 {
@@ -156,32 +155,10 @@ public abstract class KeyRange implements Comparable<RoutingKey>
         };
     }
 
-    private static KeyRange tryMergeExclusiveInclusive(KeyRange left, KeyRange right)
-    {
-        if (left.getClass() != right.getClass())
-            return null;
-
-        Preconditions.checkArgument(left instanceof EndInclusive || left instanceof StartInclusive);
-
-        int cmp = left.compareIntersecting(right);
-
-        if (cmp == 0)
-            return left.subRange(left.start.compareTo(right.start) < 0 ? left.start : right.start,
-                                 left.end.compareTo(right.end) > 0 ? left.end : right.end);
-
-        if (cmp > 0 && right.end.equals(left.start))
-            return left.subRange(right.start, left.end);
-
-        if (cmp < 0 && left.end.equals(right.start))
-            return left.subRange(left.start, right.end);
-
-        return null;
-    }
-
     private final RoutingKey start;
     private final RoutingKey end;
 
-    private KeyRange(RoutingKey start, RoutingKey end)
+    private Range(RoutingKey start, RoutingKey end)
     {
         if (start.compareTo(end) >= 0)
             throw new IllegalArgumentException(start + " >= " + end);
@@ -195,22 +172,25 @@ public abstract class KeyRange implements Comparable<RoutingKey>
     {
         return start;
     }
+
     public final RoutingKey end()
     {
         return end;
     }
 
+    public final Kind kind() { return Kind.Range; }
+
     public abstract boolean startInclusive();
     public abstract boolean endInclusive();
 
-    public abstract KeyRange subRange(RoutingKey start, RoutingKey end);
+    public abstract Range subRange(RoutingKey start, RoutingKey end);
 
     @Override
     public boolean equals(Object o)
     {
         if (this == o) return true;
         if (o == null || getClass() != o.getClass()) return false;
-        KeyRange that = (KeyRange) o;
+        Range that = (Range) o;
         return Objects.equals(start, that.start) && Objects.equals(end, that.end);
     }
 
@@ -226,31 +206,22 @@ public abstract class KeyRange implements Comparable<RoutingKey>
         return "Range[" + start + ", " + end + ']';
     }
 
-    /**
-     * Returns a negative integer, zero, or a positive integer as the provided key is less than, contained by,
-     * or greater than this range.
-     */
-    public int compareKey(RoutingKey key)
-    {
-        return -compareTo(key);
-    }
-
     /**
      * Returns a negative integer, zero, or a positive integer as the provided key is greater than, contained by,
      * or less than this range.
      */
-    public abstract int compareTo(RoutingKey key);
+    public abstract int compareTo(RoutableKey key);
 
-    public boolean containsKey(RoutingKey key)
+    public boolean containsKey(RoutableKey key)
     {
-        return compareKey(key) == 0;
+        return compareTo(key) == 0;
     }
 
     /**
      * Returns a negative integer, zero, or a positive integer if both points of the provided range are less than, the
      * range intersects this range, or both points are greater than this range
      */
-    public int compareIntersecting(KeyRange that)
+    public int compareIntersecting(Range that)
     {
         if (that.getClass() != this.getClass())
             throw new IllegalArgumentException("Cannot mix KeyRange of different types");
@@ -261,26 +232,21 @@ public abstract class KeyRange implements Comparable<RoutingKey>
         return 0;
     }
 
-    public boolean intersects(KeyRange that)
-    {
-        return compareIntersecting(that) == 0;
-    }
-
-    public boolean fullyContains(KeyRange that)
+    public boolean fullyContains(Range that)
     {
         return that.start.compareTo(this.start) >= 0 && that.end.compareTo(this.end) <= 0;
     }
 
     public boolean intersects(Keys keys)
     {
-        return SortedArrays.binarySearch(keys.keys, 0, keys.size(), this, KeyRange::compareTo, FAST) >= 0;
+        return SortedArrays.binarySearch(keys.keys, 0, keys.size(), this, Range::compareTo, FAST) >= 0;
     }
 
     /**
      * Returns a range covering the overlapping parts of this and the provided range, returns
      * null if the ranges do not overlap
      */
-    public KeyRange intersection(KeyRange that)
+    public Range intersection(Range that)
     {
         if (this.compareIntersecting(that) != 0)
             return null;
@@ -295,7 +261,7 @@ public abstract class KeyRange implements Comparable<RoutingKey>
      */
     public int nextHigherKeyIndex(AbstractKeys<?, ?> keys, int from)
     {
-        int i = SortedArrays.exponentialSearch(keys.keys, from, keys.size(), this, KeyRange::compareTo, Search.FLOOR);
+        int i = SortedArrays.exponentialSearch(keys.keys, from, keys.size(), this, Range::compareTo, Search.FLOOR);
         if (i < 0) i = -1 - i;
         else i += 1;
         return i;
@@ -309,6 +275,30 @@ public abstract class KeyRange implements Comparable<RoutingKey>
      */
     public int nextCeilKeyIndex(Keys keys, int from)
     {
-        return SortedArrays.exponentialSearch(keys.keys, from, keys.size(), this, KeyRange::compareTo, CEIL);
+        return SortedArrays.exponentialSearch(keys.keys, from, keys.size(), this, Range::compareTo, CEIL);
+    }
+
+    @Override
+    public RoutingKey someIntersectingRoutingKey()
+    {
+        return startInclusive() ? start.toUnseekable() : end.toUnseekable();
+    }
+
+    public static Range slice(Range bound, Range toSlice)
+    {
+        Invariants.checkArgument(bound.compareIntersecting(toSlice) == 0);
+        if (bound.fullyContains(toSlice))
+            return toSlice;
+
+        return toSlice.subRange(
+                toSlice.start().compareTo(bound.start()) >= 0 ? toSlice.start() : bound.start(),
+                toSlice.end().compareTo(bound.end()) <= 0 ? toSlice.end() : bound.end()
+        );
+    }
+
+    @Override
+    public Unseekable toUnseekable()
+    {
+        return this;
     }
 }
diff --git a/accord-core/src/main/java/accord/primitives/RangeRoute.java b/accord-core/src/main/java/accord/primitives/RangeRoute.java
new file mode 100644
index 0000000..8e754fd
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/RangeRoute.java
@@ -0,0 +1,35 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+import accord.utils.Invariants;
+
+import javax.annotation.Nonnull;
+
+public abstract class RangeRoute extends AbstractRanges<Route<Range>> implements Route<Range>
+{
+    public final RoutingKey homeKey;
+
+    RangeRoute(@Nonnull RoutingKey homeKey, Range[] ranges)
+    {
+        super(ranges);
+        this.homeKey = Invariants.nonNull(homeKey);
+    }
+
+    @Override
+    public PartialRangeRoute slice(Ranges ranges)
+    {
+        return slice(ranges, this, homeKey, PartialRangeRoute::new);
+    }
+
+    @Override
+    public RoutingKey homeKey()
+    {
+        return homeKey;
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return super.equals(that) && homeKey.equals(((RangeRoute)that).homeKey);
+    }
+}
diff --git a/accord-core/src/main/java/accord/primitives/Ranges.java b/accord-core/src/main/java/accord/primitives/Ranges.java
new file mode 100644
index 0000000..6101195
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/Ranges.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.primitives;
+
+import accord.api.RoutingKey;
+
+import javax.annotation.Nonnull;
+import java.util.*;
+import java.util.stream.Stream;
+
+import static accord.primitives.AbstractRanges.UnionMode.MERGE_OVERLAPPING;
+import static accord.utils.Utils.toArray;
+
+public class Ranges extends AbstractRanges<Ranges> implements Iterable<Range>, Seekables<Range, Ranges>, Unseekables<Range, Ranges>
+{
+    public static final Ranges EMPTY = ofSortedAndDeoverlappedUnchecked();
+
+    Ranges(@Nonnull Range[] ranges)
+    {
+        super(ranges);
+    }
+
+    public static Ranges of(Range... ranges)
+    {
+        return AbstractRanges.of(Ranges::construct, ranges);
+    }
+
+    public static Ranges ofSortedAndDeoverlapped(Range... ranges)
+    {
+        return AbstractRanges.ofSortedAndDeoverlapped(Ranges::construct, ranges);
+    }
+
+    static Ranges ofSortedAndDeoverlappedUnchecked(Range... ranges)
+    {
+        return new Ranges(ranges);
+    }
+
+    public static Ranges single(Range range)
+    {
+        return new Ranges(new Range[]{range});
+    }
+
+    private static Ranges construct(Range[] ranges)
+    {
+        if (ranges.length == 0)
+            return EMPTY;
+
+        return new Ranges(ranges);
+    }
+
+    public Ranges select(int[] indexes)
+    {
+        Range[] selection = new Range[indexes.length];
+        for (int i=0; i<indexes.length; i++)
+            selection[i] = ranges[indexes[i]];
+        return ofSortedAndDeoverlapped(Ranges::construct, selection);
+    }
+
+    public Stream<Range> stream()
+    {
+        return Stream.of(ranges);
+    }
+
+    @Override
+    public Ranges slice(Ranges ranges)
+    {
+        return slice(ranges, this, null, (i1, i2, rs) -> new Ranges(rs));
+    }
+
+    @Override
+    public Ranges union(Ranges that)
+    {
+        return union(MERGE_OVERLAPPING, that);
+    }
+
+    @Override
+    public Unseekables<Range, ?> with(RoutingKey withKey)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public UnseekablesKind kind()
+    {
+        return UnseekablesKind.RoutingRanges;
+    }
+
+    @Override
+    public Ranges toUnseekables()
+    {
+        return this;
+    }
+
+    @Override
+    public FullRoute<Range> toRoute(RoutingKey homeKey)
+    {
+        return new FullRangeRoute(homeKey, ranges);
+    }
+
+    public Ranges union(UnionMode mode, Ranges that)
+    {
+        return union(mode, this, that, this, that, (left, right, ranges) -> {
+            if (ranges == left.ranges) return left;
+            if (ranges == right.ranges) return right;
+            return new Ranges(ranges);
+        });
+    }
+
+    public Ranges mergeTouching()
+    {
+        return mergeTouching(this, Ranges::new);
+    }
+
+    /**
+     * Subtracts the given set of key ranges from this
+     */
+    public Ranges difference(AbstractRanges<?> that)
+    {
+        if (that == this)
+            return construct(NO_RANGES);
+
+        List<Range> result = new ArrayList<>(this.size() + that.size());
+        int thatIdx = 0;
+
+        for (int thisIdx=0; thisIdx<this.size(); thisIdx++)
+        {
+            Range thisRange = this.ranges[thisIdx];
+            while (thatIdx < that.size())
+            {
+                Range thatRange = that.ranges[thatIdx];
+
+                int cmp = thisRange.compareIntersecting(thatRange);
+                if (cmp > 0)
+                {
+                    thatIdx++;
+                    continue;
+                }
+                if (cmp < 0) break;
+
+                int scmp = thisRange.start().compareTo(thatRange.start());
+                int ecmp = thisRange.end().compareTo(thatRange.end());
+
+                if (scmp < 0)
+                    result.add(thisRange.subRange(thisRange.start(), thatRange.start()));
+
+                if (ecmp <= 0)
+                {
+                    thisRange = null;
+                    break;
+                }
+                else
+                {
+                    thisRange = thisRange.subRange(thatRange.end(), thisRange.end());
+                    thatIdx++;
+                }
+            }
+            if (thisRange != null)
+                result.add(thisRange);
+        }
+        return construct(toArray(result, Range[]::new));
+    }
+
+}
diff --git a/accord-core/src/main/java/accord/primitives/Routable.java b/accord-core/src/main/java/accord/primitives/Routable.java
new file mode 100644
index 0000000..575f1e8
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/Routable.java
@@ -0,0 +1,14 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+
+/**
+ * Something that can be found in the cluster, and MAYBE found on disk (if Seekable)
+ */
+public interface Routable
+{
+    enum Kind { Key, Range }
+    Kind kind();
+    Unseekable toUnseekable();
+    RoutingKey someIntersectingRoutingKey();
+}
diff --git a/accord-core/src/main/java/accord/primitives/RoutableKey.java b/accord-core/src/main/java/accord/primitives/RoutableKey.java
new file mode 100644
index 0000000..0e005ff
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/RoutableKey.java
@@ -0,0 +1,64 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+
+import javax.annotation.Nonnull;
+
+public interface RoutableKey extends Routable, Comparable<RoutableKey>
+{
+    /**
+     * A special RoutingKey that sorts before or after everything, so that exclusive bounds may still cover
+     * the full range of possible RoutingKey.
+     *
+     * All RoutingKey implementations must sort correctly with this type.
+     *
+     * TODO: need to partition range from/to -/+ infinity as otherwise we exclude at least one key
+     */
+    class InfiniteRoutableKey implements RoutableKey
+    {
+        public static final InfiniteRoutableKey POSITIVE_INFINITY = new InfiniteRoutableKey(1);
+        public static final InfiniteRoutableKey NEGATIVE_INFINITY = new InfiniteRoutableKey(-1);
+
+        final int compareTo;
+
+        public InfiniteRoutableKey(int compareTo)
+        {
+            this.compareTo = compareTo;
+        }
+
+        @Override
+        public int compareTo(@Nonnull RoutableKey ignore)
+        {
+            return compareTo;
+        }
+
+        @Override
+        public int routingHash() { throw new UnsupportedOperationException(); }
+
+        @Override
+        public RoutingKey toUnseekable() { throw new UnsupportedOperationException(); }
+
+        @Override
+        public RoutingKey someIntersectingRoutingKey() { throw new UnsupportedOperationException(); }
+    }
+
+    /**
+     * Implementations must be comparable with {@link InfiniteRoutableKey}
+     * @param that the object to be compared.
+     * @return
+     */
+    int compareTo(@Nonnull RoutableKey that);
+
+    /**
+     * Returns a hash code of a key to support accord internal sharding. Hash values for equal keys must be equal.
+     *
+     * TODO (now): can we remove this if we remove hashIntersects et al?
+     */
+    int routingHash();
+
+    default Kind kind() { return Kind.Key; }
+
+    RoutingKey toUnseekable();
+
+    @Override default RoutingKey someIntersectingRoutingKey() { return toUnseekable(); }
+}
diff --git a/accord-core/src/main/java/accord/primitives/Routables.java b/accord-core/src/main/java/accord/primitives/Routables.java
new file mode 100644
index 0000000..05e85ba
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/Routables.java
@@ -0,0 +1,249 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+import accord.utils.IndexedFold;
+import accord.utils.IndexedFoldToLong;
+import accord.utils.IndexedRangeFoldToLong;
+import accord.utils.SortedArrays;
+import net.nicoulaj.compilecommand.annotations.Inline;
+
+import static accord.utils.SortedArrays.Search.FLOOR;
+
+/**
+ * A collection of either Seekable or Unseekable
+ */
+public interface Routables<K extends Routable, U extends Routables<K, ?>> extends Iterable<K>
+{
+    int indexOf(K item);
+    K get(int i);
+    int size();
+
+    boolean isEmpty();
+    boolean intersects(AbstractRanges<?> ranges);
+    boolean contains(RoutableKey key);
+    boolean containsAll(Routables<?, ?> keysOrRanges);
+
+    U slice(Ranges ranges);
+    Routables<K, U> union(U with);
+
+    /**
+     * Search forwards from {code thisIndex} and {@code withIndex} to find the first entries in each collection
+     * that intersect with each other. Return their position packed in a long, with low bits representing
+     * the resultant {@code thisIndex} and high bits {@code withIndex}.
+     */
+    long findNextIntersection(int thisIndex, AbstractRanges<?> with, int withIndex);
+
+    /**
+     * Search forwards from {code thisIndex} and {@code withIndex} to find the first entries in each collection
+     * that intersect with each other. Return their position packed in a long, with low bits representing
+     * the resultant {@code thisIndex} and high bits {@code withIndex}.
+     */
+    long findNextIntersection(int thisIndex, AbstractKeys<?, ?> with, int withIndex);
+
+    /**
+     * Search forwards from {code thisIndex} and {@code withIndex} to find the first entries in each collection
+     * that intersect with each other. Return their position packed in a long, with low bits representing
+     * the resultant {@code thisIndex} and high bits {@code withIndex}.
+     */
+    long findNextIntersection(int thisIndex, Routables<K, ?> with, int withIndex);
+
+    /**
+     * Perform {@link SortedArrays#exponentialSearch} from {@code thisIndex} looking for {@code find} with behaviour of {@code search}
+     */
+    int findNext(int thisIndex, Range find, SortedArrays.Search search);
+
+    /**
+     * Perform {@link SortedArrays#exponentialSearch} from {@code thisIndex} looking for {@code find} with behaviour of {@code search}
+     */
+    int findNext(int thisIndex, K find, SortedArrays.Search search);
+
+    Routable.Kind kindOfContents();
+
+    @Inline
+    static <Input extends Routable, T> T foldl(Routables<Input, ?> inputs, AbstractRanges<?> matching, IndexedFold<? super Input, T> fold, T initialValue)
+    {
+        return Helper.foldl(Routables::findNextIntersection, Helper::findLimit, inputs, matching, fold, initialValue);
+    }
+
+    @Inline
+    static <Input extends RoutableKey, T> T foldl(AbstractKeys<Input, ?> inputs, AbstractRanges<?> matching, IndexedFold<? super Input, T> fold, T initialValue)
+    {
+        return Helper.foldl(AbstractKeys::findNextIntersection, Helper::findLimit, inputs, matching, fold, initialValue);
+    }
+
+    @Inline
+    static <Input extends RoutableKey> long foldl(AbstractKeys<Input, ?> inputs, AbstractRanges<?> matching, IndexedFoldToLong<? super Input> fold, long param, long initialValue, long terminalValue)
+    {
+        return Helper.foldl(AbstractKeys::findNextIntersection, Helper::findLimit, inputs, matching, fold, param, initialValue, terminalValue);
+    }
+
+    @Inline
+    static <Input extends Routable> long foldl(Routables<Input, ?> inputs, AbstractRanges<?> matching, IndexedFoldToLong<? super Input> fold, long param, long initialValue, long terminalValue)
+    {
+        return Helper.foldl(Routables::findNextIntersection, Helper::findLimit, inputs, matching, fold, param, initialValue, terminalValue);
+    }
+
+    @Inline
+    static <Input extends Routable> long foldl(Routables<Input, ?> inputs, AbstractKeys<?, ?> matching, IndexedFoldToLong<? super Input> fold, long param, long initialValue, long terminalValue)
+    {
+        return Helper.foldl(Routables::findNextIntersection, (ls, li, rs, ri) -> li + 1,
+                inputs, matching, fold, param, initialValue, terminalValue);
+    }
+
+    @Inline
+    static <Input extends RoutingKey, Matching extends Routable> long foldl(AbstractKeys<Input, ?> inputs, Routables<Matching, ?> matching, IndexedFoldToLong<? super Input> fold, long param, long initialValue, long terminalValue)
+    {
+        return Helper.foldl((ls, li, rs, ri) -> SortedArrays.swapHighLow32b(rs.findNextIntersection(ri, ls, li)), (ls, li, rs, ri) -> li + 1,
+                inputs, matching, fold, param, initialValue, terminalValue);
+    }
+
+    @Inline
+    static <Input extends Routable> long rangeFoldl(Routables<Input, ?> inputs, AbstractRanges<?> matching, IndexedRangeFoldToLong fold, long param, long initialValue, long terminalValue)
+    {
+        return Helper.rangeFoldl(Routables::findNextIntersection, (ls, li, rs, ri) -> li + 1,
+                inputs, matching, fold, param, initialValue, terminalValue);
+    }
+
+    @Inline
+    static <Input extends Routable> long rangeFoldl(Routables<Input, ?> inputs, AbstractKeys<?, ?> matching, IndexedRangeFoldToLong fold, long param, long initialValue, long terminalValue)
+    {
+        return Helper.rangeFoldl(Routables::findNextIntersection, (ls, li, rs, ri) -> li + 1,
+                inputs, matching, fold, param, initialValue, terminalValue);
+    }
+
+    @Inline
+    static <Input extends Routable> long foldlMissing(Routables<Input, ?> inputs, Routables<Input, ?> notMatching, IndexedFoldToLong<? super Input> fold, long param, long initialValue, long terminalValue)
+    {
+        return Helper.foldlMissing((ls, li, rs, ri) -> rs.findNextIntersection(ri, ls, li), (ls, li, rs, ri) -> li + 1,
+                inputs, notMatching, fold, param, initialValue, terminalValue);
+    }
+
+    class Helper
+    {
+        interface SetIntersections<L extends Routables<?, ?>, R extends Routables<?, ?>>
+        {
+            long findNext(L left, int li, R right, int ri);
+        }
+
+        interface ValueIntersections<L extends Routables<?, ?>, R extends Routables<?, ?>>
+        {
+            int findLimit(L left, int li, R right, int ri);
+        }
+
+        @Inline
+        static <Input extends Routable, Inputs extends Routables<Input, ?>, Matches extends Routables<?, ?>, T>
+        T foldl(SetIntersections<Inputs, Matches> setIntersections, ValueIntersections<Inputs, Matches> valueIntersections,
+                Inputs is, Matches ms, IndexedFold<? super Input, T> fold, T initialValue)
+        {
+            int i = 0, m = 0;
+            while (true)
+            {
+                long im = setIntersections.findNext(is, i, ms, m);
+                if (im < 0)
+                    break;
+
+                i = (int)(im);
+                m = (int)(im >>> 32);
+
+                int nexti = valueIntersections.findLimit(is, i, ms, m);
+                while (i < nexti)
+                {
+                    initialValue = fold.apply(i, is.get(i), initialValue);
+                    ++i;
+                }
+            }
+
+            return initialValue;
+        }
+
+        @Inline
+        static <Input extends Routable, Inputs extends Routables<Input, ?>, Matches extends Routables<?, ?>>
+        long foldl(SetIntersections<Inputs, Matches> setIntersections, ValueIntersections<Inputs, Matches> valueIntersections,
+                   Inputs is, Matches ms, IndexedFoldToLong<? super Input> fold, long param, long initialValue, long terminalValue)
+        {
+            int i = 0, m = 0;
+            done: while (true)
+            {
+                long im = setIntersections.findNext(is, i, ms, m);
+                if (im < 0)
+                    break;
+
+                i = (int)(im);
+                m = (int)(im >>> 32);
+
+                int nexti = valueIntersections.findLimit(is, i, ms, m);
+                while (i < nexti)
+                {
+                    initialValue = fold.apply(i, is.get(i), param, initialValue);
+                    if (initialValue == terminalValue)
+                        break done;
+                    ++i;
+                }
+            }
+
+            return initialValue;
+        }
+
+        @Inline
+        static <Input extends Routable, Inputs extends Routables<Input, ?>, Matches extends Routables<?, ?>>
+        long foldlMissing(SetIntersections<Inputs, Matches> setIntersections, ValueIntersections<Inputs, Matches> valueIntersections,
+                   Inputs is, Matches ms, IndexedFoldToLong<? super Input> fold, long param, long initialValue, long terminalValue)
+        {
+            int i = 0, m = 0;
+            done: while (true)
+            {
+                long im = setIntersections.findNext(is, i, ms, m);
+                if (im < 0)
+                    break;
+
+                int nexti = (int)(im);
+                while (i < nexti)
+                {
+                    initialValue = fold.apply(i, is.get(i), param, initialValue);
+                    if (initialValue == terminalValue)
+                        break done;
+                    ++i;
+                }
+
+                m = (int)(im >>> 32);
+                i = 1 + valueIntersections.findLimit(is, nexti, ms, m);
+            }
+
+            return initialValue;
+        }
+
+        static <Input extends Routable, Inputs extends Routables<Input, ?>, Matches extends Routables<?, ?>>
+        long rangeFoldl(SetIntersections<Inputs, Matches> setIntersections, ValueIntersections<Inputs, Matches> valueIntersections,
+                        Inputs is, Matches ms, IndexedRangeFoldToLong fold, long param, long initialValue, long terminalValue)
+        {
+            int i = 0, m = 0;
+            while (true)
+            {
+                long kri = setIntersections.findNext(is, i, ms, m);
+                if (kri < 0)
+                    break;
+
+                i = (int)(kri);
+                m = (int)(kri >>> 32);
+
+                int nexti = valueIntersections.findLimit(is, i, ms, m);
+                initialValue = fold.apply(i, nexti, param, initialValue);
+                if (initialValue == terminalValue)
+                    break;
+                i = nexti;
+            }
+
+            return initialValue;
+        }
+
+        static <L extends Routable> int findLimit(Routables<L, ?> ls, int li, AbstractRanges<?> rs, int ri)
+        {
+            Range range = rs.get(ri);
+
+            int nextl = ls.findNext(li + 1, range, FLOOR);
+            if (nextl < 0) nextl = -1 - nextl;
+            else nextl++;
+            return nextl;
+        }
+    }
+}
diff --git a/accord-core/src/main/java/accord/primitives/Route.java b/accord-core/src/main/java/accord/primitives/Route.java
index 6519a28..a349b4e 100644
--- a/accord-core/src/main/java/accord/primitives/Route.java
+++ b/accord-core/src/main/java/accord/primitives/Route.java
@@ -1,52 +1,100 @@
 package accord.primitives;
 
 import accord.api.RoutingKey;
-import com.google.common.base.Preconditions;
 
-public class Route extends AbstractRoute
+import javax.annotation.Nullable;
+
+public interface Route<K extends Unseekable> extends Unseekables<K, Route<K>>
 {
-    public static class SerializationSupport
+    RoutingKey homeKey();
+
+    default boolean isRoute() { return true; }
+    boolean covers(Ranges ranges);
+    boolean intersects(AbstractRanges<?> ranges);
+    Route<K> union(Route<K> route);
+    PartialRoute<K> slice(Ranges ranges);
+    PartialRoute<K> sliceStrict(Ranges ranges);
+
+    /**
+     * @return a PureRoutables that includes every shard we know of, not just those we contact
+     * (i.e., includes the homeKey if not already included)
+     */
+    Unseekables<K, ?> toMaximalUnseekables();
+
+    // this method exists solely to circumvent JDK bug with testing and casting interfaces
+    static boolean isFullRoute(@Nullable Unseekables<?, ?> unseekables) { return unseekables != null && unseekables.kind().isFullRoute(); }
+
+    // this method exists solely to circumvent JDK bug with testing and casting interfaces
+    static boolean isRoute(@Nullable Unseekables<?, ?> unseekables) { return unseekables != null && unseekables.kind().isRoute(); }
+
+    // this method exists solely to circumvent JDK bug with testing and casting interfaces
+    static FullRoute<?> castToFullRoute(@Nullable Unseekables<?, ?> unseekables)
     {
-        public static Route create(RoutingKey homeKey, RoutingKey[] keys)
+        if (unseekables == null)
+            return null;
+
+        switch (unseekables.kindOfContents())
         {
-            return new Route(homeKey, keys);
+            default: throw new AssertionError();
+            case Key: return (FullKeyRoute) unseekables;
+            case Range: return (FullRangeRoute) unseekables;
         }
     }
 
-    public Route(RoutingKey homeKey, RoutingKey[] keys)
+    static Route<?> castToRoute(@Nullable Unseekables<?, ?> unseekables)
     {
-        super(keys, homeKey);
-    }
+        if (unseekables == null)
+            return null;
 
-    @Override
-    public boolean covers(KeyRanges ranges)
-    {
-        Preconditions.checkNotNull(ranges);
-        return true;
+        switch (unseekables.kindOfContents())
+        {
+            default: throw new AssertionError();
+            case Key: return (KeyRoute) unseekables;
+            case Range: return (RangeRoute) unseekables;
+        }
     }
 
-    @Override
-    public AbstractRoute union(AbstractRoute that)
+    // this method exists solely to circumvent JDK bug with testing and casting interfaces
+    static Route<?> tryCastToRoute(@Nullable Unseekables<?, ?> unseekables)
     {
-        Preconditions.checkNotNull(that);
-        return this;
-    }
+        if (unseekables == null)
+            return null;
 
-    @Override
-    public PartialRoute slice(KeyRanges ranges)
-    {
-        return new PartialRoute(ranges, homeKey, slice(ranges, RoutingKey[]::new));
+        switch (unseekables.kind())
+        {
+            default: throw new AssertionError();
+            case RoutingKeys:
+            case RoutingRanges:
+                return null;
+            case PartialKeyRoute:
+                return (PartialKeyRoute) unseekables;
+            case PartialRangeRoute:
+                return (PartialRangeRoute) unseekables;
+            case FullKeyRoute:
+                return (FullKeyRoute) unseekables;
+            case FullRangeRoute:
+                return (FullRangeRoute) unseekables;
+        }
     }
 
-    @Override
-    public PartialRoute sliceStrict(KeyRanges ranges)
+    // this method exists solely to circumvent JDK bug with testing and casting interfaces
+    static PartialRoute<?> castToPartialRoute(@Nullable Unseekables<?, ?> unseekables)
     {
-        return slice(ranges);
+        if (unseekables == null)
+            return null;
+
+        switch (unseekables.kindOfContents())
+        {
+            default: throw new AssertionError();
+            case Key: return (PartialKeyRoute) unseekables;
+            case Range: return (PartialRangeRoute) unseekables;
+        }
     }
 
-    @Override
-    public String toString()
+    static <T extends Unseekable> Route<T> merge(@Nullable Route<T> prefer, @Nullable Route<T> defer)
     {
-        return "{homeKey:" + homeKey + ',' + super.toString() + '}';
+        if (defer == null) return prefer;
+        if (prefer == null) return defer;
+        return prefer.union(defer);
     }
 }
diff --git a/accord-core/src/main/java/accord/primitives/RoutingKeys.java b/accord-core/src/main/java/accord/primitives/RoutingKeys.java
index c6e755b..523cce5 100644
--- a/accord-core/src/main/java/accord/primitives/RoutingKeys.java
+++ b/accord-core/src/main/java/accord/primitives/RoutingKeys.java
@@ -1,14 +1,11 @@
 package accord.primitives;
 
-import java.util.Arrays;
-import java.util.function.IntFunction;
-
-import com.google.common.base.Preconditions;
-
 import accord.api.RoutingKey;
 import accord.utils.SortedArrays;
 
-public class RoutingKeys extends AbstractKeys<RoutingKey, RoutingKeys>
+import static accord.utils.ArrayBuffers.cachedRoutingKeys;
+
+public class RoutingKeys extends AbstractRoutableKeys<AbstractRoutableKeys<?>> implements Unseekables<RoutingKey, AbstractRoutableKeys<?>>
 {
     public static class SerializationSupport
     {
@@ -20,7 +17,7 @@ public class RoutingKeys extends AbstractKeys<RoutingKey, RoutingKeys>
 
     public static final RoutingKeys EMPTY = new RoutingKeys(new RoutingKey[0]);
 
-    public RoutingKeys(RoutingKey[] keys)
+    RoutingKeys(RoutingKey[] keys)
     {
         super(keys);
     }
@@ -30,45 +27,37 @@ public class RoutingKeys extends AbstractKeys<RoutingKey, RoutingKeys>
         return new RoutingKeys(sort(keys));
     }
 
-    public RoutingKeys union(RoutingKeys that)
-    {
-        return wrap(SortedArrays.linearUnion(keys, that.keys, factory()), that);
-    }
-
-    public RoutingKeys slice(KeyRanges ranges)
+    public RoutingKeys union(AbstractRoutableKeys<?> that)
     {
-        return wrap(slice(ranges, factory()));
+        return wrap(SortedArrays.linearUnion(keys, that.keys, cachedRoutingKeys()), that);
     }
 
-    public RoutingKeys with(RoutingKey addKey)
+    public RoutingKeys with(RoutingKey with)
     {
-        return wrap(SortedArrays.insert(keys, addKey, RoutingKey[]::new));
+        if (contains(with))
+            return this;
+        return wrap(toRoutingKeysArray(with));
     }
 
-    private RoutingKeys wrap(RoutingKey[] wrap, RoutingKeys that)
+    @Override
+    public UnseekablesKind kind()
     {
-        return wrap == keys ? this : wrap == that.keys ? that : new RoutingKeys(wrap);
+        return UnseekablesKind.RoutingKeys;
     }
 
-    private RoutingKeys wrap(RoutingKey[] wrap)
+    public RoutingKeys slice(Ranges ranges)
     {
-        return wrap == keys ? this : new RoutingKeys(wrap);
+        return wrap(SortedArrays.sliceWithMultipleMatches(keys, ranges.ranges, RoutingKey[]::new, (k, r) -> -r.compareTo(k), Range::compareTo));
     }
 
-    public Route toRoute(RoutingKey homeKey)
+    private RoutingKeys wrap(RoutingKey[] wrap, AbstractKeys<RoutingKey, ?> that)
     {
-        Preconditions.checkNotNull(homeKey);
-        return new Route(homeKey, keys);
+        return wrap == keys ? this : wrap == that.keys && that instanceof RoutingKeys ? (RoutingKeys)that : new RoutingKeys(wrap);
     }
 
-    private static IntFunction<RoutingKey[]> factory()
+    private RoutingKeys wrap(RoutingKey[] wrap)
     {
-        return RoutingKey[]::new;
+        return wrap == keys ? this : new RoutingKeys(wrap);
     }
 
-    private static RoutingKey[] sort(RoutingKey[] keys)
-    {
-        Arrays.sort(keys);
-        return keys;
-    }
 }
diff --git a/accord-core/src/main/java/accord/primitives/Seekable.java b/accord-core/src/main/java/accord/primitives/Seekable.java
new file mode 100644
index 0000000..cc84767
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/Seekable.java
@@ -0,0 +1,8 @@
+package accord.primitives;
+
+/**
+ * Something that can be found within the cluster AND found on disk, queried and returned
+ */
+public interface Seekable extends Routable
+{
+}
diff --git a/accord-core/src/main/java/accord/primitives/Seekables.java b/accord-core/src/main/java/accord/primitives/Seekables.java
new file mode 100644
index 0000000..da3a532
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/Seekables.java
@@ -0,0 +1,15 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+
+/**
+ * Either a Route or a collection of Routable
+ */
+public interface Seekables<K extends Seekable, U extends Seekables<K, ?>> extends Routables<K, U>
+{
+    U slice(Ranges ranges);
+    Seekables<K, U> union(U with);
+    Unseekables<?, ?> toUnseekables();
+
+    FullRoute<?> toRoute(RoutingKey homeKey);
+}
diff --git a/accord-core/src/main/java/accord/primitives/Txn.java b/accord-core/src/main/java/accord/primitives/Txn.java
index 38207cc..d475119 100644
--- a/accord-core/src/main/java/accord/primitives/Txn.java
+++ b/accord-core/src/main/java/accord/primitives/Txn.java
@@ -48,12 +48,12 @@ public interface Txn
     class InMemory implements Txn
     {
         private final Kind kind;
-        private final Keys keys;
+        private final Seekables<?, ?> keys;
         private final Read read;
         private final Query query;
         private final Update update;
 
-        public InMemory(@Nonnull Keys keys, @Nonnull Read read, @Nonnull Query query)
+        public InMemory(@Nonnull Seekables<?, ?> keys, @Nonnull Read read, @Nonnull Query query)
         {
             this.kind = Kind.READ;
             this.keys = keys;
@@ -71,7 +71,7 @@ public interface Txn
             this.query = query;
         }
 
-        protected InMemory(@Nonnull Kind kind, @Nonnull Keys keys, @Nonnull Read read, @Nullable Query query, @Nullable Update update)
+        protected InMemory(@Nonnull Kind kind, @Nonnull Seekables<?, ?> keys, @Nonnull Read read, @Nullable Query query, @Nullable Update update)
         {
             this.kind = kind;
             this.keys = keys;
@@ -81,7 +81,7 @@ public interface Txn
         }
 
         @Override
-        public PartialTxn slice(KeyRanges ranges, boolean includeQuery)
+        public PartialTxn slice(Ranges ranges, boolean includeQuery)
         {
             return new PartialTxn.InMemory(
                     ranges, kind(), keys().slice(ranges),
@@ -97,7 +97,7 @@ public interface Txn
         }
 
         @Override
-        public Keys keys()
+        public Seekables<?, ?> keys()
         {
             return keys;
         }
@@ -146,12 +146,12 @@ public interface Txn
     }
 
     @Nonnull Kind kind();
-    @Nonnull Keys keys();
+    @Nonnull Seekables<?, ?> keys();
     @Nonnull Read read();
     @Nullable Query query(); // may be null only in PartialTxn
     @Nullable Update update();
 
-    @Nonnull PartialTxn slice(KeyRanges ranges, boolean includeQuery);
+    @Nonnull PartialTxn slice(Ranges ranges, boolean includeQuery);
 
     default boolean isWrite()
     {
@@ -174,7 +174,7 @@ public interface Txn
 
     default Future<Data> read(SafeCommandStore safeStore, Command command)
     {
-        KeyRanges ranges = safeStore.ranges().at(command.executeAt().epoch);
+        Ranges ranges = safeStore.ranges().at(command.executeAt().epoch);
         List<Future<Data>> futures = read().keys().foldl(ranges, (index, key, accumulate) -> {
             if (!safeStore.commandStore().hashIntersects(key))
                 return accumulate;
diff --git a/accord-core/src/main/java/accord/primitives/Unseekable.java b/accord-core/src/main/java/accord/primitives/Unseekable.java
new file mode 100644
index 0000000..f398b52
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/Unseekable.java
@@ -0,0 +1,8 @@
+package accord.primitives;
+
+/**
+ * Something that can only be routed, i.e. is NOT a Seekable.
+ */
+public interface Unseekable extends Routable
+{
+}
diff --git a/accord-core/src/main/java/accord/primitives/Unseekables.java b/accord-core/src/main/java/accord/primitives/Unseekables.java
new file mode 100644
index 0000000..68e0206
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/Unseekables.java
@@ -0,0 +1,59 @@
+package accord.primitives;
+
+import accord.api.RoutingKey;
+
+/**
+ * Either a Route or a simple collection of Unseekable
+ */
+public interface Unseekables<K extends Unseekable, U extends Unseekables<K, ?>> extends Iterable<K>, Routables<K, U>
+{
+    enum UnseekablesKind
+    {
+        RoutingKeys, PartialKeyRoute, FullKeyRoute, RoutingRanges, PartialRangeRoute, FullRangeRoute;
+
+        public boolean isRoute()
+        {
+            return this != RoutingKeys & this != RoutingRanges;
+        }
+
+        public boolean isFullRoute()
+        {
+            return this == FullKeyRoute | this == FullRangeRoute;
+        }
+    }
+
+    U slice(Ranges ranges);
+    Unseekables<K, U> union(U with);
+    Unseekables<K, ?> with(RoutingKey withKey);
+    UnseekablesKind kind();
+
+    static <K extends Unseekable> Unseekables<K, ?> merge(Unseekables<K, ?> left, Unseekables<K, ?> right)
+    {
+        if (left == null) return right;
+        if (right == null) return left;
+
+        UnseekablesKind leftKind = left.kind();
+        UnseekablesKind rightKind = right.kind();
+        if (leftKind.isRoute() || rightKind.isRoute())
+        {
+            if (leftKind.isRoute() != rightKind.isRoute())
+            {
+                // one is a route, one is not
+                if (leftKind.isRoute() && left.containsAll(right))
+                    return left;
+                if (rightKind.isRoute() && right.containsAll(left))
+                    return right;
+
+                // non-route types can always accept route types as input, so just call its union method on the other
+                return leftKind.isRoute() ? ((Unseekables)right).union(left) : ((Unseekables)left).union(right);
+            }
+
+            if (leftKind.isFullRoute())
+                return left;
+
+            if (rightKind.isFullRoute())
+                return right;
+        }
+        return ((Unseekables)left).union(right);
+    }
+}
diff --git a/accord-core/src/main/java/accord/primitives/Writes.java b/accord-core/src/main/java/accord/primitives/Writes.java
index f1702f8..2986dd0 100644
--- a/accord-core/src/main/java/accord/primitives/Writes.java
+++ b/accord-core/src/main/java/accord/primitives/Writes.java
@@ -19,13 +19,8 @@
 package accord.primitives;
 
 import accord.api.Write;
-import accord.local.CommandStore;
 import accord.local.SafeCommandStore;
-import accord.primitives.Keys;
-import accord.primitives.Timestamp;
-import accord.primitives.KeyRanges;
 import accord.utils.ReducingFuture;
-import com.google.common.base.Preconditions;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 
@@ -72,7 +67,7 @@ public class Writes
         if (write == null)
             return SUCCESS;
 
-        KeyRanges ranges = safeStore.ranges().since(executeAt.epoch);
+        Ranges ranges = safeStore.ranges().since(executeAt.epoch);
         if (ranges == null)
             return SUCCESS;
 
diff --git a/accord-core/src/main/java/accord/primitives/package-info.java b/accord-core/src/main/java/accord/primitives/package-info.java
new file mode 100644
index 0000000..f25136d
--- /dev/null
+++ b/accord-core/src/main/java/accord/primitives/package-info.java
@@ -0,0 +1,10 @@
+package accord.primitives;
+
+/**
+ * Routable: a RoutingKey or Range (of RoutingKey). Something that can address a replica in the cluster.
+ * Seekable: a Key or Range (of either RoutingKey or Key). Something that can address some physical data on a node.
+ * Routables: a collection of Routable
+ * Seekables: a collection of Seekable
+ * Route: a collection of Routable including a homeKey. Represents a consistent slice (or slices) of token ranges.
+ *        Either a PartialRoute or a FullRoute.
+ */
\ No newline at end of file
diff --git a/accord-core/src/main/java/accord/topology/Shard.java b/accord-core/src/main/java/accord/topology/Shard.java
index 8a729dc..0e67f77 100644
--- a/accord-core/src/main/java/accord/topology/Shard.java
+++ b/accord-core/src/main/java/accord/topology/Shard.java
@@ -20,23 +20,23 @@ package accord.topology;
 
 import java.util.Collections;
 import java.util.List;
-import java.util.Objects;
 import java.util.Set;
 
 import accord.local.Node.Id;
 import accord.api.Key;
-import accord.primitives.KeyRange;
+import accord.primitives.Range;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 
+import static accord.utils.Invariants.checkArgument;
+
 // TODO: concept of region/locality
 public class Shard
 {
-    public final KeyRange range;
+    public final Range range;
     // TODO: use BTreeSet to combine these two (or introduce version that operates over long values)
     public final List<Id> nodes;
     public final Set<Id> nodeSet;
@@ -47,24 +47,22 @@ public class Shard
     public final int fastPathQuorumSize;
     public final int slowPathQuorumSize;
 
-    public Shard(KeyRange range, List<Id> nodes, Set<Id> fastPathElectorate, Set<Id> joining)
+    public Shard(Range range, List<Id> nodes, Set<Id> fastPathElectorate, Set<Id> joining)
     {
-        Preconditions.checkArgument(Iterables.all(joining, nodes::contains),
-                                    "joining nodes must also be present in nodes");
         this.range = range;
         this.nodes = ImmutableList.copyOf(nodes);
-        this.nodeSet = ImmutableSet.copyOf(nodes);
-        Preconditions.checkArgument(nodes.size() == nodeSet.size());
+        this.nodeSet = checkArgument(ImmutableSet.copyOf(nodes), set -> set.size() == nodes.size());
         this.maxFailures = maxToleratedFailures(nodes.size());
         this.fastPathElectorate = ImmutableSet.copyOf(fastPathElectorate);
-        this.joining = ImmutableSet.copyOf(joining);
+        this.joining = checkArgument(ImmutableSet.copyOf(joining), Iterables.all(joining, nodes::contains),
+                "joining nodes must also be present in nodes");
         int e = fastPathElectorate.size();
         this.recoveryFastPathSize = (maxFailures+1)/2;
         this.slowPathQuorumSize = slowPathQuorumSize(nodes.size());
         this.fastPathQuorumSize = fastPathQuorumSize(nodes.size(), e, maxFailures);
     }
 
-    public Shard(KeyRange range, List<Id> nodes, Set<Id> fastPathElectorate)
+    public Shard(Range range, List<Id> nodes, Set<Id> fastPathElectorate)
     {
         this(range, nodes, fastPathElectorate, Collections.emptySet());
     }
@@ -78,7 +76,7 @@ public class Shard
     @VisibleForTesting
     static int fastPathQuorumSize(int replicas, int electorate, int f)
     {
-        Preconditions.checkArgument(electorate >= replicas - f);
+        checkArgument(electorate >= replicas - f);
         return (f + electorate)/2 + 1;
     }
 
diff --git a/accord-core/src/main/java/accord/topology/Topologies.java b/accord-core/src/main/java/accord/topology/Topologies.java
index 6de8ddb..4881f9e 100644
--- a/accord-core/src/main/java/accord/topology/Topologies.java
+++ b/accord-core/src/main/java/accord/topology/Topologies.java
@@ -21,10 +21,10 @@ package accord.topology;
 import accord.api.TopologySorter;
 import accord.local.Node;
 import accord.local.Node.Id;
-import accord.primitives.KeyRanges;
+import accord.primitives.Ranges;
 import accord.utils.IndexedConsumer;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
+import accord.utils.Invariants;
 
 import java.util.*;
 
@@ -58,7 +58,7 @@ public interface Topologies extends TopologySorter
 
     int estimateUniqueNodes();
 
-    KeyRanges computeRangesForNode(Id node);
+    Ranges computeRangesForNode(Id node);
 
     int maxShardsPerEpoch();
 
@@ -193,7 +193,7 @@ public interface Topologies extends TopologySorter
         }
 
         @Override
-        public KeyRanges computeRangesForNode(Id node)
+        public Ranges computeRangesForNode(Id node)
         {
             return topology.rangesForNode(node);
         }
@@ -329,9 +329,9 @@ public interface Topologies extends TopologySorter
         }
 
         @Override
-        public KeyRanges computeRangesForNode(Id node)
+        public Ranges computeRangesForNode(Id node)
         {
-            KeyRanges ranges = KeyRanges.EMPTY;
+            Ranges ranges = Ranges.EMPTY;
             for (int i = 0, mi = size() ; i < mi ; i++)
                 ranges = ranges.union(get(i).rangesForNode(node));
             return ranges;
@@ -345,7 +345,7 @@ public interface Topologies extends TopologySorter
 
         public void add(Topology topology)
         {
-            Preconditions.checkArgument(topologies.isEmpty() || topology.epoch == topologies.get(topologies.size() - 1).epoch - 1);
+            Invariants.checkArgument(topologies.isEmpty() || topology.epoch == topologies.get(topologies.size() - 1).epoch - 1);
             topologies.add(topology);
         }
 
diff --git a/accord-core/src/main/java/accord/topology/Topology.java b/accord-core/src/main/java/accord/topology/Topology.java
index e457a92..d7e93a7 100644
--- a/accord-core/src/main/java/accord/topology/Topology.java
+++ b/accord-core/src/main/java/accord/topology/Topology.java
@@ -25,35 +25,34 @@ import java.util.stream.IntStream;
 
 import accord.api.RoutingKey;
 import accord.local.Node.Id;
-import accord.primitives.AbstractKeys;
-import accord.primitives.KeyRange;
-import accord.primitives.KeyRanges;
+import accord.primitives.*;
 import accord.utils.*;
 
+import static accord.utils.SortedArrays.Search.FLOOR;
 import static accord.utils.SortedArrays.exponentialSearch;
 
 public class Topology
 {
-    public static final Topology EMPTY = new Topology(0, new Shard[0], KeyRanges.EMPTY, Collections.emptyMap(), KeyRanges.EMPTY, new int[0]);
+    public static final Topology EMPTY = new Topology(0, new Shard[0], Ranges.EMPTY, Collections.emptyMap(), Ranges.EMPTY, new int[0]);
     final long epoch;
     final Shard[] shards;
-    final KeyRanges ranges;
+    final Ranges ranges;
     final Map<Id, NodeInfo> nodeLookup;
-    final KeyRanges subsetOfRanges;
     /**
      * This array is used to permit cheaper sharing of Topology objects between requests, as we must only specify
      * the indexes within the parent Topology that we contain. This also permits us to perform efficient merges with
      * {@code NodeInfo.supersetIndexes} to find the shards that intersect a given node without recomputing the NodeInfo.
      * TODO: do not recompute nodeLookup
      */
+    final Ranges subsetOfRanges;
     final int[] supersetIndexes;
 
     static class NodeInfo
     {
-        final KeyRanges ranges;
+        final Ranges ranges;
         final int[] supersetIndexes;
 
-        NodeInfo(KeyRanges ranges, int[] supersetIndexes)
+        NodeInfo(Ranges ranges, int[] supersetIndexes)
         {
             this.ranges = ranges;
             this.supersetIndexes = supersetIndexes;
@@ -69,7 +68,7 @@ public class Topology
     public Topology(long epoch, Shard... shards)
     {
         this.epoch = epoch;
-        this.ranges = KeyRanges.ofSortedAndDeoverlapped(Arrays.stream(shards).map(shard -> shard.range).toArray(KeyRange[]::new));
+        this.ranges = Ranges.ofSortedAndDeoverlapped(Arrays.stream(shards).map(shard -> shard.range).toArray(Range[]::new));
         this.shards = shards;
         this.subsetOfRanges = ranges;
         this.supersetIndexes = IntStream.range(0, shards.length).toArray();
@@ -83,12 +82,12 @@ public class Topology
         for (Map.Entry<Id, List<Integer>> e : build.entrySet())
         {
             int[] supersetIndexes = e.getValue().stream().mapToInt(i -> i).toArray();
-            KeyRanges ranges = this.ranges.select(supersetIndexes);
+            Ranges ranges = this.ranges.select(supersetIndexes);
             nodeLookup.put(e.getKey(), new NodeInfo(ranges, supersetIndexes));
         }
     }
 
-    public Topology(long epoch, Shard[] shards, KeyRanges ranges, Map<Id, NodeInfo> nodeLookup, KeyRanges subsetOfRanges, int[] supersetIndexes)
+    public Topology(long epoch, Shard[] shards, Ranges ranges, Map<Id, NodeInfo> nodeLookup, Ranges subsetOfRanges, int[] supersetIndexes)
     {
         this.epoch = epoch;
         this.shards = shards;
@@ -169,16 +168,16 @@ public class Topology
         return select(epoch, shards, this.supersetIndexes);
     }
 
-    public KeyRanges rangesForNode(Id node)
+    public Ranges rangesForNode(Id node)
     {
         NodeInfo info = nodeLookup.get(node);
-        return info != null ? info.ranges : KeyRanges.EMPTY;
+        return info != null ? info.ranges : Ranges.EMPTY;
     }
 
     // TODO: optimised HomeKey concept containing the Key, Shard and Topology to avoid lookups when topology hasn't changed
     public Shard forKey(RoutingKey key)
     {
-        int i = ranges.rangeIndexForKey(key);
+        int i = ranges.indexOf(key);
         if (i < 0)
             throw new IllegalArgumentException("Range not found for " + key);
         return shards[i];
@@ -186,20 +185,34 @@ public class Topology
 
     public int indexForKey(RoutingKey key)
     {
-        int i = ranges.rangeIndexForKey(key);
+        int i = ranges.indexOf(key);
         if (i < 0) return -1;
         return Arrays.binarySearch(supersetIndexes, i);
     }
 
-    public Topology forKeys(AbstractKeys<?, ?> select)
+    public Topology forSelection(Unseekables<?, ?> select)
     {
-        return forKeys(select, (i, shard) -> true);
+        return forSelection(select, (i, shard) -> true);
     }
 
-    public Topology forKeys(AbstractKeys<?, ?> select, IndexedPredicate<Shard> predicate)
+    public Topology forSelection(Unseekables<?, ?> select, IndexedPredicate<Shard> predicate)
     {
-        int[] newSubset = subsetForKeys(select, predicate);
-        KeyRanges rangeSubset = ranges.select(newSubset);
+        return forSubset(subsetFor(select, predicate));
+    }
+
+    public Topology forSelection(Unseekables<?, ?> select, Collection<Id> nodes)
+    {
+        return forSelection(select, nodes, (i, shard) -> true);
+    }
+
+    public Topology forSelection(Unseekables<?, ?> select, Collection<Id> nodes, IndexedPredicate<Shard> predicate)
+    {
+        return forSubset(subsetFor(select, predicate), nodes);
+    }
+
+    private Topology forSubset(int[] newSubset)
+    {
+        Ranges rangeSubset = ranges.select(newSubset);
 
         // TODO: more efficient sharing of nodeLookup state
         Map<Id, NodeInfo> nodeLookup = new HashMap<>();
@@ -212,15 +225,9 @@ public class Topology
         return new Topology(epoch, shards, ranges, nodeLookup, rangeSubset, newSubset);
     }
 
-    public Topology forKeys(AbstractKeys<?, ?> select, Collection<Id> nodes)
+    private Topology forSubset(int[] newSubset, Collection<Id> nodes)
     {
-        return forKeys(select, nodes, (i, shard) -> true);
-    }
-
-    public Topology forKeys(AbstractKeys<?, ?> select, Collection<Id> nodes, IndexedPredicate<Shard> predicate)
-    {
-        int[] newSubset = subsetForKeys(select, predicate);
-        KeyRanges rangeSubset = ranges.select(newSubset);
+        Ranges rangeSubset = ranges.select(newSubset);
 
         // TODO: more efficient sharing of nodeLookup state
         Map<Id, NodeInfo> nodeLookup = new HashMap<>();
@@ -229,33 +236,66 @@ public class Topology
         return new Topology(epoch, shards, ranges, nodeLookup, rangeSubset, newSubset);
     }
 
-    private int[] subsetForKeys(AbstractKeys<?, ?> select, IndexedPredicate<Shard> predicate)
+    private int[] subsetFor(Unseekables<?, ?> select, IndexedPredicate<Shard> predicate)
     {
-        int subsetIndex = 0;
         int count = 0;
         int[] newSubset = new int[Math.min(select.size(), subsetOfRanges.size())];
-        for (int i = 0 ; i < select.size() ; )
+        Unseekables<?, ?> as = select;
+        Ranges bs = subsetOfRanges;
+        int ai = 0, bi = 0;
+        // ailim tracks which ai have been included; since there may be multiple matches
+        // we cannot increment ai to avoid missing a match with a second bi
+        int ailim = 0;
+
+        if (subsetOfRanges == ranges)
         {
-            // TODO: use SortedArrays.findNextIntersection
-            // find the range containing the key at i
-            subsetIndex = subsetOfRanges.rangeIndexForKey(subsetIndex, subsetOfRanges.size(), select.get(i));
-            if (subsetIndex < 0 || subsetIndex >= subsetOfRanges.size())
-                throw new IllegalArgumentException("Range not found for " + select.get(i));
-            int supersetIndex = supersetIndexes[subsetIndex];
-            Shard shard = shards[supersetIndex];
-            if (predicate.test(subsetIndex, shard))
-                newSubset[count++] = supersetIndex;
-            // find the first key outside this range
-            i = shard.range.nextHigherKeyIndex(select, i);
+            while (true)
+            {
+                long abi = as.findNextIntersection(ai, bs, bi);
+                if (abi < 0)
+                {
+                    if (ailim < select.size())
+                        throw new IllegalArgumentException("Range not found for " + select.get(ailim));
+                    break;
+                }
+
+                bi = (int)(abi >>> 32);
+                if (ailim < (int)abi)
+                    throw new IllegalArgumentException("Range not found for " + select.get(ailim));
+
+                if (predicate.test(bi, shards[bi]))
+                    newSubset[count++] = bi;
+
+                ai = (int)abi;
+                ailim = as.findNext(ai + 1, bs.get(bi), FLOOR);
+                if (ailim < 0) ailim = -1 - ailim;
+                else ailim++;
+                ++bi;
+            }
+        }
+        else
+        {
+            while (true)
+            {
+                long abi = as.findNextIntersection(ai, bs, bi);
+                if (abi < 0)
+                    break;
+
+                bi = (int)(abi >>> 32);
+                if (predicate.test(bi, shards[bi]))
+                    newSubset[count++] = bi;
+
+                ++bi;
+            }
         }
         if (count != newSubset.length)
             newSubset = Arrays.copyOf(newSubset, count);
         return newSubset;
     }
 
-    public void visitNodeForKeysOnceOrMore(AbstractKeys<?, ?> select, IndexedPredicate<Shard> predicate, Consumer<Id> nodes)
+    public void visitNodeForKeysOnceOrMore(Unseekables<?, ?> select, IndexedPredicate<Shard> predicate, Consumer<Id> nodes)
     {
-        for (int shardIndex : subsetForKeys(select, predicate))
+        for (int shardIndex : subsetFor(select, predicate))
         {
             Shard shard = shards[shardIndex];
             for (Id id : shard.nodes)
@@ -263,22 +303,25 @@ public class Topology
         }
     }
 
-    public <T> T foldl(AbstractKeys<?, ?> select, IndexedBiFunction<Shard, T, T> function, T accumulator)
+    public <T> T foldl(Unseekables<?, ?> select, IndexedBiFunction<Shard, T, T> function, T accumulator)
     {
-        int subsetIndex = 0;
-        // TODO: use SortedArrays.findNextIntersection
-        for (int i = 0 ; i < select.size() ; )
+        Unseekables<?, ?> as = select;
+        Ranges bs = ranges;
+        int ai = 0, bi = 0;
+
+        while (true)
         {
-            // find the range containing the key at i
-            subsetIndex = subsetOfRanges.rangeIndexForKey(subsetIndex, subsetOfRanges.size(), select.get(i));
-            if (subsetIndex < 0 || subsetIndex >= subsetOfRanges.size())
-                throw new IllegalArgumentException("Range not found for " + select.get(i));
-            int supersetIndex = supersetIndexes[subsetIndex];
-            Shard shard = shards[supersetIndex];
-            accumulator = function.apply(subsetIndex, shard, accumulator);
-            // find the first key outside this range
-            i = shard.range.nextHigherKeyIndex(select, i);
+            long abi = as.findNextIntersection(ai, bs, bi);
+            if (abi < 0)
+                break;
+
+            ai = (int)(abi);
+            bi = (int)(abi >>> 32);
+
+            accumulator = function.apply(bi, shards[bi], accumulator);
+            ++bi;
         }
+
         return accumulator;
     }
 
@@ -458,7 +501,7 @@ public class Topology
         return nodeLookup.keySet();
     }
 
-    public KeyRanges ranges()
+    public Ranges ranges()
     {
         return ranges;
     }
diff --git a/accord-core/src/main/java/accord/topology/TopologyManager.java b/accord-core/src/main/java/accord/topology/TopologyManager.java
index 5b84e22..2e7de96 100644
--- a/accord-core/src/main/java/accord/topology/TopologyManager.java
+++ b/accord-core/src/main/java/accord/topology/TopologyManager.java
@@ -23,16 +23,11 @@ import accord.api.RoutingKey;
 import accord.api.TopologySorter;
 import accord.coordinate.tracking.QuorumTracker;
 import accord.local.Node.Id;
-import accord.messages.EpochRequest;
 import accord.messages.Request;
-import accord.primitives.AbstractKeys;
-import accord.primitives.KeyRanges;
-import accord.primitives.Keys;
-import accord.primitives.Txn;
+import accord.primitives.*;
 import accord.topology.Topologies.Single;
-import accord.primitives.Timestamp;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 import org.apache.cassandra.utils.concurrent.AsyncPromise;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.cassandra.utils.concurrent.ImmediateFuture;
@@ -41,6 +36,8 @@ import java.util.*;
 
 import static accord.coordinate.tracking.RequestStatus.Success;
 
+import static accord.utils.Invariants.checkArgument;
+
 /**
  * Manages topology state changes and update bookkeeping
  *
@@ -68,10 +65,10 @@ public class TopologyManager implements ConfigurationService.Listener
 
         EpochState(Id node, Topology global, TopologySorter sorter, boolean prevSynced)
         {
-            this.global = global;
             this.self = node;
+            this.global = checkArgument(global, !global.isSubset());
             this.local = global.forNode(node).trim();
-            Preconditions.checkArgument(!global().isSubset());
+            Invariants.checkArgument(!global().isSubset());
             this.syncTracker = new QuorumTracker(new Single(sorter, global()));
             this.prevSynced = prevSynced;
         }
@@ -110,13 +107,13 @@ public class TopologyManager implements ConfigurationService.Listener
         /**
          * determine if sync has completed for all shards intersecting with the given keys
          */
-        boolean syncCompleteFor(AbstractKeys<?, ?> keys)
+        boolean syncCompleteFor(Unseekables<?, ?> intersect)
         {
             if (!prevSynced)
                 return false;
             if (syncComplete)
                 return true;
-            Boolean result = global().foldl(keys, (i, shard, acc) -> {
+            Boolean result = global().foldl(intersect, (i, shard, acc) -> {
                 if (acc == Boolean.FALSE)
                     return acc;
                 return syncTracker.get(i).hasReachedQuorum();
@@ -151,7 +148,7 @@ public class TopologyManager implements ConfigurationService.Listener
             this.pendingSyncComplete = pendingSyncComplete;
             this.futureEpochFutures = futureEpochFutures;
             for (int i=1; i<epochs.length; i++)
-                Preconditions.checkArgument(epochs[i].epoch() == epochs[i-1].epoch() - 1);
+                checkArgument(epochs[i].epoch() == epochs[i-1].epoch() - 1);
             this.epochs = epochs;
         }
 
@@ -188,7 +185,7 @@ public class TopologyManager implements ConfigurationService.Listener
          */
         public void syncComplete(Id node, long epoch)
         {
-            Preconditions.checkArgument(epoch > 0);
+            checkArgument(epoch > 0);
             if (epoch > currentEpoch - 1)
             {
                 int idx = (int) (epoch - currentEpoch);
@@ -217,11 +214,11 @@ public class TopologyManager implements ConfigurationService.Listener
             return epochs[(int) (currentEpoch - epoch)];
         }
 
-        boolean requiresHistoricalTopologiesFor(AbstractKeys<?, ?> keys, long epoch)
+        boolean requiresHistoricalTopologiesFor(Unseekables<?, ?> intersect, long epoch)
         {
-            Preconditions.checkState(epoch <= currentEpoch);
+            Invariants.checkState(epoch <= currentEpoch);
             EpochState state = get(epoch - 1);
-            return state != null && !state.syncCompleteFor(keys);
+            return state != null && !state.syncCompleteFor(intersect);
         }
     }
 
@@ -241,7 +238,7 @@ public class TopologyManager implements ConfigurationService.Listener
     {
         Epochs current = epochs;
 
-        Preconditions.checkArgument(topology.epoch == current.nextEpoch());
+        checkArgument(topology.epoch == current.nextEpoch());
         EpochState[] nextEpochs = new EpochState[current.epochs.length + 1];
         List<Set<Id>> pendingSync = new ArrayList<>(current.pendingSyncComplete);
         if (!pendingSync.isEmpty())
@@ -295,21 +292,36 @@ public class TopologyManager implements ConfigurationService.Listener
         return epochs.get(epoch);
     }
 
-    public Topologies withUnsyncedEpochs(AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch)
+    public Topologies withUnsyncedEpochs(Unseekables<?, ?> select, Timestamp at)
+    {
+        return withUnsyncedEpochs(select, at.epoch);
+    }
+
+    public Topologies withUnsyncedEpochs(Unseekables<?, ?> select, long epoch)
+    {
+        return withUnsyncedEpochs(select, epoch, epoch);
+    }
+
+    public Topologies withUnsyncedEpochs(Unseekables<?, ?> select, Timestamp min, Timestamp max)
+    {
+        return withUnsyncedEpochs(select, min.epoch, max.epoch);
+    }
+
+    public Topologies withUnsyncedEpochs(Unseekables<?, ?> select, long minEpoch, long maxEpoch)
     {
         Epochs snapshot = epochs;
 
         if (maxEpoch == Long.MAX_VALUE) maxEpoch = snapshot.currentEpoch;
-        else Preconditions.checkState(snapshot.currentEpoch >= maxEpoch);
+        else Invariants.checkState(snapshot.currentEpoch >= maxEpoch);
 
         EpochState maxEpochState = snapshot.get(maxEpoch);
-        if (minEpoch == maxEpoch && !snapshot.requiresHistoricalTopologiesFor(keys, maxEpoch))
-            return new Single(sorter, maxEpochState.global.forKeys(keys));
+        if (minEpoch == maxEpoch && !snapshot.requiresHistoricalTopologiesFor(select, maxEpoch))
+            return new Single(sorter, maxEpochState.global.forSelection(select));
 
         int start = (int)(snapshot.currentEpoch - maxEpoch);
         int limit = (int)(Math.min(1 + snapshot.currentEpoch - minEpoch, snapshot.epochs.length));
         int count = limit - start;
-        while (limit < snapshot.epochs.length && !snapshot.epochs[limit].syncCompleteFor(keys))
+        while (limit < snapshot.epochs.length && !snapshot.epochs[limit].syncCompleteFor(select))
         {
             ++count;
             ++limit;
@@ -322,9 +334,9 @@ public class TopologyManager implements ConfigurationService.Listener
         {
             EpochState epochState = snapshot.epochs[i];
             if (epochState.epoch() < minEpoch)
-                epochState.global.visitNodeForKeysOnceOrMore(keys, epochState::shardIsUnsynced, nodes::add);
+                epochState.global.visitNodeForKeysOnceOrMore(select, epochState::shardIsUnsynced, nodes::add);
             else
-                epochState.global.visitNodeForKeysOnceOrMore(keys, (i1, i2) -> true, nodes::add);
+                epochState.global.visitNodeForKeysOnceOrMore(select, (i1, i2) -> true, nodes::add);
         }
 
         Topologies.Multi topologies = new Topologies.Multi(sorter, count);
@@ -332,25 +344,20 @@ public class TopologyManager implements ConfigurationService.Listener
         {
             EpochState epochState = snapshot.epochs[i];
             if (epochState.epoch() < minEpoch)
-                topologies.add(epochState.global.forKeys(keys, nodes, epochState::shardIsUnsynced));
+                topologies.add(epochState.global.forSelection(select, nodes, epochState::shardIsUnsynced));
             else
-                topologies.add(epochState.global.forKeys(keys, nodes, (i1, i2) -> true));
+                topologies.add(epochState.global.forSelection(select, nodes, (i1, i2) -> true));
         }
 
         return topologies;
     }
 
-    public Topologies withUnsyncedEpochs(Keys keys, long epoch)
-    {
-        return withUnsyncedEpochs(keys, epoch, epoch);
-    }
-
-    public Topologies preciseEpochs(AbstractKeys<?, ?> keys, long minEpoch, long maxEpoch)
+    public Topologies preciseEpochs(Unseekables<?, ?> keys, long minEpoch, long maxEpoch)
     {
         Epochs snapshot = epochs;
 
         if (minEpoch == maxEpoch)
-            return new Single(sorter, snapshot.get(minEpoch).global.forKeys(keys));
+            return new Single(sorter, snapshot.get(minEpoch).global.forSelection(keys));
 
         Set<Id> nodes = new LinkedHashSet<>();
         int count = (int)(1 + maxEpoch - minEpoch);
@@ -359,15 +366,15 @@ public class TopologyManager implements ConfigurationService.Listener
 
         Topologies.Multi topologies = new Topologies.Multi(sorter, count);
         for (int i = count - 1 ; i >= 0 ; --i)
-            topologies.add(snapshot.get(minEpoch + i).global.forKeys(keys, nodes));
+            topologies.add(snapshot.get(minEpoch + i).global.forSelection(keys, nodes));
 
         return topologies;
     }
 
-    public Topologies forEpoch(AbstractKeys<?, ?> keys, long epoch)
+    public Topologies forEpoch(Unseekables<?, ?> select, long epoch)
     {
         EpochState state = epochs.get(epoch);
-        return new Single(sorter, state.global.forKeys(keys));
+        return new Single(sorter, state.global.forSelection(select));
     }
 
     public Shard forEpochIfKnown(RoutingKey key, long epoch)
@@ -391,40 +398,20 @@ public class TopologyManager implements ConfigurationService.Listener
         return epochs.get(epoch) != null;
     }
 
-    public Topologies withUnsyncedEpochs(Txn txn, long epoch)
-    {
-        return withUnsyncedEpochs(txn.keys(), epoch, epoch);
-    }
-
-    public Topologies withUnsyncedEpochs(Txn txn, long minEpoch, long maxEpoch)
-    {
-        return withUnsyncedEpochs(txn.keys(), minEpoch, maxEpoch);
-    }
-
-    public Topologies withUnsyncedEpochs(AbstractKeys<?, ?> keys, long epoch)
-    {
-        return withUnsyncedEpochs(keys, epoch, epoch);
-    }
-
-    public Topologies withUnsyncedEpochs(AbstractKeys<?, ?> keys, Timestamp min, Timestamp max)
-    {
-        return withUnsyncedEpochs(keys, min.epoch, max.epoch);
-    }
-
     public Topology localForEpoch(long epoch)
     {
         return epochs.get(epoch).local();
     }
 
-    public KeyRanges localRangesForEpoch(long epoch)
+    public Ranges localRangesForEpoch(long epoch)
     {
         return epochs.get(epoch).local().rangesForNode(node);
     }
 
-    public KeyRanges localRangesForEpochs(long start, long end)
+    public Ranges localRangesForEpochs(long start, long end)
     {
         if (end < start) throw new IllegalArgumentException();
-        KeyRanges ranges = localRangesForEpoch(start);
+        Ranges ranges = localRangesForEpoch(start);
         for (long i = start + 1; i <= end ; ++i)
             ranges = ranges.union(localRangesForEpoch(i));
         return ranges;
@@ -437,13 +424,9 @@ public class TopologyManager implements ConfigurationService.Listener
 
     public long maxUnknownEpoch(Request request)
     {
-        if (!(request instanceof EpochRequest))
-            return 0;
-
-        long waitForEpoch = ((EpochRequest) request).waitForEpoch();
+        long waitForEpoch = request.waitForEpoch();
         if (epochs.currentEpoch < waitForEpoch)
             return waitForEpoch;
-
         return 0;
     }
 }
diff --git a/accord-core/src/main/java/accord/utils/ArrayBuffers.java b/accord-core/src/main/java/accord/utils/ArrayBuffers.java
index c022acf..b035e26 100644
--- a/accord-core/src/main/java/accord/utils/ArrayBuffers.java
+++ b/accord-core/src/main/java/accord/utils/ArrayBuffers.java
@@ -19,13 +19,16 @@
 package accord.utils;
 
 import accord.api.Key;
+import accord.api.RoutingKey;
+import accord.primitives.Range;
 import accord.primitives.TxnId;
-import com.google.common.base.Preconditions;
 
 import java.lang.reflect.Array;
 import java.util.Arrays;
 import java.util.function.IntFunction;
 
+import static accord.utils.Invariants.checkArgument;
+
 /**
  * A set of utility classes and interfaces for managing a collection of buffers for arrays of certain types.
  *
@@ -51,6 +54,8 @@ public class ArrayBuffers
     // TODO: we should periodically clear the thread locals to ensure we aren't slowly accumulating unnecessarily large objects on every thread
     private static final ThreadLocal<IntBufferCache> INTS = ThreadLocal.withInitial(() -> new IntBufferCache(4, 1 << 14));
     private static final ThreadLocal<ObjectBufferCache<Key>> KEYS = ThreadLocal.withInitial(() -> new ObjectBufferCache<>(3, 1 << 9, Key[]::new));
+    private static final ThreadLocal<ObjectBufferCache<RoutingKey>> ROUTINGKEYS = ThreadLocal.withInitial(() -> new ObjectBufferCache<>(3, 1 << 9, RoutingKey[]::new));
+    private static final ThreadLocal<ObjectBufferCache<Range>> KEYRANGES = ThreadLocal.withInitial(() -> new ObjectBufferCache<>(3, 1 << 7, Range[]::new));
     private static final ThreadLocal<ObjectBufferCache<TxnId>> TXN_IDS = ThreadLocal.withInitial(() -> new ObjectBufferCache<>(3, 1 << 12, TxnId[]::new));
 
     public static IntBuffers cachedInts()
@@ -63,6 +68,16 @@ public class ArrayBuffers
         return KEYS.get();
     }
 
+    public static ObjectBuffers<RoutingKey> cachedRoutingKeys()
+    {
+        return ROUTINGKEYS.get();
+    }
+
+    public static ObjectBuffers<Range> cachedRanges()
+    {
+        return KEYRANGES.get();
+    }
+
     public static ObjectBuffers<TxnId> cachedTxnIds()
     {
         return TXN_IDS.get();
@@ -87,7 +102,7 @@ public class ArrayBuffers
          * either the buffer will be returned and the size optionally captured, or else the result may be
          * shrunk to the size of the contents, depending on implementation.
          */
-        int[] complete(int[] buffer, int size);
+        int[] complete(int[] buffer, int usedSize);
 
         /**
          * The buffer is no longer needed by the caller, which is discarding the array;
@@ -100,13 +115,13 @@ public class ArrayBuffers
          *
          * @return true if the buffer is discarded (and discard-able), false if it was retained or is believed to be in use
          */
-        boolean discard(int[] buffer, int size);
+        boolean discard(int[] buffer, int usedSize);
 
         /**
          * Indicate this buffer is definitely unused, and return it to a pool if possible
          * @return true if the buffer is discarded (and discard-able), false if it was retained
          */
-        boolean forceDiscard(int[] buffer);
+        boolean forceDiscard(int[] buffer, int usedSize);
     }
 
     public interface ObjectBuffers<T>
@@ -177,22 +192,22 @@ public class ArrayBuffers
         }
 
         @Override
-        public int[] complete(int[] buffer, int size)
+        public int[] complete(int[] buffer, int usedSize)
         {
-            if (size == buffer.length)
+            if (usedSize == buffer.length)
                 return buffer;
 
-            return Arrays.copyOf(buffer, size);
+            return Arrays.copyOf(buffer, usedSize);
         }
 
         @Override
-        public boolean discard(int[] buffer, int size)
+        public boolean discard(int[] buffer, int usedSize)
         {
-            return forceDiscard(buffer);
+            return forceDiscard(buffer, usedSize);
         }
 
         @Override
-        public boolean forceDiscard(int[] buffer)
+        public boolean forceDiscard(int[] buffer, int usedSize)
         {
             // if FULLY_UNCACHED we want our caller to also not cache us, so we indicate the buffer has been retained
             return !FULLY_UNCACHED;
@@ -225,8 +240,7 @@ public class ArrayBuffers
         @Override
         public T[] completeWithExisting(T[] buffer, int size)
         {
-            Preconditions.checkArgument(buffer.length == size);
-            return buffer;
+            return checkArgument(buffer, buffer.length == size);
         }
 
         @Override
@@ -322,28 +336,28 @@ public class ArrayBuffers
     {
         IntBufferCache(int maxCount, int maxSize)
         {
-            super(int[]::new, (i1, i2) -> {}, maxCount, maxSize);
+            super(int[]::new, (array, size) -> {}, maxCount, maxSize);
         }
 
         @Override
-        public int[] complete(int[] buffer, int size)
+        public int[] complete(int[] buffer, int usedSize)
         {
-            if (size == buffer.length)
+            if (usedSize == buffer.length)
                 return buffer;
 
-            return Arrays.copyOf(buffer, size);
+            return Arrays.copyOf(buffer, usedSize);
         }
 
         @Override
-        public boolean discard(int[] buffer, int size)
+        public boolean discard(int[] buffer, int usedSize)
         {
-            return discardInternal(buffer, buffer.length, size, false);
+            return discardInternal(buffer, buffer.length, usedSize, false);
         }
 
         @Override
-        public boolean forceDiscard(int[] buffer)
+        public boolean forceDiscard(int[] buffer, int usedSize)
         {
-            return discardInternal(buffer, buffer.length, -1, true);
+            return discardInternal(buffer, buffer.length, usedSize, true);
         }
 
         @Override
@@ -522,21 +536,21 @@ public class ArrayBuffers
         }
 
         @Override
-        public int[] complete(int[] buffer, int size)
+        public int[] complete(int[] buffer, int usedSize)
         {
             return buffer;
         }
 
         @Override
-        public boolean discard(int[] buffer, int size)
+        public boolean discard(int[] buffer, int usedSize)
         {
             return false;
         }
 
         @Override
-        public boolean forceDiscard(int[] buffer)
+        public boolean forceDiscard(int[] buffer, int usedSize)
         {
-            if (!ints.forceDiscard(buffer))
+            if (!ints.forceDiscard(buffer, usedSize))
                 return false;
 
             if (savedInts != null && savedInts.length >= buffer.length)
diff --git a/accord-core/src/main/java/accord/utils/Functions.java b/accord-core/src/main/java/accord/utils/Functions.java
index 640bad8..aa61216 100644
--- a/accord-core/src/main/java/accord/utils/Functions.java
+++ b/accord-core/src/main/java/accord/utils/Functions.java
@@ -1,5 +1,6 @@
 package accord.utils;
 
+import javax.annotation.Nonnull;
 import java.util.List;
 import java.util.function.BiFunction;
 import java.util.function.Function;
@@ -12,15 +13,12 @@ public class Functions
         return a == null ? b : b == null ? a : merge.apply(a, b);
     }
 
-    public static <T> T reduceNonNull(BiFunction<T, T, T> merge, T a, T ... bs)
+    public static <T1, T2> T1 reduceNonNull(BiFunction<T1, T2, T1> merge, @Nonnull T1 a, T2 ... bs)
     {
-        for (T b : bs)
+        for (T2 b : bs)
         {
             if (b != null)
-            {
-                if (a == null) a = b;
-                else a = merge.apply(a, b);
-            }
+                a = merge.apply(a, b);
         }
         return a;
     }
diff --git a/accord-core/src/main/java/accord/utils/Invariants.java b/accord-core/src/main/java/accord/utils/Invariants.java
new file mode 100644
index 0000000..0dea428
--- /dev/null
... 2868 lines suppressed ...


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