You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2023/01/06 22:06:33 UTC

[GitHub] [cassandra-accord] belliottsmith opened a new pull request, #21: Introduce RangeDeps

belliottsmith opened a new pull request, #21:
URL: https://github.com/apache/cassandra-accord/pull/21

   Refactor `Deps` into `KeyDeps` and `RangeDeps`, with `Deps` containing one of each.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] iamaleksey commented on a diff in pull request #21: Introduce RangeDeps

Posted by "iamaleksey (via GitHub)" <gi...@apache.org>.
iamaleksey commented on code in PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21#discussion_r1085674238


##########
accord-core/src/main/java/accord/primitives/RangeDeps.java:
##########
@@ -0,0 +1,672 @@
+package accord.primitives;
+
+import accord.api.Key;
+import accord.utils.*;
+import accord.utils.RelationMultiMap.AbstractBuilder;
+import accord.utils.RelationMultiMap.Adapter;
+import net.nicoulaj.compilecommand.annotations.DontInline;
+import net.nicoulaj.compilecommand.annotations.Inline;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.RelationMultiMap.remove;
+import static accord.utils.SearchableRangeListBuilder.Links.LINKS;
+import static accord.utils.SearchableRangeListBuilder.Strategy.ACCURATE;
+import static accord.utils.SortedArrays.Search.CEIL;
+
+/**
+ * <p>Maintains a lazily-constructed, bidirectional map between Range and TxnId.
+ * <p>Ranges are stored sorted by start then end, and indexed by a secondary {@link SearchableRangeList} structure.
+ * <p>The relationship between Range and TxnId is maintained via {@code int[]} utilising {@link RelationMultiMap}
+ * functionality.
+ */
+public class RangeDeps implements Iterable<Map.Entry<Range, TxnId>>
+{
+    public static class SerializerSupport
+    {
+        private SerializerSupport() {}
+
+        public static int rangesToTxnIdsCount(RangeDeps deps)
+        {
+            return deps.rangesToTxnIds.length;
+        }
+
+        public static int rangesToTxnIds(RangeDeps deps, int idx)
+        {
+            return deps.rangesToTxnIds[idx];
+        }
+
+        public static RangeDeps create(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+        {
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    private static final Range[] NO_RANGES = new Range[0];
+    public static final RangeDeps NONE = new RangeDeps(new Range[0], new TxnId[0], new int[0], new int[0]);
+
+    final TxnId[] txnIds;
+    // the list of ranges and their mappings to txnIds
+    // unique, and sorted by start()
+    final Range[] ranges;
+    /**
+     * See {@link RelationMultiMap}.
+     * TODO consider alternative layout depending on real-world data distributions:
+     *      if most ranges have at most TxnId (or vice-versa) might be better to use negative values
+     *      to index into the dynamic portion of the array. We started with this, but decided it was
+     *      hard to justify the extra work for two layouts for the moment.
+     */
+    final int[] rangesToTxnIds;
+    int[] txnIdsToRanges;
+
+    private SearchableRangeList searchable;
+
+    public static <T1, T2> RangeDeps merge(List<T1> merge, Function<T1, T2> getter1, Function<T2, RangeDeps> getter2)
+    {
+        try (LinearMerger<Range, TxnId, RangeDeps> linearMerger = new LinearMerger<>(ADAPTER))
+        {
+            int mergeIndex = 0, mergeSize = merge.size();
+            while (mergeIndex < mergeSize)
+            {
+                T2 intermediate = getter1.apply(merge.get(mergeIndex++));
+                if (intermediate == null)
+                    continue;
+
+                RangeDeps deps = getter2.apply(intermediate);
+                if (deps == null || deps.isEmpty())
+                    continue;
+
+                linearMerger.update(deps, deps.ranges, deps.txnIds, deps.rangesToTxnIds);
+            }
+
+            return linearMerger.get(RangeDeps::new, NONE);
+        }
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+    {
+        this(ranges, txnIds, rangesToTxnIds, null);
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds, int[] txnIdsToRanges)
+    {
+        Invariants.checkArgument(rangesToTxnIds.length >= ranges.length);
+        Invariants.checkArgument(ranges.length > 0 || rangesToTxnIds.length == 0);
+        Invariants.paranoid(SortedArrays.isSorted(ranges, Range::compare));
+        this.ranges = ranges;
+        this.txnIds = txnIds;
+        this.rangesToTxnIds = rangesToTxnIds;
+        this.txnIdsToRanges = txnIdsToRanges;
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Key key, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(key, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Key key, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(key, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Range range, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(range, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private <P1, P2, P3> void forEach(Ranges ranges, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Range range, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(range, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int rangeIndex)
+    {
+        for (int i = startOffset(ranges, rangesToTxnIds, rangeIndex), end = endOffset(rangesToTxnIds, rangeIndex) ; i < end ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int start, int end)
+    {
+        if (end == 0)
+            return;
+        for (int i = startOffset(ranges, rangesToTxnIds, start) ; i < endOffset(rangesToTxnIds, end - 1) ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    // TODO (low priority, efficiency): ideally we would accept something like a BitHashSet or IntegerTrie
+    //   as O(N) space needed for BitSet here (but with a very low constant multiplier)
+    private void visitTxnIdx(int txnIdx, Consumer<TxnId> forEach, @Nullable BitSet visited)
+    {
+        if (visited == null || !visited.get(txnIdx))
+        {
+            if (visited != null)
+                visited.set(txnIdx);
+            forEach.accept(txnIds[txnIdx]);
+        }
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Key key, Consumer<TxnId> forEach)
+    {
+        forEach(key, forEach, 0, new BitSet());
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, null);
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, null);
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, new BitSet());
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     *
+     * @param ranges to match on
+     * @param forEach function to call on each unique {@link TxnId}
+     */
+    public void forEachUniqueTxnId(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, new BitSet());
+    }
+
+    // return true iff we map any ranges to any txnId
+    // if the mapping is empty we return false, whether or not we have any ranges or txnId by themselves
+    public boolean isEmpty()
+    {
+        return RelationMultiMap.isEmpty(ranges, rangesToTxnIds);
+    }
+
+    public Unseekables<Range, ?> someUnseekables(TxnId txnId)
+    {
+        return toUnseekables(txnId, array -> {
+            if (array.length == 0)
+                throw new IllegalStateException("Cannot create a RouteFragment without any keys");
+            return new Ranges(array);
+        });
+    }
+
+    private <R> R toUnseekables(TxnId txnId, Function<Range[], R> constructor)
+    {
+        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
+        if (txnIdIndex < 0)
+            constructor.apply(NO_RANGES);
+
+        ensureTxnIdToRange();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToRanges[txnIdIndex - 1];
+        int end = txnIdsToRanges[txnIdIndex];
+        Range[] result = new Range[end - start];
+        if (start == end)
+            constructor.apply(NO_RANGES);
+
+        result[0] = ranges[txnIdsToRanges[start]].toUnseekable();
+        int resultCount = 1;
+        for (int i = start + 1 ; i < end ; ++i)
+        {
+            Range next = ranges[txnIdsToRanges[i]];
+            if (!next.equals(result[resultCount - 1]))
+                result[resultCount++] = next;
+        }
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+        return constructor.apply(result);
+    }
+
+    void ensureTxnIdToRange()
+    {
+        if (txnIdsToRanges != null)
+            return;
+
+        txnIdsToRanges = invert(rangesToTxnIds, rangesToTxnIds.length, ranges.length, txnIds.length);
+    }
+
+    public RangeDeps slice(Ranges select)
+    {
+        if (isEmpty())
+            return new RangeDeps(NO_RANGES, txnIds, NO_INTS);
+
+        try (RangeAndMapCollector collector = new RangeAndMapCollector(ensureSearchable().maxScanAndCheckpointMatches))
+        {
+            forEach(select, collector, collector, ranges, rangesToTxnIds, null);
+
+            if (collector.rangesCount == 0)
+                return new RangeDeps(NO_RANGES, NO_TXNIDS, NO_INTS);
+
+            if (collector.rangesCount == this.ranges.length)
+                return this;
+
+            Range[] ranges = collector.getRanges();
+            int[] rangesToTxnIds = collector.getRangesToTxnIds();
+            TxnId[] txnIds = trimUnusedValues(ranges, this.txnIds, rangesToTxnIds, TxnId[]::new);
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    public RangeDeps with(RangeDeps that)
+    {
+        if (isEmpty() || that.isEmpty())
+            return isEmpty() ? that : this;
+
+        return linearUnion(
+                this.ranges, this.ranges.length, this.txnIds, this.txnIds.length, this.rangesToTxnIds, this.rangesToTxnIds.length,
+                that.ranges, that.ranges.length, that.txnIds, that.txnIds.length, that.rangesToTxnIds, that.rangesToTxnIds.length,
+                rangeComparator(), TxnId::compareTo,
+                cachedRanges(), cachedTxnIds(), cachedInts(),
+                (ranges, rangesLength, txnIds, txnIdsLength, out, outLength) ->
+                        new RangeDeps(cachedRanges().complete(ranges, rangesLength),
+                                cachedTxnIds().complete(txnIds, txnIdsLength),
+                                cachedInts().complete(out, outLength))
+        );
+    }
+
+    public RangeDeps without(Predicate<TxnId> remove)
+    {
+        return remove(this, ranges, txnIds, rangesToTxnIds, remove,
+                NONE, TxnId[]::new, ranges, RangeDeps::new);
+    }
+
+    public boolean contains(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId) >= 0;
+    }
+
+    public boolean isCoveredBy(Ranges covering)
+    {
+        // check that every entry intersects with some entry in covering
+        int prev = 0;
+        for (Range range : covering)
+        {
+            int start = SortedArrays.binarySearch(ranges, 0, ranges.length, range.start(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (start < 0) start = -1 - start;
+            int end = SortedArrays.binarySearch(ranges, 0, ranges.length, range.end(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (end < 0) end = -1 - end;
+            for (int i = prev; i < start ; ++i)
+            {
+                if (range.compareIntersecting(ranges[i]) != 0)
+                    return false;
+            }
+            prev = end;
+        }
+        return prev == ranges.length;
+    }
+
+    public List<TxnId> txnIds(Key key)
+    {
+        List<TxnId> result = new ArrayList<>();
+        forEachUniqueTxnId(key, result::add);
+        result.sort(TxnId::compareTo);
+        return result;
+    }
+
+    public List<TxnId> txnIds(Range key)

Review Comment:
   Rename the argument to `range`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] iamaleksey commented on a diff in pull request #21: Introduce RangeDeps

Posted by "iamaleksey (via GitHub)" <gi...@apache.org>.
iamaleksey commented on code in PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21#discussion_r1084220086


##########
accord-core/src/main/java/accord/primitives/KeyDeps.java:
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.Key;
+import accord.api.RoutingKey;
+import accord.utils.ArrayBuffers;
+import accord.utils.SymmetricComparator;
+
+import java.util.*;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.SortedArrays.Search.FAST;
+
+/**
+ * A collection of dependencies for a transaction, organised by the key the dependency is adopted via.
+ * An inverse map from TxnId to Key may also be constructed and stored in this collection.
+ */
+// TODO (desired, consider): switch to RoutingKey? Would mean adopting execution dependencies less precisely, but saving ser/deser of large keys
+public class KeyDeps implements Iterable<Map.Entry<Key, TxnId>>
+{
+    public static final KeyDeps NONE = new KeyDeps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+
+    public static class SerializerSupport
+    {
+        private SerializerSupport() {}
+
+        public static int keysToTxnIdsCount(KeyDeps deps)
+        {
+            return deps.keysToTxnIds.length;
+        }
+
+        public static int keysToTxnIds(KeyDeps deps, int idx)
+        {
+            return deps.keysToTxnIds[idx];
+        }
+
+        public static KeyDeps create(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
+        {
+            return new KeyDeps(keys, txnIds, keyToTxnId);
+        }
+    }
+
+    public static KeyDeps none(Keys keys)
+    {
+        int[] keysToTxnId = new int[keys.size()];
+        Arrays.fill(keysToTxnId, keys.size());
+        return new KeyDeps(keys, NO_TXNIDS, keysToTxnId);
+    }
+
+    /**
+     * Expects Command to be provided in TxnId order
+     */
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static class Builder extends AbstractBuilder<Key, TxnId, KeyDeps>
+    {
+        public Builder()
+        {
+            super(ADAPTER);
+        }
+
+        @Override
+        protected KeyDeps none()
+        {
+            return KeyDeps.NONE;
+        }
+
+        @Override
+        protected KeyDeps build(Key[] keys, TxnId[] txnIds, int[] keysToTxnIds)
+        {
+            return new KeyDeps(Keys.ofSorted(keys), txnIds, keysToTxnIds);
+        }
+    }
+
+    public static <T1, T2> KeyDeps merge(List<T1> merge, Function<T1, T2> getter1, Function<T2, KeyDeps> getter2)
+    {
+        try (LinearMerger<Key, TxnId, KeyDeps> linearMerger = new LinearMerger<>(ADAPTER))
+        {
+            int mergeIndex = 0, mergeSize = merge.size();
+            while (mergeIndex < mergeSize)
+            {
+                T2 intermediate = getter1.apply(merge.get(mergeIndex++));
+                if (intermediate == null)
+                    continue;
+
+                KeyDeps deps = getter2.apply(intermediate);
+                if (deps == null || deps.isEmpty())
+                    continue;
+
+                linearMerger.update(deps, deps.keys.keys, deps.txnIds, deps.keysToTxnIds);
+            }
+
+            return linearMerger.get(KeyDeps::new, NONE);
+        }
+    }
+
+    final Keys keys; // unique Keys
+    final TxnId[] txnIds; // unique TxnId
+
+    /**
+     * This represents a map of {@code Key -> [TxnId] } where each TxnId is actually a pointer into the txnIds array.
+     * The beginning of the array (the first keys.size() entries) are offsets into this array.
+     * <p/>
+     * Example:
+     * <p/>
+     * {@code
+     *   int keyIdx = keys.indexOf(key);
+     *   int startOfTxnOffset = keyIdx == 0 ? keys.size() : keyToTxnId[keyIdx - 1];
+     *   int endOfTxnOffset = keyToTxnId[keyIdx];
+     *   for (int i = startOfTxnOffset; i < endOfTxnOffset; i++)
+     *   {
+     *       TxnId id = txnIds[keyToTxnId[i]]
+     *       ...
+     *   }
+     * }
+     */
+    final int[] keysToTxnIds; // Key -> [TxnId]
+    // Lazy loaded in ensureTxnIdToKey()
+    int[] txnIdsToKeys; // TxnId -> [Key] TODO (low priority, efficiency): this could be a BTree?
+
+    KeyDeps(Key[] keys, TxnId[] txnIds, int[] keysToTxnIds)
+    {
+        this(Keys.ofSorted(keys), txnIds, keysToTxnIds);
+    }
+
+    KeyDeps(Keys keys, TxnId[] txnIds, int[] keysToTxnIds)
+    {
+        this.keys = keys;
+        this.txnIds = txnIds;
+        this.keysToTxnIds = keysToTxnIds;
+        if (!(keys.isEmpty() || keysToTxnIds[keys.size() - 1] == keysToTxnIds.length))
+            throw new IllegalArgumentException(String.format("Last key (%s) in keyToTxnId does not point (%d) to the end of the array (%d);\nkeyToTxnId=%s", keys.get(keys.size() - 1), keysToTxnIds[keys.size() - 1], keysToTxnIds.length, Arrays.toString(keysToTxnIds)));
+        checkValid(keys.keys, txnIds, keysToTxnIds);
+    }
+
+    public KeyDeps slice(Ranges ranges)
+    {
+        if (isEmpty())
+            return new KeyDeps(keys, txnIds, keysToTxnIds);
+
+        // TODO (low priority, efficiency): can slice in parallel with selecting keyToTxnId contents to avoid duplicate merging
+        Keys select = keys.slice(ranges);
+
+        if (select.isEmpty())
+            return new KeyDeps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+
+        if (select.size() == keys.size())
+            return new KeyDeps(keys, txnIds, keysToTxnIds);
+
+        int i = 0;
+        int offset = select.size();
+        for (int j = 0 ; j < select.size() ; ++j)
+        {
+            int findi = keys.findNext(i, select.get(j), FAST);
+            if (findi < 0)
+                continue;
+
+            i = findi;
+            offset += keysToTxnIds[i] - (i == 0 ? keys.size() : keysToTxnIds[i - 1]);
+        }
+
+        int[] src = keysToTxnIds;
+        int[] trg = new int[offset];
+
+        i = 0;
+        offset = select.size();
+        for (int j = 0 ; j < select.size() ; ++j)
+        {
+            int findi = keys.findNext(i, select.get(j), FAST);
+            if (findi >= 0)
+            {
+                i = findi;
+                int start = i == 0 ? keys.size() : src[i - 1];
+                int count = src[i] - start;
+                System.arraycopy(src, start, trg, offset, count);
+                offset += count;
+            }
+            trg[j] = offset;
+        }
+
+        TxnId[] txnIds = trimUnusedValues(select.keys, this.txnIds, trg, TxnId[]::new);
+        return new KeyDeps(select, txnIds, trg);
+    }
+
+    public KeyDeps with(KeyDeps that)
+    {
+        if (isEmpty() || that.isEmpty())
+            return isEmpty() ? that : this;
+
+        return linearUnion(
+                this.keys.keys, this.keys.keys.length, this.txnIds, this.txnIds.length, this.keysToTxnIds, this.keysToTxnIds.length,
+                that.keys.keys, that.keys.keys.length, that.txnIds, that.txnIds.length, that.keysToTxnIds, that.keysToTxnIds.length,
+                Key::compareTo, TxnId::compareTo,
+                cachedKeys(), cachedTxnIds(), cachedInts(),
+                (keys, keysLength, txnIds, txnIdsLength, out, outLength) ->
+                        new KeyDeps(Keys.ofSortedUnchecked(cachedKeys().complete(keys, keysLength)),
+                                cachedTxnIds().complete(txnIds, txnIdsLength),
+                                cachedInts().complete(out, outLength))
+                );
+    }
+
+    public KeyDeps without(Predicate<TxnId> remove)
+    {
+        return remove(this, keys.keys, txnIds, keysToTxnIds, remove,
+                NONE, TxnId[]::new, keys, KeyDeps::new);
+    }
+
+    public boolean contains(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId) >= 0;
+    }
+
+    // return true iff we map any keys to any txnId
+    // if the mapping is empty we return false, whether or not we have any keys or txnId by themselves
+    public boolean isEmpty()
+    {
+        return keysToTxnIds.length == keys.size();
+    }
+
+    public Keys someKeys(TxnId txnId)
+    {
+        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
+        if (txnIdIndex < 0)
+            return Keys.EMPTY;
+
+        ensureTxnIdToKey();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToKeys[txnIdIndex - 1];
+        int end = txnIdsToKeys[txnIdIndex];
+        if (start == end)
+            return Keys.EMPTY;
+
+        Key[] result = new Key[end - start];
+        for (int i = start ; i < end ; ++i)
+            result[i - start] = keys.get(txnIdsToKeys[i]);
+        return Keys.of(result);
+    }
+
+    public Unseekables<RoutingKey, ?> someUnseekables(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)
+            constructor.apply(RoutingKeys.EMPTY.keys);
+
+        ensureTxnIdToKey();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToKeys[txnIdIndex - 1];
+        int end = txnIdsToKeys[txnIdIndex];
+        RoutingKey[] result = new RoutingKey[end - start];
+        if (start == end)
+            constructor.apply(RoutingKeys.EMPTY.keys);
+
+        result[0] = keys.get(txnIdsToKeys[start]).toUnseekable();
+        int resultCount = 1;
+        for (int i = start + 1 ; i < end ; ++i)
+        {
+            RoutingKey next = keys.get(txnIdsToKeys[i]).toUnseekable();
+            if (!next.equals(result[resultCount - 1]))
+                result[resultCount++] = next;
+        }
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+        return constructor.apply(result);
+    }
+
+    void ensureTxnIdToKey()
+    {
+        if (txnIdsToKeys != null)
+            return;
+
+        txnIdsToKeys = invert(keysToTxnIds, keysToTxnIds.length, keys.size(), txnIds.length);
+    }
+
+    public void forEach(Ranges ranges, BiConsumer<Key, TxnId> forEach)
+    {
+        Routables.foldl(keys, ranges, (key, value, index) -> {
+            for (int t = startOffset(index), end = endOffset(index); t < end ; ++t)
+            {
+                TxnId txnId = txnIds[keysToTxnIds[t]];
+                forEach.accept(key, txnId);
+            }
+            return null;
+        }, null);
+    }
+
+    /**
+     * 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 forEach function to call on each unique {@link TxnId}
+     */
+    public void forEachUniqueTxnId(Ranges ranges, 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
+        // the property that forEach is called in TxnId order.
+        //TODO (expected, efficiency): reconsider this, probably not worth trying to save allocations at cost of multiple loop
+        //                             use BitSet, or perhaps extend so we can have no nested allocations when few bits
+        for (int offset = 0 ; offset < txnIds.length ; offset += 64)
+        {
+            long bitset = Routables.foldl(keys, ranges, (key, off, value, keyIndex) -> {
+                int index = startOffset(keyIndex);
+                int end = endOffset(keyIndex);
+                if (off > 0)
+                {
+                    // TODO (low priority, efficiency): interpolation search probably great here
+                    index = Arrays.binarySearch(keysToTxnIds, index, end, (int)off);
+                    if (index < 0)
+                        index = -1 - index;
+                }
+
+                while (index < end)
+                {
+                    long next = keysToTxnIds[index++] - off;
+                    if (next >= 64)
+                        break;
+                    value |= 1L << next;
+                }
+
+                return value;
+            }, offset, 0, -1L);
+
+            while (bitset != 0)
+            {
+                int i = Long.numberOfTrailingZeros(bitset);
+                TxnId txnId = txnIds[offset + i];
+                forEach.accept(txnId);
+                bitset ^= Long.lowestOneBit(bitset);
+            }
+        }
+    }
+
+    public void forEach(Key key, Consumer<TxnId> forEach)
+    {
+        int keyIndex = keys.indexOf(key);
+        if (keyIndex < 0)
+            return;
+
+        int index = startOffset(keyIndex);
+        int end = endOffset(keyIndex);
+        while (index < end)
+            forEach.accept(txnIds[keysToTxnIds[index++]]);
+    }
+
+    public Keys keys()
+    {
+        return keys;
+    }
+
+    public int txnIdCount()
+    {
+        return txnIds.length;
+    }
+
+    public int totalCount()
+    {
+        return keysToTxnIds.length - keys.size();
+    }
+
+    public TxnId txnId(int i)
+    {
+        return txnIds[i];
+    }
+
+    public Collection<TxnId> txnIds()
+    {
+        return Arrays.asList(txnIds);
+    }
+
+    public List<TxnId> txnIds(Key key)
+    {
+        int keyIndex = keys.indexOf(key);
+        if (keyIndex < 0)
+            return Collections.emptyList();
+
+        int start = startOffset(keyIndex);
+        int end = endOffset(keyIndex);
+        int size = end - start;
+        return txnIds(keysToTxnIds, start, size);
+    }
+
+    public List<TxnId> txnIds(Range range)
+    {
+        int startIndex = keys.indexOf(range.start());
+        if (startIndex < 0) startIndex = -1 - startIndex;
+        else if (!range.startInclusive()) ++startIndex;
+        int endIndex = keys.indexOf(range.end());
+        if (endIndex < 0) endIndex = -1 - endIndex;
+        else if (range.endInclusive()) ++endIndex;
+
+        if (startIndex == endIndex)
+            return Collections.emptyList();
+
+        int maxLength = Math.min(txnIds.length, startOffset(endIndex) - startOffset(startIndex));
+        int[] scratch = cachedInts().getInts(maxLength);
+        int count = 0;
+        for (int i = startIndex ; i < endIndex ; ++i)
+        {
+            int ri = startOffset(i), re = endOffset(i);
+            if (ri == re) continue;
+            if (count == 0)
+            {
+                count = re - ri;
+                System.arraycopy(keysToTxnIds, ri, scratch, 0, count);
+            }
+            else
+            {
+                if (count == maxLength)
+                    break;
+
+                System.arraycopy(scratch, 0, scratch, maxLength - count, count);
+                int li = maxLength - count, le = maxLength;
+                count = 0;
+                while (li < le && ri < re)
+                {
+                    int c = keysToTxnIds[ri] - scratch[li];
+                    if (c <= 0)
+                    {
+                        scratch[count++] = scratch[li++];

Review Comment:
   These branches might be backwards?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] iamaleksey commented on a diff in pull request #21: Introduce RangeDeps

Posted by "iamaleksey (via GitHub)" <gi...@apache.org>.
iamaleksey commented on code in PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21#discussion_r1085670711


##########
accord-core/src/main/java/accord/primitives/RangeDeps.java:
##########
@@ -0,0 +1,672 @@
+package accord.primitives;
+
+import accord.api.Key;
+import accord.utils.*;
+import accord.utils.RelationMultiMap.AbstractBuilder;
+import accord.utils.RelationMultiMap.Adapter;
+import net.nicoulaj.compilecommand.annotations.DontInline;
+import net.nicoulaj.compilecommand.annotations.Inline;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.RelationMultiMap.remove;
+import static accord.utils.SearchableRangeListBuilder.Links.LINKS;
+import static accord.utils.SearchableRangeListBuilder.Strategy.ACCURATE;
+import static accord.utils.SortedArrays.Search.CEIL;
+
+/**
+ * <p>Maintains a lazily-constructed, bidirectional map between Range and TxnId.
+ * <p>Ranges are stored sorted by start then end, and indexed by a secondary {@link SearchableRangeList} structure.
+ * <p>The relationship between Range and TxnId is maintained via {@code int[]} utilising {@link RelationMultiMap}
+ * functionality.
+ */
+public class RangeDeps implements Iterable<Map.Entry<Range, TxnId>>
+{
+    public static class SerializerSupport
+    {
+        private SerializerSupport() {}
+
+        public static int rangesToTxnIdsCount(RangeDeps deps)
+        {
+            return deps.rangesToTxnIds.length;
+        }
+
+        public static int rangesToTxnIds(RangeDeps deps, int idx)
+        {
+            return deps.rangesToTxnIds[idx];
+        }
+
+        public static RangeDeps create(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+        {
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    private static final Range[] NO_RANGES = new Range[0];
+    public static final RangeDeps NONE = new RangeDeps(new Range[0], new TxnId[0], new int[0], new int[0]);
+
+    final TxnId[] txnIds;
+    // the list of ranges and their mappings to txnIds
+    // unique, and sorted by start()
+    final Range[] ranges;
+    /**
+     * See {@link RelationMultiMap}.
+     * TODO consider alternative layout depending on real-world data distributions:
+     *      if most ranges have at most TxnId (or vice-versa) might be better to use negative values
+     *      to index into the dynamic portion of the array. We started with this, but decided it was
+     *      hard to justify the extra work for two layouts for the moment.
+     */
+    final int[] rangesToTxnIds;
+    int[] txnIdsToRanges;
+
+    private SearchableRangeList searchable;
+
+    public static <T1, T2> RangeDeps merge(List<T1> merge, Function<T1, T2> getter1, Function<T2, RangeDeps> getter2)
+    {
+        try (LinearMerger<Range, TxnId, RangeDeps> linearMerger = new LinearMerger<>(ADAPTER))
+        {
+            int mergeIndex = 0, mergeSize = merge.size();
+            while (mergeIndex < mergeSize)
+            {
+                T2 intermediate = getter1.apply(merge.get(mergeIndex++));
+                if (intermediate == null)
+                    continue;
+
+                RangeDeps deps = getter2.apply(intermediate);
+                if (deps == null || deps.isEmpty())
+                    continue;
+
+                linearMerger.update(deps, deps.ranges, deps.txnIds, deps.rangesToTxnIds);
+            }
+
+            return linearMerger.get(RangeDeps::new, NONE);
+        }
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+    {
+        this(ranges, txnIds, rangesToTxnIds, null);
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds, int[] txnIdsToRanges)
+    {
+        Invariants.checkArgument(rangesToTxnIds.length >= ranges.length);
+        Invariants.checkArgument(ranges.length > 0 || rangesToTxnIds.length == 0);
+        Invariants.paranoid(SortedArrays.isSorted(ranges, Range::compare));
+        this.ranges = ranges;
+        this.txnIds = txnIds;
+        this.rangesToTxnIds = rangesToTxnIds;
+        this.txnIdsToRanges = txnIdsToRanges;
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Key key, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(key, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Key key, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(key, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Range range, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(range, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private <P1, P2, P3> void forEach(Ranges ranges, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Range range, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(range, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int rangeIndex)
+    {
+        for (int i = startOffset(ranges, rangesToTxnIds, rangeIndex), end = endOffset(rangesToTxnIds, rangeIndex) ; i < end ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int start, int end)
+    {
+        if (end == 0)
+            return;
+        for (int i = startOffset(ranges, rangesToTxnIds, start) ; i < endOffset(rangesToTxnIds, end - 1) ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    // TODO (low priority, efficiency): ideally we would accept something like a BitHashSet or IntegerTrie
+    //   as O(N) space needed for BitSet here (but with a very low constant multiplier)
+    private void visitTxnIdx(int txnIdx, Consumer<TxnId> forEach, @Nullable BitSet visited)
+    {
+        if (visited == null || !visited.get(txnIdx))
+        {
+            if (visited != null)
+                visited.set(txnIdx);
+            forEach.accept(txnIds[txnIdx]);
+        }
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Key key, Consumer<TxnId> forEach)
+    {
+        forEach(key, forEach, 0, new BitSet());
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, null);
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, null);
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, new BitSet());
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     *
+     * @param ranges to match on
+     * @param forEach function to call on each unique {@link TxnId}
+     */
+    public void forEachUniqueTxnId(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, new BitSet());
+    }
+
+    // return true iff we map any ranges to any txnId
+    // if the mapping is empty we return false, whether or not we have any ranges or txnId by themselves
+    public boolean isEmpty()
+    {
+        return RelationMultiMap.isEmpty(ranges, rangesToTxnIds);
+    }
+
+    public Unseekables<Range, ?> someUnseekables(TxnId txnId)
+    {
+        return toUnseekables(txnId, array -> {
+            if (array.length == 0)
+                throw new IllegalStateException("Cannot create a RouteFragment without any keys");
+            return new Ranges(array);
+        });
+    }
+
+    private <R> R toUnseekables(TxnId txnId, Function<Range[], R> constructor)
+    {
+        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
+        if (txnIdIndex < 0)
+            constructor.apply(NO_RANGES);
+
+        ensureTxnIdToRange();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToRanges[txnIdIndex - 1];
+        int end = txnIdsToRanges[txnIdIndex];
+        Range[] result = new Range[end - start];
+        if (start == end)
+            constructor.apply(NO_RANGES);
+
+        result[0] = ranges[txnIdsToRanges[start]].toUnseekable();
+        int resultCount = 1;
+        for (int i = start + 1 ; i < end ; ++i)
+        {
+            Range next = ranges[txnIdsToRanges[i]];
+            if (!next.equals(result[resultCount - 1]))
+                result[resultCount++] = next;
+        }
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+        return constructor.apply(result);
+    }
+
+    void ensureTxnIdToRange()
+    {
+        if (txnIdsToRanges != null)
+            return;
+
+        txnIdsToRanges = invert(rangesToTxnIds, rangesToTxnIds.length, ranges.length, txnIds.length);
+    }
+
+    public RangeDeps slice(Ranges select)
+    {
+        if (isEmpty())
+            return new RangeDeps(NO_RANGES, txnIds, NO_INTS);
+
+        try (RangeAndMapCollector collector = new RangeAndMapCollector(ensureSearchable().maxScanAndCheckpointMatches))
+        {
+            forEach(select, collector, collector, ranges, rangesToTxnIds, null);
+
+            if (collector.rangesCount == 0)
+                return new RangeDeps(NO_RANGES, NO_TXNIDS, NO_INTS);
+
+            if (collector.rangesCount == this.ranges.length)
+                return this;
+
+            Range[] ranges = collector.getRanges();
+            int[] rangesToTxnIds = collector.getRangesToTxnIds();
+            TxnId[] txnIds = trimUnusedValues(ranges, this.txnIds, rangesToTxnIds, TxnId[]::new);
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    public RangeDeps with(RangeDeps that)
+    {
+        if (isEmpty() || that.isEmpty())
+            return isEmpty() ? that : this;
+
+        return linearUnion(
+                this.ranges, this.ranges.length, this.txnIds, this.txnIds.length, this.rangesToTxnIds, this.rangesToTxnIds.length,
+                that.ranges, that.ranges.length, that.txnIds, that.txnIds.length, that.rangesToTxnIds, that.rangesToTxnIds.length,
+                rangeComparator(), TxnId::compareTo,
+                cachedRanges(), cachedTxnIds(), cachedInts(),
+                (ranges, rangesLength, txnIds, txnIdsLength, out, outLength) ->
+                        new RangeDeps(cachedRanges().complete(ranges, rangesLength),
+                                cachedTxnIds().complete(txnIds, txnIdsLength),
+                                cachedInts().complete(out, outLength))
+        );
+    }
+
+    public RangeDeps without(Predicate<TxnId> remove)
+    {
+        return remove(this, ranges, txnIds, rangesToTxnIds, remove,
+                NONE, TxnId[]::new, ranges, RangeDeps::new);
+    }
+
+    public boolean contains(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId) >= 0;
+    }
+
+    public boolean isCoveredBy(Ranges covering)
+    {
+        // check that every entry intersects with some entry in covering
+        int prev = 0;
+        for (Range range : covering)
+        {
+            int start = SortedArrays.binarySearch(ranges, 0, ranges.length, range.start(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (start < 0) start = -1 - start;
+            int end = SortedArrays.binarySearch(ranges, 0, ranges.length, range.end(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (end < 0) end = -1 - end;
+            for (int i = prev; i < start ; ++i)
+            {
+                if (range.compareIntersecting(ranges[i]) != 0)
+                    return false;
+            }
+            prev = end;
+        }
+        return prev == ranges.length;
+    }
+
+    public List<TxnId> txnIds(Key key)
+    {
+        List<TxnId> result = new ArrayList<>();
+        forEachUniqueTxnId(key, result::add);
+        result.sort(TxnId::compareTo);
+        return result;
+    }
+
+    public List<TxnId> txnIds(Range key)
+    {
+        List<TxnId> result = new ArrayList<>();
+        forEachUniqueTxnId(key, result::add);
+        result.sort(TxnId::compareTo);
+        return result;
+    }
+
+    public TxnId txnId(int i)
+    {
+        return txnIds[i];
+    }
+
+    public int txnIdCount()
+    {
+        return txnIds.length;
+    }
+
+    public Range range(int i)
+    {
+        return ranges[i];
+    }
+
+    public int rangeCount()
+    {
+        return ranges.length;
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return this == that || (that instanceof RangeDeps && equals((RangeDeps)that));
+    }
+
+    public boolean equals(RangeDeps that)
+    {
+        return testEquality(this.ranges, this.txnIds, this.rangesToTxnIds, that.ranges, that.txnIds, that.rangesToTxnIds);
+    }
+
+    @Override
+    public String toString()
+    {
+        return RelationMultiMap.toSimpleString(ranges, txnIds, rangesToTxnIds);
+    }
+
+    @Nonnull
+    @Override
+    public Iterator<Map.Entry<Range, TxnId>> iterator()
+    {
+        return newIterator(ranges, txnIds, rangesToTxnIds);
+    }
+
+    private SearchableRangeList ensureSearchable()
+    {
+        if (searchable == null)
+            buildSearchable();
+        return searchable;
+    }
+
+    @DontInline
+    private void buildSearchable()
+    {
+        searchable = SearchableRangeList.build(ranges);
+    }
+
+    public boolean isSearchable()

Review Comment:
   Also unused.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] iamaleksey commented on a diff in pull request #21: Introduce RangeDeps

Posted by "iamaleksey (via GitHub)" <gi...@apache.org>.
iamaleksey commented on code in PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21#discussion_r1081613800


##########
accord-maelstrom/src/main/java/accord/maelstrom/Json.java:
##########
@@ -376,7 +387,7 @@ public void write(JsonWriter out, Writes value) throws IOException
             out.name("executeAt");
             GSON.toJson(value.executeAt, Timestamp.class, out);
             out.name("keys");
-            Keys keys = value.keys;
+            Keys keys = (Keys)value.keys;

Review Comment:
   Redundant cast?



##########
accord-core/src/main/java/accord/primitives/AbstractKeys.java:
##########
@@ -35,7 +35,12 @@ public boolean equals(Object o)
         return Arrays.equals(keys, that.keys);
     }
 
-    public final int indexOf(K key)
+    public int indexOf(K key)

Review Comment:
   Missing `@Override` (also for `get()`).



##########
accord-core/src/main/java/accord/primitives/KeyDeps.java:
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.Key;
+import accord.api.RoutingKey;
+import accord.utils.ArrayBuffers;
+import accord.utils.SymmetricComparator;
+
+import java.util.*;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.SortedArrays.Search.FAST;
+
+/**
+ * A collection of dependencies for a transaction, organised by the key the dependency is adopted via.
+ * An inverse map from TxnId to Key may also be constructed and stored in this collection.
+ */
+// TODO (desired, consider): switch to RoutingKey? Would mean adopting execution dependencies less precisely, but saving ser/deser of large keys
+public class KeyDeps implements Iterable<Map.Entry<Key, TxnId>>
+{
+    public static final KeyDeps NONE = new KeyDeps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+
+    public static class SerializerSupport
+    {
+        private SerializerSupport() {}
+
+        public static int keysToTxnIdsCount(KeyDeps deps)
+        {
+            return deps.keysToTxnIds.length;
+        }
+
+        public static int keysToTxnIds(KeyDeps deps, int idx)
+        {
+            return deps.keysToTxnIds[idx];
+        }
+
+        public static KeyDeps create(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
+        {
+            return new KeyDeps(keys, txnIds, keyToTxnId);
+        }
+    }
+
+    public static KeyDeps none(Keys keys)
+    {
+        int[] keysToTxnId = new int[keys.size()];
+        Arrays.fill(keysToTxnId, keys.size());
+        return new KeyDeps(keys, NO_TXNIDS, keysToTxnId);
+    }
+
+    /**
+     * Expects Command to be provided in TxnId order
+     */
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static class Builder extends AbstractBuilder<Key, TxnId, KeyDeps>
+    {
+        public Builder()
+        {
+            super(ADAPTER);
+        }
+
+        @Override
+        protected KeyDeps none()
+        {
+            return KeyDeps.NONE;
+        }
+
+        @Override
+        protected KeyDeps build(Key[] keys, TxnId[] txnIds, int[] keysToTxnIds)
+        {
+            return new KeyDeps(Keys.ofSorted(keys), txnIds, keysToTxnIds);
+        }
+    }
+
+    public static <T1, T2> KeyDeps merge(List<T1> merge, Function<T1, T2> getter1, Function<T2, KeyDeps> getter2)
+    {
+        try (LinearMerger<Key, TxnId, KeyDeps> linearMerger = new LinearMerger<>(ADAPTER))
+        {
+            int mergeIndex = 0, mergeSize = merge.size();
+            while (mergeIndex < mergeSize)
+            {
+                T2 intermediate = getter1.apply(merge.get(mergeIndex++));
+                if (intermediate == null)
+                    continue;
+
+                KeyDeps deps = getter2.apply(intermediate);
+                if (deps == null || deps.isEmpty())
+                    continue;
+
+                linearMerger.update(deps, deps.keys.keys, deps.txnIds, deps.keysToTxnIds);
+            }
+
+            return linearMerger.get(KeyDeps::new, NONE);
+        }
+    }
+
+    final Keys keys; // unique Keys
+    final TxnId[] txnIds; // unique TxnId
+
+    /**
+     * This represents a map of {@code Key -> [TxnId] } where each TxnId is actually a pointer into the txnIds array.
+     * The beginning of the array (the first keys.size() entries) are offsets into this array.
+     * <p/>
+     * Example:
+     * <p/>
+     * {@code
+     *   int keyIdx = keys.indexOf(key);
+     *   int startOfTxnOffset = keyIdx == 0 ? keys.size() : keyToTxnId[keyIdx - 1];
+     *   int endOfTxnOffset = keyToTxnId[keyIdx];
+     *   for (int i = startOfTxnOffset; i < endOfTxnOffset; i++)
+     *   {
+     *       TxnId id = txnIds[keyToTxnId[i]]
+     *       ...
+     *   }
+     * }
+     */
+    final int[] keysToTxnIds; // Key -> [TxnId]
+    // Lazy loaded in ensureTxnIdToKey()
+    int[] txnIdsToKeys; // TxnId -> [Key] TODO (low priority, efficiency): this could be a BTree?
+
+    KeyDeps(Key[] keys, TxnId[] txnIds, int[] keysToTxnIds)
+    {
+        this(Keys.ofSorted(keys), txnIds, keysToTxnIds);
+    }
+
+    KeyDeps(Keys keys, TxnId[] txnIds, int[] keysToTxnIds)
+    {
+        this.keys = keys;
+        this.txnIds = txnIds;
+        this.keysToTxnIds = keysToTxnIds;
+        if (!(keys.isEmpty() || keysToTxnIds[keys.size() - 1] == keysToTxnIds.length))
+            throw new IllegalArgumentException(String.format("Last key (%s) in keyToTxnId does not point (%d) to the end of the array (%d);\nkeyToTxnId=%s", keys.get(keys.size() - 1), keysToTxnIds[keys.size() - 1], keysToTxnIds.length, Arrays.toString(keysToTxnIds)));
+        checkValid(keys.keys, txnIds, keysToTxnIds);
+    }
+
+    public KeyDeps slice(Ranges ranges)
+    {
+        if (isEmpty())
+            return new KeyDeps(keys, txnIds, keysToTxnIds);
+
+        // TODO (low priority, efficiency): can slice in parallel with selecting keyToTxnId contents to avoid duplicate merging
+        Keys select = keys.slice(ranges);
+
+        if (select.isEmpty())
+            return new KeyDeps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+
+        if (select.size() == keys.size())
+            return new KeyDeps(keys, txnIds, keysToTxnIds);
+
+        int i = 0;
+        int offset = select.size();
+        for (int j = 0 ; j < select.size() ; ++j)
+        {
+            int findi = keys.findNext(i, select.get(j), FAST);
+            if (findi < 0)
+                continue;
+
+            i = findi;
+            offset += keysToTxnIds[i] - (i == 0 ? keys.size() : keysToTxnIds[i - 1]);
+        }
+
+        int[] src = keysToTxnIds;
+        int[] trg = new int[offset];
+
+        i = 0;
+        offset = select.size();
+        for (int j = 0 ; j < select.size() ; ++j)
+        {
+            int findi = keys.findNext(i, select.get(j), FAST);
+            if (findi >= 0)
+            {
+                i = findi;
+                int start = i == 0 ? keys.size() : src[i - 1];
+                int count = src[i] - start;
+                System.arraycopy(src, start, trg, offset, count);
+                offset += count;
+            }
+            trg[j] = offset;
+        }
+
+        TxnId[] txnIds = trimUnusedValues(select.keys, this.txnIds, trg, TxnId[]::new);
+        return new KeyDeps(select, txnIds, trg);
+    }
+
+    public KeyDeps with(KeyDeps that)
+    {
+        if (isEmpty() || that.isEmpty())
+            return isEmpty() ? that : this;
+
+        return linearUnion(
+                this.keys.keys, this.keys.keys.length, this.txnIds, this.txnIds.length, this.keysToTxnIds, this.keysToTxnIds.length,
+                that.keys.keys, that.keys.keys.length, that.txnIds, that.txnIds.length, that.keysToTxnIds, that.keysToTxnIds.length,
+                Key::compareTo, TxnId::compareTo,
+                cachedKeys(), cachedTxnIds(), cachedInts(),
+                (keys, keysLength, txnIds, txnIdsLength, out, outLength) ->
+                        new KeyDeps(Keys.ofSortedUnchecked(cachedKeys().complete(keys, keysLength)),
+                                cachedTxnIds().complete(txnIds, txnIdsLength),
+                                cachedInts().complete(out, outLength))
+                );
+    }
+
+    public KeyDeps without(Predicate<TxnId> remove)
+    {
+        return remove(this, keys.keys, txnIds, keysToTxnIds, remove,
+                NONE, TxnId[]::new, keys, KeyDeps::new);
+    }
+
+    public boolean contains(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId) >= 0;
+    }
+
+    // return true iff we map any keys to any txnId
+    // if the mapping is empty we return false, whether or not we have any keys or txnId by themselves
+    public boolean isEmpty()
+    {
+        return keysToTxnIds.length == keys.size();
+    }
+
+    public Keys someKeys(TxnId txnId)
+    {
+        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
+        if (txnIdIndex < 0)
+            return Keys.EMPTY;
+
+        ensureTxnIdToKey();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToKeys[txnIdIndex - 1];
+        int end = txnIdsToKeys[txnIdIndex];
+        if (start == end)
+            return Keys.EMPTY;
+
+        Key[] result = new Key[end - start];
+        for (int i = start ; i < end ; ++i)
+            result[i - start] = keys.get(txnIdsToKeys[i]);
+        return Keys.of(result);
+    }
+
+    public Unseekables<RoutingKey, ?> someUnseekables(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)
+            constructor.apply(RoutingKeys.EMPTY.keys);
+
+        ensureTxnIdToKey();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToKeys[txnIdIndex - 1];
+        int end = txnIdsToKeys[txnIdIndex];
+        RoutingKey[] result = new RoutingKey[end - start];
+        if (start == end)
+            constructor.apply(RoutingKeys.EMPTY.keys);
+
+        result[0] = keys.get(txnIdsToKeys[start]).toUnseekable();
+        int resultCount = 1;
+        for (int i = start + 1 ; i < end ; ++i)
+        {
+            RoutingKey next = keys.get(txnIdsToKeys[i]).toUnseekable();
+            if (!next.equals(result[resultCount - 1]))
+                result[resultCount++] = next;
+        }
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+        return constructor.apply(result);
+    }
+
+    void ensureTxnIdToKey()
+    {
+        if (txnIdsToKeys != null)
+            return;
+
+        txnIdsToKeys = invert(keysToTxnIds, keysToTxnIds.length, keys.size(), txnIds.length);
+    }
+
+    public void forEach(Ranges ranges, BiConsumer<Key, TxnId> forEach)
+    {
+        Routables.foldl(keys, ranges, (key, value, index) -> {
+            for (int t = startOffset(index), end = endOffset(index); t < end ; ++t)
+            {
+                TxnId txnId = txnIds[keysToTxnIds[t]];
+                forEach.accept(key, txnId);
+            }
+            return null;
+        }, null);
+    }
+
+    /**
+     * 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 forEach function to call on each unique {@link TxnId}
+     */
+    public void forEachUniqueTxnId(Ranges ranges, 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
+        // the property that forEach is called in TxnId order.
+        //TODO (expected, efficiency): reconsider this, probably not worth trying to save allocations at cost of multiple loop
+        //                             use BitSet, or perhaps extend so we can have no nested allocations when few bits
+        for (int offset = 0 ; offset < txnIds.length ; offset += 64)
+        {
+            long bitset = Routables.foldl(keys, ranges, (key, off, value, keyIndex) -> {
+                int index = startOffset(keyIndex);
+                int end = endOffset(keyIndex);
+                if (off > 0)
+                {
+                    // TODO (low priority, efficiency): interpolation search probably great here
+                    index = Arrays.binarySearch(keysToTxnIds, index, end, (int)off);
+                    if (index < 0)
+                        index = -1 - index;
+                }
+
+                while (index < end)
+                {
+                    long next = keysToTxnIds[index++] - off;
+                    if (next >= 64)
+                        break;
+                    value |= 1L << next;
+                }
+
+                return value;
+            }, offset, 0, -1L);
+
+            while (bitset != 0)
+            {
+                int i = Long.numberOfTrailingZeros(bitset);
+                TxnId txnId = txnIds[offset + i];
+                forEach.accept(txnId);
+                bitset ^= Long.lowestOneBit(bitset);
+            }
+        }
+    }
+
+    public void forEach(Key key, Consumer<TxnId> forEach)
+    {
+        int keyIndex = keys.indexOf(key);
+        if (keyIndex < 0)
+            return;
+
+        int index = startOffset(keyIndex);
+        int end = endOffset(keyIndex);
+        while (index < end)
+            forEach.accept(txnIds[keysToTxnIds[index++]]);
+    }
+
+    public Keys keys()
+    {
+        return keys;
+    }
+
+    public int txnIdCount()
+    {
+        return txnIds.length;
+    }
+
+    public int totalCount()
+    {
+        return keysToTxnIds.length - keys.size();
+    }
+
+    public TxnId txnId(int i)
+    {
+        return txnIds[i];
+    }
+
+    public Collection<TxnId> txnIds()
+    {
+        return Arrays.asList(txnIds);
+    }
+
+    public List<TxnId> txnIds(Key key)
+    {
+        int keyIndex = keys.indexOf(key);
+        if (keyIndex < 0)
+            return Collections.emptyList();
+
+        int start = startOffset(keyIndex);
+        int end = endOffset(keyIndex);
+        int size = end - start;
+        return txnIds(keysToTxnIds, start, size);
+    }
+
+    public List<TxnId> txnIds(Range range)
+    {
+        int startIndex = keys.indexOf(range.start());
+        if (startIndex < 0) startIndex = -1 - startIndex;
+        else if (!range.startInclusive()) ++startIndex;
+        int endIndex = keys.indexOf(range.end());
+        if (endIndex < 0) endIndex = -1 - endIndex;
+        else if (range.endInclusive()) ++endIndex;
+
+        if (startIndex == endIndex)
+            return Collections.emptyList();
+
+        int maxLength = Math.min(txnIds.length, startOffset(endIndex) - startOffset(startIndex));
+        int[] scratch = cachedInts().getInts(maxLength);
+        int count = 0;
+        for (int i = startIndex ; i < endIndex ; ++i)
+        {
+            int ri = startOffset(i), re = endOffset(i);
+            if (ri == re) continue;
+            if (count == 0)
+            {
+                count = re - ri;
+                System.arraycopy(keysToTxnIds, ri, scratch, 0, count);
+            }
+            else
+            {
+                if (count == maxLength)
+                    break;
+
+                System.arraycopy(scratch, 0, scratch, maxLength - count, count);
+                int li = maxLength - count, le = maxLength;
+                count = 0;
+                while (li < le && ri < re)
+                {
+                    int c = keysToTxnIds[ri] - scratch[li];
+                    if (c <= 0)
+                    {
+                        scratch[count++] = scratch[li++];
+                        ri += c == 0 ? 1 : 0;
+                    }
+                    else
+                    {
+                        scratch[count++] = keysToTxnIds[ri++];
+                    }
+                }
+                while (li < le)
+                    scratch[count++] = scratch[li++];
+                while (ri < re)
+                    scratch[count++] = keysToTxnIds[ri++];
+            }
+        }
+
+        int[] ids = cachedInts().completeAndDiscard(scratch, count);
+        return txnIds(ids, 0, count);
+    }
+
+    private List<TxnId> txnIds(int[] ids, int start, int size)
+    {
+        return new AbstractList<TxnId>()
+        {
+            @Override
+            public TxnId get(int index)
+            {
+                if (index > size)
+                    throw new IndexOutOfBoundsException();
+                return txnIds[ids[start + index]];
+            }
+
+            @Override
+            public int size()
+            {
+                return size;
+            }
+        };
+    }
+
+    private int startOffset(int keyIndex)
+    {
+        return keyIndex == 0 ? keys.size() : keysToTxnIds[keyIndex - 1];
+    }
+
+    private int endOffset(int keyIndex)
+    {
+        return keysToTxnIds[keyIndex];
+    }
+
+    public boolean equals(Object that)
+    {
+        return this == that || (that instanceof KeyDeps && equals((KeyDeps)that));
+    }
+
+    public boolean equals(KeyDeps that)
+    {
+        return testEquality(this.keys.keys, this.txnIds, this.keysToTxnIds, that.keys.keys, that.txnIds, that.keysToTxnIds);
+    }
+
+    public Iterator<Map.Entry<Key, TxnId>> iterator()
+    {
+        return newIterator(keys.keys, txnIds, keysToTxnIds);
+    }
+
+    @Override
+    public String toString()
+    {
+        return toSimpleString(keys.keys, txnIds, keysToTxnIds);
+    }
+
+    static abstract class AbstractDepsAdapter<D extends KeyDeps> implements Adapter<Key, TxnId>

Review Comment:
   Only one implementation of it (`KeyDepsAdapter`). Another one planned? Otherwise might as well fold?



##########
accord-core/src/test/java/accord/primitives/RangeDepsTest.java:
##########
@@ -0,0 +1,230 @@
+package accord.primitives;
+
+import accord.impl.IntKey;
+import accord.local.Node;
+import accord.primitives.Routable.Domain;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.*;
+
+import static accord.primitives.Txn.Kind.Write;
+
+public class RangeDepsTest
+{
+    static class GenerateRanges
+    {
+        final int rangeDomain;
+        final float minRangesDomain, maxRangesDomain, minRangesSpan, maxRangesSpan;
+
+        GenerateRanges(int rangeDomain, float minRangesDomain, float maxRangesDomain, float minRangesSpan, float maxRangesSpan)
+        {
+            this.rangeDomain = rangeDomain;
+            this.minRangesDomain = minRangesDomain;
+            this.maxRangesDomain = maxRangesDomain;
+            this.minRangesSpan = minRangesSpan;
+            this.maxRangesSpan = maxRangesSpan;
+        }
+
+        private Range[] generateRanges(Random random, int rangeCount)
+        {
+            Range[] ranges = new Range[rangeCount];
+            int txnDomain = Math.max(ranges.length, (int) (((random.nextFloat() * (maxRangesDomain - minRangesDomain)) + minRangesDomain) * rangeDomain));
+            int txnSpan = Math.max(txnDomain, (int) (((random.nextFloat() * (maxRangesSpan - minRangesSpan)) + minRangesSpan) * rangeDomain));
+            int gapSpan = txnSpan - txnDomain;
+            int start = rangeDomain == txnSpan ? 0 : random.nextInt(rangeDomain - txnSpan);
+            int end = start + txnSpan;
+            int[] gaps = new int[ranges.length];
+            int[] gapSpans = new int[ranges.length];
+            for (int i = 0 ; i < ranges.length ; ++i)
+            {
+                gaps[i] = start + random.nextInt(end - start);
+                gapSpans[i] = i == ranges.length - 1 || gapSpan <= 1 ? gapSpan : 1 + random.nextInt(Math.max(1, 2 * gapSpan/(ranges.length - i)));
+            }
+            Arrays.sort(gaps);
+            for (int i = 0 ; i < ranges.length ; ++i)
+            {
+                end = Math.max(start + 1, gaps[i]);
+                ranges[i] = r(start, end);
+                start = end + gapSpans[i];
+            }
+            return ranges;
+        }
+    }
+
+    static class Validate
+    {
+        final GenerateRanges generate;
+        final Map<TxnId, Ranges> canonical;
+        final RangeDeps test;
+
+        Validate(GenerateRanges generate, Map<TxnId, Ranges> canonical, RangeDeps test)
+        {
+            this.generate = generate;
+            this.canonical = canonical;
+            this.test = test;
+        }
+
+        Set<TxnId> canonicalOverlaps(Range range)
+        {
+            Set<TxnId> set = new TreeSet<>();
+            for (Map.Entry<TxnId, Ranges> e : canonical.entrySet())
+            {
+                if (e.getValue().intersects(range))
+                    set.add(e.getKey());
+            }
+            return set;
+        }
+
+        Set<TxnId> testOverlaps(Range range)
+        {
+            Set<TxnId> set = new TreeSet<>();
+            test.forEachUniqueTxnId(range, set::add);
+            return set;
+        }
+
+        void validate(Random random)
+        {
+            Assertions.assertEquals(canonical.size(), test.txnIdCount());
+            Assertions.assertArrayEquals(canonical.keySet().stream().toArray(TxnId[]::new), test.txnIds);
+            for (int i = 0 ; i < test.rangeCount() ; ++i)
+            {
+                Assertions.assertEquals(canonicalOverlaps(test.range(i)), testOverlaps(test.range(i)));
+            }
+            for (int i = 0 ; i < test.rangeCount() ; ++i)
+            {
+                Range range = generate.generateRanges(random, 1)[0];
+                Assertions.assertEquals(canonicalOverlaps(range), testOverlaps(range));
+            }
+        }
+    }
+
+    private static final boolean EXHAUSTIVE = false;
+
+    // TODO (expected, testing): generate ranges of different sizes at different ratios
+    private static Validate generate(Random random, GenerateRanges generate, int txnIdCount, int rangeCount)
+    {
+        Map<TxnId, Ranges> map = new TreeMap<>();
+        for (int txnId = 0; txnId < txnIdCount ; ++txnId)
+        {
+            int thisRangeCount = txnIdCount == 1 ? rangeCount : 1 + random.nextInt((2 * rangeCount / txnIdCount) - 1);
+            Range[] ranges = generate.generateRanges(random, thisRangeCount);
+            map.put(id(txnId), Ranges.of(ranges));
+        }
+        return new Validate(generate, map, RangeDeps.of(map));
+    }
+
+    private static Validate generateIdenticalTxns(Random random, GenerateRanges generate, int txnIdCount, int rangeCount)
+    {
+        Map<TxnId, Ranges> map = new TreeMap<>();
+        Range[] ranges = generate.generateRanges(random, rangeCount / txnIdCount);
+        for (int txnId = 0; txnId < txnIdCount ; ++txnId)
+            map.put(id(txnId), Ranges.of(ranges));
+        return new Validate(generate, map, RangeDeps.of(map));
+    }
+
+    private static Validate generateNemesisRanges(int width, int nemesisTxnIdCount, int rangeCount, int nonNemesisEntryPerNemesisEntry)
+    {
+        Map<Integer, List<Range>> build = new TreeMap<>();
+        int nonNemesisTxnIdCount = nemesisTxnIdCount * nonNemesisEntryPerNemesisEntry;
+        rangeCount /= (1 + nonNemesisEntryPerNemesisEntry);
+        int rangeDomain = 0;
+        for (int i = 0 ; i < rangeCount ; ++i)
+        {
+            build.computeIfAbsent(i % nemesisTxnIdCount, ignore -> new ArrayList<>()).add(r(i, i + width));
+            for (int j = 0 ; j < nonNemesisEntryPerNemesisEntry ; ++j)
+                build.computeIfAbsent(nemesisTxnIdCount + (i % nonNemesisTxnIdCount), ignore -> new ArrayList<>()).add(r(i, i + 1));
+            rangeDomain = i + width;
+        }
+        Map<TxnId, Ranges> result = new TreeMap<>();
+        for (int txnId = 0; txnId < nemesisTxnIdCount + nonNemesisTxnIdCount ; ++txnId)
+            result.put(id(txnId), Ranges.of(build.get(txnId).toArray(new Range[0])));
+        return new Validate(new GenerateRanges(rangeDomain, 0f, 1f, 0f, 1f), result, RangeDeps.of(result));
+    }
+
+
+    // TODO (now): broaden patterns of random contents, sizes of collection etc.
+    @Test
+    public void testRandom()
+    {
+        Random random = new Random();
+//        long[] seeds = new long[]{-6268194734307361517L, -1531261279965735959L};
+//        Validate[] validates = new Validate[seeds.length];
+        for (int i = 0 ; i < 1000 ; ++i)
+        {
+            long seed = random.nextLong();
+//            long seed = -5637243003494330136L;
+            System.out.println("Seed: " + seed);
+            random.setSeed(seed);
+            generate(random, new GenerateRanges(1000, 0.01f, 0.3f, 0.1f, 1f), 100, 1000)
+                    .validate(random);
+        }
+    }
+
+    @Test
+    public void testIdenticalTransactions()
+    {
+        Random random = new Random();
+        for (int copies = 1 ; copies < 500 ; ++copies)
+        {
+            long seed = random.nextLong();
+//            long seed = -4951029115911714505L;
+            System.out.println("Seed: " + seed + ", copies: " + copies);
+            random.setSeed(seed);
+            generateIdenticalTxns(random, new GenerateRanges(1000, 0.01f, 0.3f, 0.1f, 1f), copies, 1000)
+                    .validate(random);
+        }
+    }
+
+    @Test
+    public void testNemesisRanges()
+    {
+        testNemesisRanges(0);
+    }
+
+    @Test
+    public void testHalfNemesisRanges()
+    {
+        testNemesisRanges(1);
+    }
+
+    private static void testNemesisRanges(int nonNemesisEntryPerNemesisEntry)
+    {
+        Random random = new Random();
+//        long seed = random.nextLong();
+        long seed = 3630794399457130643L;

Review Comment:
   Make random before commit?



##########
accord-core/src/main/java/accord/primitives/Deps.java:
##########
@@ -18,1311 +18,220 @@
 
 package accord.primitives;
 
-import java.util.*;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-import java.util.function.Function;
-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 accord.utils.Invariants;
 
-import static accord.utils.ArrayBuffers.*;
-import static accord.utils.SortedArrays.*;
-import static accord.utils.SortedArrays.Search.FAST;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
 
 /**
- * A collection of dependencies for a transaction, organised by the key the dependency is adopted via.
- * An inverse map from TxnId to Key may also be constructed and stored in this collection.
+ * A collection of transaction dependencies, keyed by the key or range on which they were adopted
  */
-// TODO (desired, consider): switch to RoutingKey? Would mean adopting execution dependencies less precisely, but saving ser/deser of large keys
-public class Deps implements Iterable<Map.Entry<Key, TxnId>>
+public class Deps
 {
-    private static final boolean DEBUG_CHECKS = true;
-
-    private static final TxnId[] NO_TXNIDS = new TxnId[0];
-    private static final int[] NO_INTS = new int[0];
-    public static final Deps NONE = new Deps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+    public static final Deps NONE = new Deps(KeyDeps.NONE, RangeDeps.NONE);
 
-    public static class SerializerSupport
+    public static Builder builder()
     {
-        private SerializerSupport() {}
-
-        public static int keyToTxnIdCount(Deps deps)
-        {
-            return deps.keyToTxnId.length;
-        }
-
-        public static int keyToTxnId(Deps deps, int idx)
-        {
-            return deps.keyToTxnId[idx];
-        }
-
-        public static Deps create(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
-        {
-            return new Deps(keys, txnIds, keyToTxnId);
-        }
-    }
-
-    public static Deps none(Keys keys)
-    {
-        int[] keysToTxnId = new int[keys.size()];
-        Arrays.fill(keysToTxnId, keys.size());
-        return new Deps(keys, NO_TXNIDS, keysToTxnId);
-    }
-
-    /**
-     * Expects Command to be provided in TxnId order
-     */
-    public static OrderedBuilder orderedBuilder(boolean hasOrderedTxnId)
-    {
-        return new OrderedBuilder(hasOrderedTxnId);
+        return new Builder();
     }
 
     // TODO (expected, efficiency): cache this object per thread
-    public static abstract class AbstractOrderedBuilder<T extends Deps> implements AutoCloseable
+    public static abstract class AbstractBuilder<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;
-        // txnId -> Offset
-        TxnId[] keyToTxnId;
-        int keyCount;
-        int keyOffset;
-        int totalCount;
-
-        public AbstractOrderedBuilder(boolean hasOrderedTxnId)
-        {
-            this.keys = cachedKeys.get(16);
-            this.keyLimits = cachedInts.getInts(keys.length);
-            this.hasOrderedTxnId = hasOrderedTxnId;
-            this.keyToTxnId = cachedTxnIds.get(16);
-        }
-
-        public boolean isEmpty()
-        {
-            return totalCount() == 0;
-        }
-
-        private int totalCount()
-        {
-            return totalCount;
-        }
+        final KeyDeps.Builder keyBuilder;
+        RangeDeps.Builder rangeBuilder;
 
-        public void nextKey(Key key)
+        AbstractBuilder()
         {
-            if (keyCount > 0 && keys[keyCount - 1].compareTo(key) >= 0)
-            {
-                throw new IllegalArgumentException("Key " + key + " has already been visited or was provided out of order ("
-                        + Arrays.toString(Arrays.copyOf(keys, keyCount)) + ")");
-            }
-
-            finishKey();
-
-            if (keyCount == keys.length)
-            {
-                Key[] newKeys = cachedKeys.get(keyCount * 2);
-                System.arraycopy(keys, 0, newKeys, 0, keyCount);
-                cachedKeys.forceDiscard(keys, keyCount);
-                keys = newKeys;
-                int[] newKeyLimits = cachedInts.getInts(keyCount * 2);
-                System.arraycopy(keyLimits, 0, newKeyLimits, 0, keyCount);
-                cachedInts.forceDiscard(keyLimits, keyCount);
-                keyLimits = newKeyLimits;
-            }
-            keys[keyCount++] = key;
+            this.keyBuilder = KeyDeps.builder();
         }
 
-        private void finishKey()
+        public AbstractBuilder<T> add(Seekable keyOrRange, TxnId txnId)
         {
-            if (totalCount == keyOffset && keyCount > 0)
-                --keyCount; // remove this key; no data
-
-            if (keyCount == 0)
-                return;
-
-            if (totalCount != keyOffset && !hasOrderedTxnId)
+            switch (keyOrRange.domain())
             {
-                // TODO (low priority, efficiency): this allocates a significant amount of memory: would be preferable to be able to sort using a pre-defined scratch buffer
-                Arrays.sort(keyToTxnId, keyOffset, totalCount);
-                for (int i = keyOffset + 1 ; i < totalCount ; ++i)
-                {
-                    if (keyToTxnId[i - 1].equals(keyToTxnId[i]))
-                        throw new IllegalArgumentException("TxnId for " + keys[keyCount - 1] + " are not unique: " + Arrays.asList(keyToTxnId).subList(keyOffset, totalCount));
-                }
+                default: throw new AssertionError();
+                case Key:
+                    keyBuilder.add(keyOrRange.asKey(), txnId);
+                    break;
+                case Range:
+                    if (rangeBuilder == null)
+                        rangeBuilder = RangeDeps.builder();
+                    rangeBuilder.add(keyOrRange.asRange(), txnId);
+                    break;
             }
-
-            keyLimits[keyCount - 1] = totalCount;
-            keyOffset = totalCount;
-        }
-
-        public void add(Key key, TxnId txnId)
-        {
-            if (keyCount == 0 || !keys[keyCount - 1].equals(key))
-                nextKey(key);
-            add(txnId);
-        }
-
-        /**
-         * Add this command as a dependency for each intersecting key
-         */
-        public void add(TxnId txnId)
-        {
-            if (hasOrderedTxnId && totalCount > keyOffset && keyToTxnId[totalCount - 1].compareTo(txnId) >= 0)
-                throw new IllegalArgumentException("TxnId provided out of order");
-
-            if (totalCount >= keyToTxnId.length)
-            {
-                TxnId[] newTxnIds = cachedTxnIds.get(keyToTxnId.length * 2);
-                System.arraycopy(keyToTxnId, 0, newTxnIds, 0, totalCount);
-                cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
-                keyToTxnId = newTxnIds;
-            }
-
-            keyToTxnId[totalCount++] = txnId;
-        }
-
-        public T build()
-        {
-            if (totalCount == 0)
-                return build(Keys.EMPTY, NO_TXNIDS, NO_INTS);
-
-            finishKey();
-
-            TxnId[] uniqueTxnId = cachedTxnIds.get(totalCount);
-            System.arraycopy(keyToTxnId, 0, uniqueTxnId, 0, totalCount);
-            Arrays.sort(uniqueTxnId, 0, totalCount);
-            int txnIdCount = 1;
-            for (int i = 1 ; i < totalCount ; ++i)
-            {
-                if (!uniqueTxnId[txnIdCount - 1].equals(uniqueTxnId[i]))
-                    uniqueTxnId[txnIdCount++] = uniqueTxnId[i];
-            }
-
-            TxnId[] txnIds = cachedTxnIds.complete(uniqueTxnId, txnIdCount);
-            cachedTxnIds.discard(uniqueTxnId, totalCount);
-
-            int[] result = new int[keyCount + totalCount];
-            int offset = keyCount;
-            for (int k = 0 ; k < keyCount ; ++k)
-            {
-                result[k] = keyCount + keyLimits[k];
-                int from = k == 0 ? 0 : keyLimits[k - 1];
-                int to = keyLimits[k];
-                offset = (int)SortedArrays.foldlIntersection(txnIds, 0, txnIdCount, keyToTxnId, from, to, (key, p, v, li, ri) -> {
-                    result[(int)v] = li;
-                    return v + 1;
-                }, keyCount, offset, -1);
-            }
-
-            return build(Keys.ofSortedUnchecked(cachedKeys.complete(keys, keyCount)), txnIds, result);
+            return this;
         }
 
-        abstract T build(Keys keys, TxnId[] txnIds, int[] keyToTxnId);
+        public abstract T build();
 
         @Override
         public void close()
         {
-            cachedKeys.discard(keys, keyCount);
-            cachedInts.forceDiscard(keyLimits, keyCount);
-            cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
-        }
-    }
-
-    public static class OrderedBuilder extends AbstractOrderedBuilder<Deps>
-    {
-        public OrderedBuilder(boolean hasOrderedTxnId)
-        {
-            super(hasOrderedTxnId);
-        }
-
-        @Override
-        Deps build(Keys keys, TxnId[] txnIds, int[] keysToTxnIds)
-        {
-            return new Deps(keys, txnIds, keysToTxnIds);
+            keyBuilder.close();
+            if (rangeBuilder != null)
+                rangeBuilder.close();
         }
     }
 
-    /**
-     * An object for managing a sequence of efficient linear merges Deps objects.
-     * Its primary purpose is to manage input and output buffers, so that we reuse output buffers
-     * as input to the next merge, and if any input is a superset of the other inputs that this input
-     * is returned unmodified.
-     *
-     * This is achieved by using PassThroughXBuffers so that the result buffers (and their sizes) are returned
-     * unmodified, and the buffers are cached as far as possible. In general, the buffers should be taken
-     * out of pre-existing caches, but if the buffers are too large then we cache any additional buffers we
-     * allocate for the duration of the merge.
-     */
-    private static class LinearMerger extends PassThroughObjectAndIntBuffers<TxnId> implements DepsConstructor<Key, TxnId, Object>
+    public static class Builder extends AbstractBuilder<Deps>
     {
-        final PassThroughObjectBuffers<Key> keyBuffers;
-        Key[] bufKeys;
-        TxnId[] bufTxnIds;
-        int[] buf = null;
-        int bufKeysLength, bufTxnIdsLength = 0, bufLength = 0;
-        Deps from = null;
-
-        LinearMerger()
-        {
-            super(cachedTxnIds(), cachedInts());
-            keyBuffers = new PassThroughObjectBuffers<>(cachedKeys());
-        }
-
-        @Override
-        public Object construct(Key[] keys, int keysLength, TxnId[] txnIds, int txnIdsLength, int[] out, int outLength)
-        {
-            if (from == null)
-            {
-                // if our input buffers were themselves buffers, we want to discard them unless they have been returned back to us
-                discard(keys, txnIds, out);
-            }
-            else if (buf != out)
-            {
-                // the output is not equal to a prior input
-                from = null;
-            }
-
-            if (from == null)
-            {
-                bufKeys = keys;
-                bufKeysLength = keysLength;
-                bufTxnIds = txnIds;
-                bufTxnIdsLength = txnIdsLength;
-                buf = out;
-                bufLength = outLength;
-            }
-            else
-            {
-                Invariants.checkState(keys == bufKeys && keysLength == bufKeysLength);
-                Invariants.checkState(txnIds == bufTxnIds && txnIdsLength == bufTxnIdsLength);
-                Invariants.checkState(outLength == bufLength);
-            }
-            return null;
-        }
-
-        void update(Deps deps)
+        public Builder()
         {
-            if (buf == null)
-            {
-                bufKeys = deps.keys.keys;
-                bufKeysLength = deps.keys.keys.length;
-                bufTxnIds = deps.txnIds;
-                bufTxnIdsLength = deps.txnIds.length;
-                buf = deps.keyToTxnId;
-                bufLength = deps.keyToTxnId.length;
-                from = deps;
-                return;
-            }
-
-            linearUnion(
-                    bufKeys, bufKeysLength, bufTxnIds, bufTxnIdsLength, buf, bufLength,
-                    deps.keys.keys, deps.keys.keys.length, deps.txnIds, deps.txnIds.length, deps.keyToTxnId, deps.keyToTxnId.length,
-                    keyBuffers, this, this, this
-            );
-            if (buf == deps.keyToTxnId)
-            {
-                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;
-            }
+            super();
         }
 
-        Deps get()
+        public Deps build()
         {
-            if (buf == null)
-                return NONE;
-
-            if (from != null)
-                return from;
-
-            return new Deps(
-                    Keys.ofSortedUnchecked(keyBuffers.realComplete(bufKeys, bufKeysLength)),
-                    realComplete(bufTxnIds, bufTxnIdsLength),
-                    realComplete(buf, bufLength));
-        }
-
-        /**
-         * Free any buffers we no longer need
-         */
-        void discard()
-        {
-            if (from == null)
-                discard(null, null, null);
-        }
-
-        /**
-         * Free buffers unless they are equal to the corresponding parameter
-         */
-        void discard(Key[] freeKeysIfNot, TxnId[] freeTxnIdsIfNot, int[] freeBufIfNot)
-        {
-            if (from != null)
-                return;
-
-            if (bufKeys != freeKeysIfNot)
-            {
-                keyBuffers.realDiscard(bufKeys, bufKeysLength);
-                bufKeys = null;
-            }
-            if (bufTxnIds != freeTxnIdsIfNot)
-            {
-                realDiscard(bufTxnIds, bufTxnIdsLength);
-                bufTxnIds = null;
-            }
-            if (buf != freeBufIfNot)
-            {
-                realDiscard(buf, bufLength);
-                buf = null;
-            }
+            return new Deps(keyBuilder.build(), rangeBuilder == null ? RangeDeps.NONE : rangeBuilder.build());
         }
     }
 
-    public static <T> Deps merge(List<T> merge, Function<T, Deps> getter)
-    {
-        LinearMerger linearMerger = new LinearMerger();
-        try
-        {
-            int mergeIndex = 0, mergeSize = merge.size();
-            while (mergeIndex < mergeSize)
-            {
-                Deps deps = getter.apply(merge.get(mergeIndex++));
-                if (deps == null || deps.isEmpty())
-                    continue;
-
-                linearMerger.update(deps);
-            }
-
-            return linearMerger.get();
-        }
-        finally
-        {
-            linearMerger.discard();
-        }
-    }
-
-    final Keys keys; // unique Keys
-    final TxnId[] txnIds; // unique TxnId TODO (low priority, efficiency): this could be a BTree?
+    public final KeyDeps keyDeps;
+    public final RangeDeps rangeDeps;
 
-    /**
-     * This represents a map of {@code Key -> [TxnId] } where each TxnId is actually a pointer into the txnIds array.
-     * The beginning of the array (the first keys.size() entries) are offsets into this array.
-     * <p/>
-     * Example:
-     * <p/>
-     * {@code
-     *   int keyIdx = keys.indexOf(key);
-     *   int startOfTxnOffset = keyIdx == 0 ? keys.size() : keyToTxnId[keyIdx - 1];
-     *   int endOfTxnOffset = keyToTxnId[keyIdx];
-     *   for (int i = startOfTxnOffset; i < endOfTxnOffset; i++)
-     *   {
-     *       TxnId id = txnIds[keyToTxnId[i]]
-     *       ...
-     *   }
-     * }
-     */
-    final int[] keyToTxnId; // Key -> [TxnId]
-    // Lazy loaded in ensureTxnIdToKey()
-    int[] txnIdToKey; // TxnId -> [Key]
-
-    Deps(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
+    public Deps(KeyDeps keyDeps, RangeDeps rangeDeps)
     {
-        this.keys = keys;
-        this.txnIds = txnIds;
-        this.keyToTxnId = keyToTxnId;
-        if (!(keys.isEmpty() || keyToTxnId[keys.size() - 1] == keyToTxnId.length))
-            throw new IllegalArgumentException(String.format("Last key (%s) in keyToTxnId does not point (%d) to the end of the array (%d);\nkeyToTxnId=%s", keys.get(keys.size() - 1), keyToTxnId[keys.size() - 1], keyToTxnId.length, Arrays.toString(keyToTxnId)));
-        if (DEBUG_CHECKS)
-            checkValid();
+        this.keyDeps = keyDeps;
+        this.rangeDeps = rangeDeps;
     }
 
-    public PartialDeps slice(Ranges ranges)
-    {
-        if (isEmpty())
-            return new PartialDeps(ranges, keys, txnIds, keyToTxnId);
-
-        Keys select = keys.slice(ranges);
-
-        if (select.isEmpty())
-            return new PartialDeps(ranges, Keys.EMPTY, NO_TXNIDS, NO_INTS);
-
-        if (select.size() == keys.size())
-            return new PartialDeps(ranges, keys, txnIds, keyToTxnId);
-
-        int i = 0;
-        int offset = select.size();
-        for (int j = 0 ; j < select.size() ; ++j)
-        {
-            int findi = keys.findNext(i, select.get(j), FAST);
-            if (findi < 0)
-                continue;
-
-            i = findi;
-            offset += keyToTxnId[i] - (i == 0 ? keys.size() : keyToTxnId[i - 1]);
-        }
-
-        int[] src = keyToTxnId;
-        int[] trg = new int[offset];
-
-        i = 0;
-        offset = select.size();
-        for (int j = 0 ; j < select.size() ; ++j)
-        {
-            int findi = keys.findNext(i, select.get(j), FAST);
-            if (findi >= 0)
-            {
-                i = findi;
-                int start = i == 0 ? keys.size() : src[i - 1];
-                int count = src[i] - start;
-                System.arraycopy(src, start, trg, offset, count);
-                offset += count;
-            }
-            trg[j] = offset;
-        }
-
-        TxnId[] txnIds = trimUnusedTxnId(select, this.txnIds, trg);
-        return new PartialDeps(ranges, select, txnIds, trg);
-    }
-
-    /**
-     * Returns the set of {@link TxnId}s that are referenced by {@code keysToTxnId}, and <strong>updates</strong>
-     * {@code keysToTxnId} to point to the new offsets in the returned set.
-     * @param keys object referenced by {@code keysToTxnId} index
-     * @param txnIds to trim to the seen {@link TxnId}s
-     * @param keysToTxnId to use as reference for trimming, this index will be updated to reflect the trimmed offsets.
-     * @return smallest set of {@link TxnId} seen in {@code keysToTxnId}
-     */
-    private static TxnId[] trimUnusedTxnId(Keys keys, TxnId[] txnIds, int[] keysToTxnId)
+    public boolean contains(TxnId txnId)
     {
-        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
-        {
-            Arrays.fill(remapTxnId, bitMapOffset, txnIds.length + 1, 0);
-            for (int i = keys.size() ; i < keysToTxnId.length ; ++i)
-                setBit(remapTxnId, bitMapOffset, keysToTxnId[i]);
-
-            int offset = 0;
-            for (int i = 0 ; i < txnIds.length ; ++i)
-            {
-                if (hasSetBit(remapTxnId, bitMapOffset, i)) remapTxnId[i] = offset++;
-                else remapTxnId[i] = -1;
-            }
-
-            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);
-        }
+        return keyDeps.contains(txnId) || rangeDeps.contains(txnId);
     }
 
     public Deps with(Deps that)
     {
-        if (isEmpty() || that.isEmpty())
-            return isEmpty() ? that : this;
-
-        return linearUnion(
-                this.keys.keys, this.keys.keys.length, this.txnIds, this.txnIds.length, this.keyToTxnId, this.keyToTxnId.length,
-                that.keys.keys, that.keys.keys.length, that.txnIds, that.txnIds.length, that.keyToTxnId, that.keyToTxnId.length,
-                cachedKeys(), cachedTxnIds(), cachedInts(),
-                (keys, keysLength, txnIds, txnIdsLength, out, outLength) ->
-                        new Deps(Keys.ofSortedUnchecked(cachedKeys().complete(keys, keysLength)),
-                                cachedTxnIds().complete(txnIds, txnIdsLength),
-                                cachedInts().complete(out, outLength))
-                );
-    }
-
-    /**
-     * Turn a set of key, value and mapping buffers into a merge result;
-     * K and V are either Key and TxnId, or vice versa, depending on which mapping direction was present
-     */
-    interface DepsConstructor<K, V, T>
-    {
-        T construct(K[] keys, int keysLength, V[] values, int valuesLength, int[] out, int outLength);
-    }
-
-    private static boolean arraysEqual(int[] left, int[] right, int length)
-    {
-        if (left.length < length || right.length < length)
-            return false;
-
-        for (int i=0; i<length; i++)
-            if (left[i] !=right[i])
-                return false;
-
-        return true;
-    }
-
-    private static <T> boolean arraysEqual(T[] left, T[] right, int length)
-    {
-        if (left.length < length || right.length < length)
-            return false;
-
-        for (int i=0; i<length; i++)
-            if (!Objects.equals(left[i], right[i]))
-                return false;
-
-        return true;
-    }
-
-    // TODO (low priority, efficiency): this method supports merging keyToTxnId OR txnIdToKey; we can perhaps save time
-    //  and effort when constructing Deps on remote hosts by only producing txnIdToKey with OrderedCollector and serializing
-    //  only this, and merging on the recipient before inverting, so that we only have to invert the final assembled deps
-    private static <K extends Comparable<? super K>, V extends Comparable<? super V>, T>
-    T linearUnion(K[] leftKeys, int leftKeysLength, V[] leftValues, int leftValuesLength, int[] left, int leftLength,
-                  K[] rightKeys, int rightKeysLength, V[] rightValues, int rightValuesLength, int[] right, int rightLength,
-                  ObjectBuffers<K> keyBuffers, ObjectBuffers<V> valueBuffers, IntBuffers intBuffers, DepsConstructor<K, V, T> constructor)
-    {
-        K[] outKeys = null;
-        V[] outValues = null;
-        int[] remapLeft = null, remapRight = null, out = null;
-        int outLength = 0, outKeysLength = 0, outTxnIdsLength = 0;
-
-        try
-        {
-            outKeys = SortedArrays.linearUnion(leftKeys, leftKeysLength, rightKeys, rightKeysLength, keyBuffers);
-            outKeysLength = keyBuffers.lengthOfLast(outKeys);
-            outValues = SortedArrays.linearUnion(leftValues, leftValuesLength, rightValues, rightValuesLength, valueBuffers);
-            outTxnIdsLength = valueBuffers.lengthOfLast(outValues);
-
-            remapLeft = remapToSuperset(leftValues, leftValuesLength, outValues, outTxnIdsLength, intBuffers);
-            remapRight = remapToSuperset(rightValues, rightValuesLength, outValues, outTxnIdsLength, intBuffers);
-
-            if (remapLeft == null && remapRight == null && leftLength == rightLength && leftKeysLength == rightKeysLength
-                    && arraysEqual(left, right, rightLength)
-                    && arraysEqual(leftKeys, rightKeys, rightKeysLength)
-                )
-            {
-                return constructor.construct(leftKeys, leftKeysLength, leftValues, leftValuesLength, left, leftLength);
-            }
-
-            int lk = 0, rk = 0, ok = 0, l = leftKeysLength, r = rightKeysLength;
-            outLength = outKeysLength;
-
-            if (remapLeft == null && outKeys == leftKeys)
-            {
-                // "this" knows all the TxnId and Keys already, but do both agree on what Keys map to TxnIds?
-                noOp: while (lk < leftKeysLength && rk < rightKeysLength)
-                {
-                    int ck = leftKeys[lk].compareTo(rightKeys[rk]);
-                    if (ck < 0)
-                    {
-                        // "this" knows of a key not present in "that"
-                        outLength += left[lk] - l; // logically append the key's TxnIds to the size
-                        l = left[lk];
-                        assert outLength == l && ok == lk && left[ok] == outLength;
-                        ok++;
-                        lk++;
-                    }
-                    else if (ck > 0)
-                    {
-                        // if this happened there is a bug with keys.union or keys are not actually sorted
-                        throwUnexpectedMissingKeyException(leftKeys, lk, leftKeysLength, rightKeys, rk, rightKeysLength, true);
-                    }
-                    else
-                    {
-                        // both "this" and "that" know of the key
-                        while (l < left[lk] && r < right[rk])
-                        {
-                            int nextLeft = left[l];
-                            int nextRight = remap(right[r], remapRight);
-
-                            if (nextLeft < nextRight)
-                            {
-                                // "this" knows of the txn that "that" didn't
-                                outLength++;
-                                l++;
-                            }
-                            else if (nextRight < nextLeft)
-                            {
-                                out = copy(left, outLength, leftLength + rightLength - r, intBuffers);
-                                break noOp;
-                            }
-                            else
-                            {
-                                outLength++;
-                                l++;
-                                r++;
-                            }
-                        }
-
-                        if (l < left[lk])
-                        {
-                            outLength += left[lk] - l;
-                            l = left[lk];
-                        }
-                        else if (r < right[rk])
-                        {
-                            // "that" thinks a key includes a TxnId as a dependency but "this" doesn't, need to include this knowledge
-                            out = copy(left, outLength, leftLength + rightLength - r, intBuffers);
-                            break;
-                        }
-
-                        assert outLength == l && ok == lk && left[ok] == outLength;
-                        ok++;
-                        rk++;
-                        lk++;
-                    }
-                }
-
-                if (out == null)
-                    return constructor.construct(leftKeys, leftKeysLength, leftValues, leftValuesLength, left, leftLength);
-            }
-            else if (remapRight == null && outKeys == rightKeys)
-            {
-                // "that" knows all the TxnId and keys already, but "this" does not
-                noOp: while (lk < leftKeysLength && rk < rightKeysLength)
-                {
-                    int ck = leftKeys[lk].compareTo(rightKeys[rk]);
-                    if (ck < 0)
-                    {
-                        // if this happened there is a bug with keys.union or keys are not actually sorted
-                        throwUnexpectedMissingKeyException(leftKeys, lk, leftKeysLength, rightKeys, rk, rightKeysLength, false);
-                    }
-                    else if (ck > 0)
-                    {
-                        outLength += right[rk] - r;
-                        r = right[rk];
-                        assert outLength == r && ok == rk && right[ok] == outLength;
-                        ok++;
-                        rk++;
-                    }
-                    else
-                    {
-                        // both "this" and "that" know of the key
-                        while (l < left[lk] && r < right[rk])
-                        {
-                            int nextLeft = remap(left[l], remapLeft);
-                            int nextRight = right[r];
-
-                            if (nextLeft < nextRight)
-                            {
-                                // "this" thinks a TxnID depends on Key but "that" doesn't, need to include this knowledge
-                                out = copy(right, outLength, rightLength + leftLength - l, intBuffers);
-                                break noOp;
-                            }
-                            else if (nextRight < nextLeft)
-                            {
-                                // "that" knows of the txn that "this" didn't
-                                outLength++;
-                                r++;
-                            }
-                            else
-                            {
-                                outLength++;
-                                l++;
-                                r++;
-                            }
-                        }
-
-                        if (l < left[lk])
-                        {
-                            out = copy(right, outLength, rightLength + leftLength - l, intBuffers);
-                            break;
-                        }
-                        else if (r < right[rk])
-                        {
-                            outLength += right[rk] - r;
-                            r = right[rk];
-                        }
-
-                        assert outLength == r && ok == rk && right[ok] == outLength;
-                        ok++;
-                        rk++;
-                        lk++;
-                    }
-                }
-
-                if (out == null)
-                    return constructor.construct(rightKeys, rightKeysLength, rightValues, rightValuesLength, right, rightLength);
-            }
-            else
-            {
-                out = intBuffers.getInts(leftLength + rightLength);
-            }
-
-            while (lk < leftKeysLength && rk < rightKeysLength)
-            {
-                int ck = leftKeys[lk].compareTo(rightKeys[rk]);
-                if (ck < 0)
-                {
-                    while (l < left[lk])
-                        out[outLength++] = remap(left[l++], remapLeft);
-                    out[ok++] = outLength;
-                    lk++;
-                }
-                else if (ck > 0)
-                {
-                    while (r < right[rk])
-                        out[outLength++] = remap(right[r++], remapRight);
-                    out[ok++] = outLength;
-                    rk++;
-                }
-                else
-                {
-                    while (l < left[lk] && r < right[rk])
-                    {
-                        int nextLeft = remap(left[l], remapLeft);
-                        int nextRight = remap(right[r], remapRight);
-
-                        if (nextLeft <= nextRight)
-                        {
-                            out[outLength++] = nextLeft;
-                            l += 1;
-                            r += nextLeft == nextRight ? 1 : 0;
-                        }
-                        else
-                        {
-                            out[outLength++] = nextRight;
-                            ++r;
-                        }
-                    }
-
-                    while (l < left[lk])
-                        out[outLength++] = remap(left[l++], remapLeft);
-
-                    while (r < right[rk])
-                        out[outLength++] = remap(right[r++], remapRight);
-
-                    out[ok++] = outLength;
-                    rk++;
-                    lk++;
-                }
-            }
-
-            while (lk < leftKeysLength)
-            {
-                while (l < left[lk])
-                    out[outLength++] = remap(left[l++], remapLeft);
-                out[ok++] = outLength;
-                lk++;
-            }
-
-            while (rk < rightKeysLength)
-            {
-                while (r < right[rk])
-                    out[outLength++] = remap(right[r++], remapRight);
-                out[ok++] = outLength;
-                rk++;
-            }
-
-            return constructor.construct(outKeys, outKeysLength, outValues, outTxnIdsLength, out, outLength);
-        }
-        finally
-        {
-            if (outKeys != null)
-                keyBuffers.discard(outKeys, outKeysLength);
-            if (outValues != null)
-                valueBuffers.discard(outValues, outTxnIdsLength);
-            if (out != null)
-                intBuffers.discard(out, outLength);
-            if (remapLeft != null)
-                intBuffers.forceDiscard(remapLeft, leftValuesLength);
-            if (remapRight != null)
-                intBuffers.forceDiscard(remapRight, rightValuesLength);
-        }
-    }
-
-    private static <A> void throwUnexpectedMissingKeyException(A[] leftKeys, int leftKeyIndex, int leftKeyLength, A[] rightKeys, int rightKeyIndex, int rightKeyLength, boolean isMissingLeft)
-    {
-        StringBuilder sb = new StringBuilder();
-        String missing = isMissingLeft ? "left" : "right";
-        String extra = isMissingLeft ? "right" : "left";
-        sb.append(missing).append(" knows all keys, yet ").append(extra).append(" knew of an extra key at indexes left[")
-                .append(leftKeyIndex).append("] = ").append(leftKeys[leftKeyIndex])
-                .append(", right[").append(rightKeyIndex).append("] = ").append(rightKeys[rightKeyIndex]).append("\n");
-        sb.append("leftKeys = ").append(Arrays.stream(leftKeys, 0, leftKeyLength).map(Object::toString).collect(Collectors.joining())).append('\n');
-        sb.append("rightKeys = ").append(Arrays.stream(rightKeys, 0, rightKeyLength).map(Object::toString).collect(Collectors.joining())).append('\n');
-        throw new IllegalStateException(sb.toString());
-    }
-
-    private static int[] copy(int[] src, int to, int length, IntBuffers bufferManager)
-    {
-        if (length == 0)
-            return NO_INTS;
-
-        int[] result = bufferManager.getInts(length);
-        if (result.length < length)
-            throw new IllegalStateException();
-        System.arraycopy(src, 0, result, 0, to);
-        return result;
+        return new Deps(this.keyDeps.with(that.keyDeps), this.rangeDeps.with(that.rangeDeps));
     }
 
     public Deps without(Predicate<TxnId> remove)
     {
-        if (isEmpty())
-            return this;
-
-        IntBuffers cache = ArrayBuffers.cachedInts();
-        TxnId[] oldTxnIds = txnIds;
-        int[] oldKeyToTxnId = keyToTxnId;
-        int[] remapTxnIds = cache.getInts(oldTxnIds.length);
-        int[] newKeyToTxnId = null;
-        TxnId[] newTxnIds;
-        int o = 0;
-        try
-        {
-            int count = 0;
-            for (int i = 0 ; i < oldTxnIds.length ; ++i)
-            {
-                if (remove.test(oldTxnIds[i])) remapTxnIds[i] = -1;
-                else remapTxnIds[i] = count++;
-            }
-
-            if (count == oldTxnIds.length)
-                return this;
-
-            if (count == 0)
-                return NONE;
-
-            newTxnIds = new TxnId[count];
-            for (int i = 0 ; i < oldTxnIds.length ; ++i)
-            {
-                if (remapTxnIds[i] >= 0)
-                    newTxnIds[remapTxnIds[i]] = oldTxnIds[i];
-            }
-
-            newKeyToTxnId = cache.getInts(oldKeyToTxnId.length);
-            int k = 0, i = keys.size();
-            o = i;
-            while (i < oldKeyToTxnId.length)
-            {
-                while (oldKeyToTxnId[k] == i)
-                    newKeyToTxnId[k++] = o;
-
-                int remapped = remapTxnIds[oldKeyToTxnId[i]];
-                if (remapped >= 0)
-                    newKeyToTxnId[o++] = remapped;
-                ++i;
-            }
-
-            while (k < keys.size())
-                newKeyToTxnId[k++] = o;
-        }
-        catch (Throwable t)
-        {
-            cache.forceDiscard(newKeyToTxnId, o);
-            throw t;
-        }
-        finally
-        {
-            cache.forceDiscard(remapTxnIds, oldTxnIds.length);
-        }
-
-        newKeyToTxnId = cache.completeAndDiscard(newKeyToTxnId, o);
-        return new Deps(keys, newTxnIds, newKeyToTxnId);
+        return new Deps(keyDeps.without(remove), rangeDeps.without(remove));
     }
 
-    public boolean contains(TxnId txnId)
+    public PartialDeps slice(Ranges covering)
     {
-        return Arrays.binarySearch(txnIds, txnId) >= 0;
+        return new PartialDeps(covering, keyDeps.slice(covering), rangeDeps.slice(covering));
     }
 
-    // return true iff we map any keys to any txnId
-    // if the mapping is empty we return false, whether or not we have any keys or txnId by themselves
     public boolean isEmpty()
     {
-        return keyToTxnId.length == keys.size();
-    }
-
-    public Keys someKeys(TxnId txnId)
-    {
-        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
-        if (txnIdIndex < 0)
-            return Keys.EMPTY;
-
-        ensureTxnIdToKey();
-
-        int start = txnIdIndex == 0 ? txnIds.length : txnIdToKey[txnIdIndex - 1];
-        int end = txnIdToKey[txnIdIndex];
-        if (start == end)
-            return Keys.EMPTY;
-
-        Key[] result = new Key[end - start];
-        for (int i = start ; i < end ; ++i)
-            result[i - start] = keys.get(txnIdToKey[i]);
-        return Keys.of(result);
-    }
-
-    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)
-            constructor.apply(RoutingKeys.EMPTY.keys);
-
-        ensureTxnIdToKey();
-
-        int start = txnIdIndex == 0 ? txnIds.length : txnIdToKey[txnIdIndex - 1];
-        int end = txnIdToKey[txnIdIndex];
-        RoutingKey[] result = new RoutingKey[end - start];
-        if (start == end)
-            constructor.apply(RoutingKeys.EMPTY.keys);
-
-        result[0] = keys.get(txnIdToKey[start]).toUnseekable();
-        int resultCount = 1;
-        for (int i = start + 1 ; i < end ; ++i)
-        {
-            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 constructor.apply(result);
-    }
-
-    void ensureTxnIdToKey()
-    {
-        if (txnIdToKey != null)
-            return;
-
-        txnIdToKey = invert(keyToTxnId, keyToTxnId.length, keys.size(), txnIds.length);
-    }
-
-    private static int[] invert(int[] src, int srcLength, int srcKeyCount, int trgKeyCount)
-    {
-        int[] trg = new int[trgKeyCount + srcLength - srcKeyCount];
-
-        // first pass, count number of txnId per key
-        for (int i = srcKeyCount ; i < srcLength ; ++i)
-            trg[src[i]]++;
-
-        // turn into offsets (i.e. add txnIds.size() and then sum them)
-        trg[0] += trgKeyCount;
-        for (int i = 1; i < trgKeyCount ; ++i)
-            trg[i] += trg[i - 1];
-
-        // shuffle forwards one, so we have the start index rather than end
-        System.arraycopy(trg, 0, trg, 1, trgKeyCount - 1);
-        trg[0] = trgKeyCount;
-
-        // convert the offsets to end, and set the key at the target positions
-        int k = 0;
-        for (int i = srcKeyCount ; i < srcLength ; ++i)
-        {
-            // if at the end offset, switch to the next key
-            while (i == src[k])
-                ++k;
-
-            // find the next key offset for the TxnId and set the offset to this key
-            trg[trg[src[i]]++] = k;
-        }
-
-        return trg;
-    }
-
-    public void forEachOn(Ranges ranges, Predicate<Key> include, BiConsumer<Key, TxnId> forEach)
-    {
-        Routables.foldl(keys, ranges, (key, value, index) -> {
-            if (!include.test(key))
-                return null;
-
-            for (int t = startOffset(index), end = endOffset(index); t < end ; ++t)
-            {
-                TxnId txnId = txnIds[keyToTxnId[t]];
-                forEach.accept(key, txnId);
-            }
-            return null;
-        }, null);
-    }
-
-    /**
-     * 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 forEach function to call on each unique {@link TxnId}
-     */
-    public void forEachOn(Ranges ranges, 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
-        // the property that forEach is called in TxnId order.
-        //TODO (expected, efficiency): reconsider this, probably not worth trying to save allocations at cost of multiple loop
-        //                             use BitSet, or perhaps extend so we can have no nested allocations when few bits
-        for (int offset = 0 ; offset < txnIds.length ; offset += 64)
-        {
-            long bitset = Routables.foldl(keys, ranges, (key, off, value, keyIndex) -> {
-                int index = startOffset(keyIndex);
-                int end = endOffset(keyIndex);
-                if (off > 0)
-                {
-                    // TODO (low priority, efficiency): interpolation search probably great here
-                    index = Arrays.binarySearch(keyToTxnId, index, end, (int)off);
-                    if (index < 0)
-                        index = -1 - index;
-                }
-
-                while (index < end)
-                {
-                    long next = keyToTxnId[index++] - off;
-                    if (next >= 64)
-                        break;
-                    value |= 1L << next;
-                }
-
-                return value;
-            }, offset, 0, -1L);
-
-            while (bitset != 0)
-            {
-                int i = Long.numberOfTrailingZeros(bitset);
-                TxnId txnId = txnIds[offset + i];
-                forEach.accept(txnId);
-                bitset ^= Long.lowestOneBit(bitset);
-            }
-        }
-    }
-
-    public void forEach(Key key, Consumer<TxnId> forEach)
-    {
-        int keyIndex = keys.indexOf(key);
-        if (keyIndex < 0)
-            return;
-
-        int index = startOffset(keyIndex);
-        int end = endOffset(keyIndex);
-        while (index < end)
-            forEach.accept(txnIds[keyToTxnId[index++]]);
-    }
-
-    public Keys keys()
-    {
-        return keys;
+        return keyDeps.isEmpty() && rangeDeps.isEmpty();
     }
 
     public int txnIdCount()
     {
-        return txnIds.length;
-    }
-
-    public int totalCount()
-    {
-        return keyToTxnId.length - keys.size();
+        return keyDeps.txnIdCount() + rangeDeps.txnIdCount();
     }
 
     public TxnId txnId(int i)
     {
-        return txnIds[i];
+        return i < keyDeps.txnIdCount()
+                ? keyDeps.txnId(i)
+                : rangeDeps.txnId(i - keyDeps.txnIdCount());
     }
 
-    public Collection<TxnId> txnIds()
+    public List<TxnId> txnIds()
     {
-        return Arrays.asList(txnIds);
-    }
-
-    public List<TxnId> txnIds(Key key)
-    {
-        int keyIndex = keys.indexOf(key);
-        if (keyIndex < 0)
-            return Collections.emptyList();
-
-        int start = startOffset(keyIndex);
-        int end = endOffset(keyIndex);
-        int size = end - start;
-
-        return new AbstractList<TxnId>()
-        {
+        final int txnIdCount = txnIdCount();
+        final int keyDepsCount = keyDeps.txnIdCount();
+        return new AbstractList<TxnId>() {
             @Override
             public TxnId get(int index)
             {
-                if (index > end)
-                    throw new IndexOutOfBoundsException();
-                return txnIds[keyToTxnId[start + index]];
+                return index < keyDepsCount
+                        ? keyDeps.txnId(index)
+                        : rangeDeps.txnId(index - keyDepsCount);
             }
 
             @Override
-            public int size()
-            {
-                return size;
+            public int size() {

Review Comment:
   Newline?



##########
accord-core/src/main/java/accord/primitives/PartialDeps.java:
##########
@@ -1,49 +1,37 @@
 package accord.primitives;
 
-import com.google.common.base.Preconditions;
+import accord.utils.Invariants;
 
 public class PartialDeps extends Deps
 {
-    public static final PartialDeps NONE = new PartialDeps(Ranges.EMPTY, Deps.NONE.keys, Deps.NONE.txnIds, Deps.NONE.keyToTxnId);
+    public static final PartialDeps NONE = new PartialDeps(Ranges.EMPTY, KeyDeps.NONE, RangeDeps.NONE);
 
-    public static class SerializerSupport
+    public static Builder builder(Ranges covering)
     {
-        private SerializerSupport() {}
-
-        public static PartialDeps create(Ranges covering, Keys keys, TxnId[] txnIds, int[] keyToTxnId)
-        {
-            return new PartialDeps(covering, keys, txnIds, keyToTxnId);
-        }
+        return new Builder(covering);
     }
-
-    public static class OrderedBuilder extends AbstractOrderedBuilder<PartialDeps>
+    public static class Builder extends AbstractBuilder<PartialDeps>
     {
         final Ranges covering;
-        public OrderedBuilder(Ranges covering, boolean hasOrderedTxnId)
+        public Builder(Ranges covering)
         {
-            super(hasOrderedTxnId);
             this.covering = covering;
         }
 
-        @Override
-        PartialDeps build(Keys keys, TxnId[] txnIds, int[] keysToTxnIds)
+        public PartialDeps build()

Review Comment:
   Missing `@Override`?



##########
accord-core/src/main/java/accord/primitives/Deps.java:
##########
@@ -18,1311 +18,220 @@
 
 package accord.primitives;
 
-import java.util.*;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-import java.util.function.Function;
-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 accord.utils.Invariants;
 
-import static accord.utils.ArrayBuffers.*;
-import static accord.utils.SortedArrays.*;
-import static accord.utils.SortedArrays.Search.FAST;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
 
 /**
- * A collection of dependencies for a transaction, organised by the key the dependency is adopted via.
- * An inverse map from TxnId to Key may also be constructed and stored in this collection.
+ * A collection of transaction dependencies, keyed by the key or range on which they were adopted
  */
-// TODO (desired, consider): switch to RoutingKey? Would mean adopting execution dependencies less precisely, but saving ser/deser of large keys
-public class Deps implements Iterable<Map.Entry<Key, TxnId>>
+public class Deps
 {
-    private static final boolean DEBUG_CHECKS = true;
-
-    private static final TxnId[] NO_TXNIDS = new TxnId[0];
-    private static final int[] NO_INTS = new int[0];
-    public static final Deps NONE = new Deps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+    public static final Deps NONE = new Deps(KeyDeps.NONE, RangeDeps.NONE);
 
-    public static class SerializerSupport
+    public static Builder builder()
     {
-        private SerializerSupport() {}
-
-        public static int keyToTxnIdCount(Deps deps)
-        {
-            return deps.keyToTxnId.length;
-        }
-
-        public static int keyToTxnId(Deps deps, int idx)
-        {
-            return deps.keyToTxnId[idx];
-        }
-
-        public static Deps create(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
-        {
-            return new Deps(keys, txnIds, keyToTxnId);
-        }
-    }
-
-    public static Deps none(Keys keys)
-    {
-        int[] keysToTxnId = new int[keys.size()];
-        Arrays.fill(keysToTxnId, keys.size());
-        return new Deps(keys, NO_TXNIDS, keysToTxnId);
-    }
-
-    /**
-     * Expects Command to be provided in TxnId order
-     */
-    public static OrderedBuilder orderedBuilder(boolean hasOrderedTxnId)
-    {
-        return new OrderedBuilder(hasOrderedTxnId);
+        return new Builder();
     }
 
     // TODO (expected, efficiency): cache this object per thread
-    public static abstract class AbstractOrderedBuilder<T extends Deps> implements AutoCloseable
+    public static abstract class AbstractBuilder<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;
-        // txnId -> Offset
-        TxnId[] keyToTxnId;
-        int keyCount;
-        int keyOffset;
-        int totalCount;
-
-        public AbstractOrderedBuilder(boolean hasOrderedTxnId)
-        {
-            this.keys = cachedKeys.get(16);
-            this.keyLimits = cachedInts.getInts(keys.length);
-            this.hasOrderedTxnId = hasOrderedTxnId;
-            this.keyToTxnId = cachedTxnIds.get(16);
-        }
-
-        public boolean isEmpty()
-        {
-            return totalCount() == 0;
-        }
-
-        private int totalCount()
-        {
-            return totalCount;
-        }
+        final KeyDeps.Builder keyBuilder;
+        RangeDeps.Builder rangeBuilder;
 
-        public void nextKey(Key key)
+        AbstractBuilder()
         {
-            if (keyCount > 0 && keys[keyCount - 1].compareTo(key) >= 0)
-            {
-                throw new IllegalArgumentException("Key " + key + " has already been visited or was provided out of order ("
-                        + Arrays.toString(Arrays.copyOf(keys, keyCount)) + ")");
-            }
-
-            finishKey();
-
-            if (keyCount == keys.length)
-            {
-                Key[] newKeys = cachedKeys.get(keyCount * 2);
-                System.arraycopy(keys, 0, newKeys, 0, keyCount);
-                cachedKeys.forceDiscard(keys, keyCount);
-                keys = newKeys;
-                int[] newKeyLimits = cachedInts.getInts(keyCount * 2);
-                System.arraycopy(keyLimits, 0, newKeyLimits, 0, keyCount);
-                cachedInts.forceDiscard(keyLimits, keyCount);
-                keyLimits = newKeyLimits;
-            }
-            keys[keyCount++] = key;
+            this.keyBuilder = KeyDeps.builder();
         }
 
-        private void finishKey()
+        public AbstractBuilder<T> add(Seekable keyOrRange, TxnId txnId)
         {
-            if (totalCount == keyOffset && keyCount > 0)
-                --keyCount; // remove this key; no data
-
-            if (keyCount == 0)
-                return;
-
-            if (totalCount != keyOffset && !hasOrderedTxnId)
+            switch (keyOrRange.domain())
             {
-                // TODO (low priority, efficiency): this allocates a significant amount of memory: would be preferable to be able to sort using a pre-defined scratch buffer
-                Arrays.sort(keyToTxnId, keyOffset, totalCount);
-                for (int i = keyOffset + 1 ; i < totalCount ; ++i)
-                {
-                    if (keyToTxnId[i - 1].equals(keyToTxnId[i]))
-                        throw new IllegalArgumentException("TxnId for " + keys[keyCount - 1] + " are not unique: " + Arrays.asList(keyToTxnId).subList(keyOffset, totalCount));
-                }
+                default: throw new AssertionError();
+                case Key:
+                    keyBuilder.add(keyOrRange.asKey(), txnId);
+                    break;
+                case Range:
+                    if (rangeBuilder == null)
+                        rangeBuilder = RangeDeps.builder();
+                    rangeBuilder.add(keyOrRange.asRange(), txnId);
+                    break;
             }
-
-            keyLimits[keyCount - 1] = totalCount;
-            keyOffset = totalCount;
-        }
-
-        public void add(Key key, TxnId txnId)
-        {
-            if (keyCount == 0 || !keys[keyCount - 1].equals(key))
-                nextKey(key);
-            add(txnId);
-        }
-
-        /**
-         * Add this command as a dependency for each intersecting key
-         */
-        public void add(TxnId txnId)
-        {
-            if (hasOrderedTxnId && totalCount > keyOffset && keyToTxnId[totalCount - 1].compareTo(txnId) >= 0)
-                throw new IllegalArgumentException("TxnId provided out of order");
-
-            if (totalCount >= keyToTxnId.length)
-            {
-                TxnId[] newTxnIds = cachedTxnIds.get(keyToTxnId.length * 2);
-                System.arraycopy(keyToTxnId, 0, newTxnIds, 0, totalCount);
-                cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
-                keyToTxnId = newTxnIds;
-            }
-
-            keyToTxnId[totalCount++] = txnId;
-        }
-
-        public T build()
-        {
-            if (totalCount == 0)
-                return build(Keys.EMPTY, NO_TXNIDS, NO_INTS);
-
-            finishKey();
-
-            TxnId[] uniqueTxnId = cachedTxnIds.get(totalCount);
-            System.arraycopy(keyToTxnId, 0, uniqueTxnId, 0, totalCount);
-            Arrays.sort(uniqueTxnId, 0, totalCount);
-            int txnIdCount = 1;
-            for (int i = 1 ; i < totalCount ; ++i)
-            {
-                if (!uniqueTxnId[txnIdCount - 1].equals(uniqueTxnId[i]))
-                    uniqueTxnId[txnIdCount++] = uniqueTxnId[i];
-            }
-
-            TxnId[] txnIds = cachedTxnIds.complete(uniqueTxnId, txnIdCount);
-            cachedTxnIds.discard(uniqueTxnId, totalCount);
-
-            int[] result = new int[keyCount + totalCount];
-            int offset = keyCount;
-            for (int k = 0 ; k < keyCount ; ++k)
-            {
-                result[k] = keyCount + keyLimits[k];
-                int from = k == 0 ? 0 : keyLimits[k - 1];
-                int to = keyLimits[k];
-                offset = (int)SortedArrays.foldlIntersection(txnIds, 0, txnIdCount, keyToTxnId, from, to, (key, p, v, li, ri) -> {
-                    result[(int)v] = li;
-                    return v + 1;
-                }, keyCount, offset, -1);
-            }
-
-            return build(Keys.ofSortedUnchecked(cachedKeys.complete(keys, keyCount)), txnIds, result);
+            return this;
         }
 
-        abstract T build(Keys keys, TxnId[] txnIds, int[] keyToTxnId);
+        public abstract T build();
 
         @Override
         public void close()
         {
-            cachedKeys.discard(keys, keyCount);
-            cachedInts.forceDiscard(keyLimits, keyCount);
-            cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
-        }
-    }
-
-    public static class OrderedBuilder extends AbstractOrderedBuilder<Deps>
-    {
-        public OrderedBuilder(boolean hasOrderedTxnId)
-        {
-            super(hasOrderedTxnId);
-        }
-
-        @Override
-        Deps build(Keys keys, TxnId[] txnIds, int[] keysToTxnIds)
-        {
-            return new Deps(keys, txnIds, keysToTxnIds);
+            keyBuilder.close();
+            if (rangeBuilder != null)
+                rangeBuilder.close();
         }
     }
 
-    /**
-     * An object for managing a sequence of efficient linear merges Deps objects.
-     * Its primary purpose is to manage input and output buffers, so that we reuse output buffers
-     * as input to the next merge, and if any input is a superset of the other inputs that this input
-     * is returned unmodified.
-     *
-     * This is achieved by using PassThroughXBuffers so that the result buffers (and their sizes) are returned
-     * unmodified, and the buffers are cached as far as possible. In general, the buffers should be taken
-     * out of pre-existing caches, but if the buffers are too large then we cache any additional buffers we
-     * allocate for the duration of the merge.
-     */
-    private static class LinearMerger extends PassThroughObjectAndIntBuffers<TxnId> implements DepsConstructor<Key, TxnId, Object>
+    public static class Builder extends AbstractBuilder<Deps>
     {
-        final PassThroughObjectBuffers<Key> keyBuffers;
-        Key[] bufKeys;
-        TxnId[] bufTxnIds;
-        int[] buf = null;
-        int bufKeysLength, bufTxnIdsLength = 0, bufLength = 0;
-        Deps from = null;
-
-        LinearMerger()
-        {
-            super(cachedTxnIds(), cachedInts());
-            keyBuffers = new PassThroughObjectBuffers<>(cachedKeys());
-        }
-
-        @Override
-        public Object construct(Key[] keys, int keysLength, TxnId[] txnIds, int txnIdsLength, int[] out, int outLength)
-        {
-            if (from == null)
-            {
-                // if our input buffers were themselves buffers, we want to discard them unless they have been returned back to us
-                discard(keys, txnIds, out);
-            }
-            else if (buf != out)
-            {
-                // the output is not equal to a prior input
-                from = null;
-            }
-
-            if (from == null)
-            {
-                bufKeys = keys;
-                bufKeysLength = keysLength;
-                bufTxnIds = txnIds;
-                bufTxnIdsLength = txnIdsLength;
-                buf = out;
-                bufLength = outLength;
-            }
-            else
-            {
-                Invariants.checkState(keys == bufKeys && keysLength == bufKeysLength);
-                Invariants.checkState(txnIds == bufTxnIds && txnIdsLength == bufTxnIdsLength);
-                Invariants.checkState(outLength == bufLength);
-            }
-            return null;
-        }
-
-        void update(Deps deps)
+        public Builder()
         {
-            if (buf == null)
-            {
-                bufKeys = deps.keys.keys;
-                bufKeysLength = deps.keys.keys.length;
-                bufTxnIds = deps.txnIds;
-                bufTxnIdsLength = deps.txnIds.length;
-                buf = deps.keyToTxnId;
-                bufLength = deps.keyToTxnId.length;
-                from = deps;
-                return;
-            }
-
-            linearUnion(
-                    bufKeys, bufKeysLength, bufTxnIds, bufTxnIdsLength, buf, bufLength,
-                    deps.keys.keys, deps.keys.keys.length, deps.txnIds, deps.txnIds.length, deps.keyToTxnId, deps.keyToTxnId.length,
-                    keyBuffers, this, this, this
-            );
-            if (buf == deps.keyToTxnId)
-            {
-                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;
-            }
+            super();
         }
 
-        Deps get()
+        public Deps build()
         {
-            if (buf == null)
-                return NONE;
-
-            if (from != null)
-                return from;
-
-            return new Deps(
-                    Keys.ofSortedUnchecked(keyBuffers.realComplete(bufKeys, bufKeysLength)),
-                    realComplete(bufTxnIds, bufTxnIdsLength),
-                    realComplete(buf, bufLength));
-        }
-
-        /**
-         * Free any buffers we no longer need
-         */
-        void discard()
-        {
-            if (from == null)
-                discard(null, null, null);
-        }
-
-        /**
-         * Free buffers unless they are equal to the corresponding parameter
-         */
-        void discard(Key[] freeKeysIfNot, TxnId[] freeTxnIdsIfNot, int[] freeBufIfNot)
-        {
-            if (from != null)
-                return;
-
-            if (bufKeys != freeKeysIfNot)
-            {
-                keyBuffers.realDiscard(bufKeys, bufKeysLength);
-                bufKeys = null;
-            }
-            if (bufTxnIds != freeTxnIdsIfNot)
-            {
-                realDiscard(bufTxnIds, bufTxnIdsLength);
-                bufTxnIds = null;
-            }
-            if (buf != freeBufIfNot)
-            {
-                realDiscard(buf, bufLength);
-                buf = null;
-            }
+            return new Deps(keyBuilder.build(), rangeBuilder == null ? RangeDeps.NONE : rangeBuilder.build());
         }
     }
 
-    public static <T> Deps merge(List<T> merge, Function<T, Deps> getter)
-    {
-        LinearMerger linearMerger = new LinearMerger();
-        try
-        {
-            int mergeIndex = 0, mergeSize = merge.size();
-            while (mergeIndex < mergeSize)
-            {
-                Deps deps = getter.apply(merge.get(mergeIndex++));
-                if (deps == null || deps.isEmpty())
-                    continue;
-
-                linearMerger.update(deps);
-            }
-
-            return linearMerger.get();
-        }
-        finally
-        {
-            linearMerger.discard();
-        }
-    }
-
-    final Keys keys; // unique Keys
-    final TxnId[] txnIds; // unique TxnId TODO (low priority, efficiency): this could be a BTree?
+    public final KeyDeps keyDeps;
+    public final RangeDeps rangeDeps;
 
-    /**
-     * This represents a map of {@code Key -> [TxnId] } where each TxnId is actually a pointer into the txnIds array.
-     * The beginning of the array (the first keys.size() entries) are offsets into this array.
-     * <p/>
-     * Example:
-     * <p/>
-     * {@code
-     *   int keyIdx = keys.indexOf(key);
-     *   int startOfTxnOffset = keyIdx == 0 ? keys.size() : keyToTxnId[keyIdx - 1];
-     *   int endOfTxnOffset = keyToTxnId[keyIdx];
-     *   for (int i = startOfTxnOffset; i < endOfTxnOffset; i++)
-     *   {
-     *       TxnId id = txnIds[keyToTxnId[i]]
-     *       ...
-     *   }
-     * }
-     */
-    final int[] keyToTxnId; // Key -> [TxnId]
-    // Lazy loaded in ensureTxnIdToKey()
-    int[] txnIdToKey; // TxnId -> [Key]
-
-    Deps(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
+    public Deps(KeyDeps keyDeps, RangeDeps rangeDeps)
     {
-        this.keys = keys;
-        this.txnIds = txnIds;
-        this.keyToTxnId = keyToTxnId;
-        if (!(keys.isEmpty() || keyToTxnId[keys.size() - 1] == keyToTxnId.length))
-            throw new IllegalArgumentException(String.format("Last key (%s) in keyToTxnId does not point (%d) to the end of the array (%d);\nkeyToTxnId=%s", keys.get(keys.size() - 1), keyToTxnId[keys.size() - 1], keyToTxnId.length, Arrays.toString(keyToTxnId)));
-        if (DEBUG_CHECKS)
-            checkValid();
+        this.keyDeps = keyDeps;
+        this.rangeDeps = rangeDeps;
     }
 
-    public PartialDeps slice(Ranges ranges)
-    {
-        if (isEmpty())
-            return new PartialDeps(ranges, keys, txnIds, keyToTxnId);
-
-        Keys select = keys.slice(ranges);
-
-        if (select.isEmpty())
-            return new PartialDeps(ranges, Keys.EMPTY, NO_TXNIDS, NO_INTS);
-
-        if (select.size() == keys.size())
-            return new PartialDeps(ranges, keys, txnIds, keyToTxnId);
-
-        int i = 0;
-        int offset = select.size();
-        for (int j = 0 ; j < select.size() ; ++j)
-        {
-            int findi = keys.findNext(i, select.get(j), FAST);
-            if (findi < 0)
-                continue;
-
-            i = findi;
-            offset += keyToTxnId[i] - (i == 0 ? keys.size() : keyToTxnId[i - 1]);
-        }
-
-        int[] src = keyToTxnId;
-        int[] trg = new int[offset];
-
-        i = 0;
-        offset = select.size();
-        for (int j = 0 ; j < select.size() ; ++j)
-        {
-            int findi = keys.findNext(i, select.get(j), FAST);
-            if (findi >= 0)
-            {
-                i = findi;
-                int start = i == 0 ? keys.size() : src[i - 1];
-                int count = src[i] - start;
-                System.arraycopy(src, start, trg, offset, count);
-                offset += count;
-            }
-            trg[j] = offset;
-        }
-
-        TxnId[] txnIds = trimUnusedTxnId(select, this.txnIds, trg);
-        return new PartialDeps(ranges, select, txnIds, trg);
-    }
-
-    /**
-     * Returns the set of {@link TxnId}s that are referenced by {@code keysToTxnId}, and <strong>updates</strong>
-     * {@code keysToTxnId} to point to the new offsets in the returned set.
-     * @param keys object referenced by {@code keysToTxnId} index
-     * @param txnIds to trim to the seen {@link TxnId}s
-     * @param keysToTxnId to use as reference for trimming, this index will be updated to reflect the trimmed offsets.
-     * @return smallest set of {@link TxnId} seen in {@code keysToTxnId}
-     */
-    private static TxnId[] trimUnusedTxnId(Keys keys, TxnId[] txnIds, int[] keysToTxnId)
+    public boolean contains(TxnId txnId)
     {
-        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
-        {
-            Arrays.fill(remapTxnId, bitMapOffset, txnIds.length + 1, 0);
-            for (int i = keys.size() ; i < keysToTxnId.length ; ++i)
-                setBit(remapTxnId, bitMapOffset, keysToTxnId[i]);
-
-            int offset = 0;
-            for (int i = 0 ; i < txnIds.length ; ++i)
-            {
-                if (hasSetBit(remapTxnId, bitMapOffset, i)) remapTxnId[i] = offset++;
-                else remapTxnId[i] = -1;
-            }
-
-            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);
-        }
+        return keyDeps.contains(txnId) || rangeDeps.contains(txnId);
     }
 
     public Deps with(Deps that)
     {
-        if (isEmpty() || that.isEmpty())
-            return isEmpty() ? that : this;
-
-        return linearUnion(
-                this.keys.keys, this.keys.keys.length, this.txnIds, this.txnIds.length, this.keyToTxnId, this.keyToTxnId.length,
-                that.keys.keys, that.keys.keys.length, that.txnIds, that.txnIds.length, that.keyToTxnId, that.keyToTxnId.length,
-                cachedKeys(), cachedTxnIds(), cachedInts(),
-                (keys, keysLength, txnIds, txnIdsLength, out, outLength) ->
-                        new Deps(Keys.ofSortedUnchecked(cachedKeys().complete(keys, keysLength)),
-                                cachedTxnIds().complete(txnIds, txnIdsLength),
-                                cachedInts().complete(out, outLength))
-                );
-    }
-
-    /**
-     * Turn a set of key, value and mapping buffers into a merge result;
-     * K and V are either Key and TxnId, or vice versa, depending on which mapping direction was present
-     */
-    interface DepsConstructor<K, V, T>
-    {
-        T construct(K[] keys, int keysLength, V[] values, int valuesLength, int[] out, int outLength);
-    }
-
-    private static boolean arraysEqual(int[] left, int[] right, int length)
-    {
-        if (left.length < length || right.length < length)
-            return false;
-
-        for (int i=0; i<length; i++)
-            if (left[i] !=right[i])
-                return false;
-
-        return true;
-    }
-
-    private static <T> boolean arraysEqual(T[] left, T[] right, int length)
-    {
-        if (left.length < length || right.length < length)
-            return false;
-
-        for (int i=0; i<length; i++)
-            if (!Objects.equals(left[i], right[i]))
-                return false;
-
-        return true;
-    }
-
-    // TODO (low priority, efficiency): this method supports merging keyToTxnId OR txnIdToKey; we can perhaps save time
-    //  and effort when constructing Deps on remote hosts by only producing txnIdToKey with OrderedCollector and serializing
-    //  only this, and merging on the recipient before inverting, so that we only have to invert the final assembled deps
-    private static <K extends Comparable<? super K>, V extends Comparable<? super V>, T>
-    T linearUnion(K[] leftKeys, int leftKeysLength, V[] leftValues, int leftValuesLength, int[] left, int leftLength,
-                  K[] rightKeys, int rightKeysLength, V[] rightValues, int rightValuesLength, int[] right, int rightLength,
-                  ObjectBuffers<K> keyBuffers, ObjectBuffers<V> valueBuffers, IntBuffers intBuffers, DepsConstructor<K, V, T> constructor)
-    {
-        K[] outKeys = null;
-        V[] outValues = null;
-        int[] remapLeft = null, remapRight = null, out = null;
-        int outLength = 0, outKeysLength = 0, outTxnIdsLength = 0;
-
-        try
-        {
-            outKeys = SortedArrays.linearUnion(leftKeys, leftKeysLength, rightKeys, rightKeysLength, keyBuffers);
-            outKeysLength = keyBuffers.lengthOfLast(outKeys);
-            outValues = SortedArrays.linearUnion(leftValues, leftValuesLength, rightValues, rightValuesLength, valueBuffers);
-            outTxnIdsLength = valueBuffers.lengthOfLast(outValues);
-
-            remapLeft = remapToSuperset(leftValues, leftValuesLength, outValues, outTxnIdsLength, intBuffers);
-            remapRight = remapToSuperset(rightValues, rightValuesLength, outValues, outTxnIdsLength, intBuffers);
-
-            if (remapLeft == null && remapRight == null && leftLength == rightLength && leftKeysLength == rightKeysLength
-                    && arraysEqual(left, right, rightLength)
-                    && arraysEqual(leftKeys, rightKeys, rightKeysLength)
-                )
-            {
-                return constructor.construct(leftKeys, leftKeysLength, leftValues, leftValuesLength, left, leftLength);
-            }
-
-            int lk = 0, rk = 0, ok = 0, l = leftKeysLength, r = rightKeysLength;
-            outLength = outKeysLength;
-
-            if (remapLeft == null && outKeys == leftKeys)
-            {
-                // "this" knows all the TxnId and Keys already, but do both agree on what Keys map to TxnIds?
-                noOp: while (lk < leftKeysLength && rk < rightKeysLength)
-                {
-                    int ck = leftKeys[lk].compareTo(rightKeys[rk]);
-                    if (ck < 0)
-                    {
-                        // "this" knows of a key not present in "that"
-                        outLength += left[lk] - l; // logically append the key's TxnIds to the size
-                        l = left[lk];
-                        assert outLength == l && ok == lk && left[ok] == outLength;
-                        ok++;
-                        lk++;
-                    }
-                    else if (ck > 0)
-                    {
-                        // if this happened there is a bug with keys.union or keys are not actually sorted
-                        throwUnexpectedMissingKeyException(leftKeys, lk, leftKeysLength, rightKeys, rk, rightKeysLength, true);
-                    }
-                    else
-                    {
-                        // both "this" and "that" know of the key
-                        while (l < left[lk] && r < right[rk])
-                        {
-                            int nextLeft = left[l];
-                            int nextRight = remap(right[r], remapRight);
-
-                            if (nextLeft < nextRight)
-                            {
-                                // "this" knows of the txn that "that" didn't
-                                outLength++;
-                                l++;
-                            }
-                            else if (nextRight < nextLeft)
-                            {
-                                out = copy(left, outLength, leftLength + rightLength - r, intBuffers);
-                                break noOp;
-                            }
-                            else
-                            {
-                                outLength++;
-                                l++;
-                                r++;
-                            }
-                        }
-
-                        if (l < left[lk])
-                        {
-                            outLength += left[lk] - l;
-                            l = left[lk];
-                        }
-                        else if (r < right[rk])
-                        {
-                            // "that" thinks a key includes a TxnId as a dependency but "this" doesn't, need to include this knowledge
-                            out = copy(left, outLength, leftLength + rightLength - r, intBuffers);
-                            break;
-                        }
-
-                        assert outLength == l && ok == lk && left[ok] == outLength;
-                        ok++;
-                        rk++;
-                        lk++;
-                    }
-                }
-
-                if (out == null)
-                    return constructor.construct(leftKeys, leftKeysLength, leftValues, leftValuesLength, left, leftLength);
-            }
-            else if (remapRight == null && outKeys == rightKeys)
-            {
-                // "that" knows all the TxnId and keys already, but "this" does not
-                noOp: while (lk < leftKeysLength && rk < rightKeysLength)
-                {
-                    int ck = leftKeys[lk].compareTo(rightKeys[rk]);
-                    if (ck < 0)
-                    {
-                        // if this happened there is a bug with keys.union or keys are not actually sorted
-                        throwUnexpectedMissingKeyException(leftKeys, lk, leftKeysLength, rightKeys, rk, rightKeysLength, false);
-                    }
-                    else if (ck > 0)
-                    {
-                        outLength += right[rk] - r;
-                        r = right[rk];
-                        assert outLength == r && ok == rk && right[ok] == outLength;
-                        ok++;
-                        rk++;
-                    }
-                    else
-                    {
-                        // both "this" and "that" know of the key
-                        while (l < left[lk] && r < right[rk])
-                        {
-                            int nextLeft = remap(left[l], remapLeft);
-                            int nextRight = right[r];
-
-                            if (nextLeft < nextRight)
-                            {
-                                // "this" thinks a TxnID depends on Key but "that" doesn't, need to include this knowledge
-                                out = copy(right, outLength, rightLength + leftLength - l, intBuffers);
-                                break noOp;
-                            }
-                            else if (nextRight < nextLeft)
-                            {
-                                // "that" knows of the txn that "this" didn't
-                                outLength++;
-                                r++;
-                            }
-                            else
-                            {
-                                outLength++;
-                                l++;
-                                r++;
-                            }
-                        }
-
-                        if (l < left[lk])
-                        {
-                            out = copy(right, outLength, rightLength + leftLength - l, intBuffers);
-                            break;
-                        }
-                        else if (r < right[rk])
-                        {
-                            outLength += right[rk] - r;
-                            r = right[rk];
-                        }
-
-                        assert outLength == r && ok == rk && right[ok] == outLength;
-                        ok++;
-                        rk++;
-                        lk++;
-                    }
-                }
-
-                if (out == null)
-                    return constructor.construct(rightKeys, rightKeysLength, rightValues, rightValuesLength, right, rightLength);
-            }
-            else
-            {
-                out = intBuffers.getInts(leftLength + rightLength);
-            }
-
-            while (lk < leftKeysLength && rk < rightKeysLength)
-            {
-                int ck = leftKeys[lk].compareTo(rightKeys[rk]);
-                if (ck < 0)
-                {
-                    while (l < left[lk])
-                        out[outLength++] = remap(left[l++], remapLeft);
-                    out[ok++] = outLength;
-                    lk++;
-                }
-                else if (ck > 0)
-                {
-                    while (r < right[rk])
-                        out[outLength++] = remap(right[r++], remapRight);
-                    out[ok++] = outLength;
-                    rk++;
-                }
-                else
-                {
-                    while (l < left[lk] && r < right[rk])
-                    {
-                        int nextLeft = remap(left[l], remapLeft);
-                        int nextRight = remap(right[r], remapRight);
-
-                        if (nextLeft <= nextRight)
-                        {
-                            out[outLength++] = nextLeft;
-                            l += 1;
-                            r += nextLeft == nextRight ? 1 : 0;
-                        }
-                        else
-                        {
-                            out[outLength++] = nextRight;
-                            ++r;
-                        }
-                    }
-
-                    while (l < left[lk])
-                        out[outLength++] = remap(left[l++], remapLeft);
-
-                    while (r < right[rk])
-                        out[outLength++] = remap(right[r++], remapRight);
-
-                    out[ok++] = outLength;
-                    rk++;
-                    lk++;
-                }
-            }
-
-            while (lk < leftKeysLength)
-            {
-                while (l < left[lk])
-                    out[outLength++] = remap(left[l++], remapLeft);
-                out[ok++] = outLength;
-                lk++;
-            }
-
-            while (rk < rightKeysLength)
-            {
-                while (r < right[rk])
-                    out[outLength++] = remap(right[r++], remapRight);
-                out[ok++] = outLength;
-                rk++;
-            }
-
-            return constructor.construct(outKeys, outKeysLength, outValues, outTxnIdsLength, out, outLength);
-        }
-        finally
-        {
-            if (outKeys != null)
-                keyBuffers.discard(outKeys, outKeysLength);
-            if (outValues != null)
-                valueBuffers.discard(outValues, outTxnIdsLength);
-            if (out != null)
-                intBuffers.discard(out, outLength);
-            if (remapLeft != null)
-                intBuffers.forceDiscard(remapLeft, leftValuesLength);
-            if (remapRight != null)
-                intBuffers.forceDiscard(remapRight, rightValuesLength);
-        }
-    }
-
-    private static <A> void throwUnexpectedMissingKeyException(A[] leftKeys, int leftKeyIndex, int leftKeyLength, A[] rightKeys, int rightKeyIndex, int rightKeyLength, boolean isMissingLeft)
-    {
-        StringBuilder sb = new StringBuilder();
-        String missing = isMissingLeft ? "left" : "right";
-        String extra = isMissingLeft ? "right" : "left";
-        sb.append(missing).append(" knows all keys, yet ").append(extra).append(" knew of an extra key at indexes left[")
-                .append(leftKeyIndex).append("] = ").append(leftKeys[leftKeyIndex])
-                .append(", right[").append(rightKeyIndex).append("] = ").append(rightKeys[rightKeyIndex]).append("\n");
-        sb.append("leftKeys = ").append(Arrays.stream(leftKeys, 0, leftKeyLength).map(Object::toString).collect(Collectors.joining())).append('\n');
-        sb.append("rightKeys = ").append(Arrays.stream(rightKeys, 0, rightKeyLength).map(Object::toString).collect(Collectors.joining())).append('\n');
-        throw new IllegalStateException(sb.toString());
-    }
-
-    private static int[] copy(int[] src, int to, int length, IntBuffers bufferManager)
-    {
-        if (length == 0)
-            return NO_INTS;
-
-        int[] result = bufferManager.getInts(length);
-        if (result.length < length)
-            throw new IllegalStateException();
-        System.arraycopy(src, 0, result, 0, to);
-        return result;
+        return new Deps(this.keyDeps.with(that.keyDeps), this.rangeDeps.with(that.rangeDeps));
     }
 
     public Deps without(Predicate<TxnId> remove)
     {
-        if (isEmpty())
-            return this;
-
-        IntBuffers cache = ArrayBuffers.cachedInts();
-        TxnId[] oldTxnIds = txnIds;
-        int[] oldKeyToTxnId = keyToTxnId;
-        int[] remapTxnIds = cache.getInts(oldTxnIds.length);
-        int[] newKeyToTxnId = null;
-        TxnId[] newTxnIds;
-        int o = 0;
-        try
-        {
-            int count = 0;
-            for (int i = 0 ; i < oldTxnIds.length ; ++i)
-            {
-                if (remove.test(oldTxnIds[i])) remapTxnIds[i] = -1;
-                else remapTxnIds[i] = count++;
-            }
-
-            if (count == oldTxnIds.length)
-                return this;
-
-            if (count == 0)
-                return NONE;
-
-            newTxnIds = new TxnId[count];
-            for (int i = 0 ; i < oldTxnIds.length ; ++i)
-            {
-                if (remapTxnIds[i] >= 0)
-                    newTxnIds[remapTxnIds[i]] = oldTxnIds[i];
-            }
-
-            newKeyToTxnId = cache.getInts(oldKeyToTxnId.length);
-            int k = 0, i = keys.size();
-            o = i;
-            while (i < oldKeyToTxnId.length)
-            {
-                while (oldKeyToTxnId[k] == i)
-                    newKeyToTxnId[k++] = o;
-
-                int remapped = remapTxnIds[oldKeyToTxnId[i]];
-                if (remapped >= 0)
-                    newKeyToTxnId[o++] = remapped;
-                ++i;
-            }
-
-            while (k < keys.size())
-                newKeyToTxnId[k++] = o;
-        }
-        catch (Throwable t)
-        {
-            cache.forceDiscard(newKeyToTxnId, o);
-            throw t;
-        }
-        finally
-        {
-            cache.forceDiscard(remapTxnIds, oldTxnIds.length);
-        }
-
-        newKeyToTxnId = cache.completeAndDiscard(newKeyToTxnId, o);
-        return new Deps(keys, newTxnIds, newKeyToTxnId);
+        return new Deps(keyDeps.without(remove), rangeDeps.without(remove));
     }
 
-    public boolean contains(TxnId txnId)
+    public PartialDeps slice(Ranges covering)
     {
-        return Arrays.binarySearch(txnIds, txnId) >= 0;
+        return new PartialDeps(covering, keyDeps.slice(covering), rangeDeps.slice(covering));
     }
 
-    // return true iff we map any keys to any txnId
-    // if the mapping is empty we return false, whether or not we have any keys or txnId by themselves
     public boolean isEmpty()
     {
-        return keyToTxnId.length == keys.size();
-    }
-
-    public Keys someKeys(TxnId txnId)
-    {
-        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
-        if (txnIdIndex < 0)
-            return Keys.EMPTY;
-
-        ensureTxnIdToKey();
-
-        int start = txnIdIndex == 0 ? txnIds.length : txnIdToKey[txnIdIndex - 1];
-        int end = txnIdToKey[txnIdIndex];
-        if (start == end)
-            return Keys.EMPTY;
-
-        Key[] result = new Key[end - start];
-        for (int i = start ; i < end ; ++i)
-            result[i - start] = keys.get(txnIdToKey[i]);
-        return Keys.of(result);
-    }
-
-    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)
-            constructor.apply(RoutingKeys.EMPTY.keys);
-
-        ensureTxnIdToKey();
-
-        int start = txnIdIndex == 0 ? txnIds.length : txnIdToKey[txnIdIndex - 1];
-        int end = txnIdToKey[txnIdIndex];
-        RoutingKey[] result = new RoutingKey[end - start];
-        if (start == end)
-            constructor.apply(RoutingKeys.EMPTY.keys);
-
-        result[0] = keys.get(txnIdToKey[start]).toUnseekable();
-        int resultCount = 1;
-        for (int i = start + 1 ; i < end ; ++i)
-        {
-            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 constructor.apply(result);
-    }
-
-    void ensureTxnIdToKey()
-    {
-        if (txnIdToKey != null)
-            return;
-
-        txnIdToKey = invert(keyToTxnId, keyToTxnId.length, keys.size(), txnIds.length);
-    }
-
-    private static int[] invert(int[] src, int srcLength, int srcKeyCount, int trgKeyCount)
-    {
-        int[] trg = new int[trgKeyCount + srcLength - srcKeyCount];
-
-        // first pass, count number of txnId per key
-        for (int i = srcKeyCount ; i < srcLength ; ++i)
-            trg[src[i]]++;
-
-        // turn into offsets (i.e. add txnIds.size() and then sum them)
-        trg[0] += trgKeyCount;
-        for (int i = 1; i < trgKeyCount ; ++i)
-            trg[i] += trg[i - 1];
-
-        // shuffle forwards one, so we have the start index rather than end
-        System.arraycopy(trg, 0, trg, 1, trgKeyCount - 1);
-        trg[0] = trgKeyCount;
-
-        // convert the offsets to end, and set the key at the target positions
-        int k = 0;
-        for (int i = srcKeyCount ; i < srcLength ; ++i)
-        {
-            // if at the end offset, switch to the next key
-            while (i == src[k])
-                ++k;
-
-            // find the next key offset for the TxnId and set the offset to this key
-            trg[trg[src[i]]++] = k;
-        }
-
-        return trg;
-    }
-
-    public void forEachOn(Ranges ranges, Predicate<Key> include, BiConsumer<Key, TxnId> forEach)
-    {
-        Routables.foldl(keys, ranges, (key, value, index) -> {
-            if (!include.test(key))
-                return null;
-
-            for (int t = startOffset(index), end = endOffset(index); t < end ; ++t)
-            {
-                TxnId txnId = txnIds[keyToTxnId[t]];
-                forEach.accept(key, txnId);
-            }
-            return null;
-        }, null);
-    }
-
-    /**
-     * 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 forEach function to call on each unique {@link TxnId}
-     */
-    public void forEachOn(Ranges ranges, 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
-        // the property that forEach is called in TxnId order.
-        //TODO (expected, efficiency): reconsider this, probably not worth trying to save allocations at cost of multiple loop
-        //                             use BitSet, or perhaps extend so we can have no nested allocations when few bits
-        for (int offset = 0 ; offset < txnIds.length ; offset += 64)
-        {
-            long bitset = Routables.foldl(keys, ranges, (key, off, value, keyIndex) -> {
-                int index = startOffset(keyIndex);
-                int end = endOffset(keyIndex);
-                if (off > 0)
-                {
-                    // TODO (low priority, efficiency): interpolation search probably great here
-                    index = Arrays.binarySearch(keyToTxnId, index, end, (int)off);
-                    if (index < 0)
-                        index = -1 - index;
-                }
-
-                while (index < end)
-                {
-                    long next = keyToTxnId[index++] - off;
-                    if (next >= 64)
-                        break;
-                    value |= 1L << next;
-                }
-
-                return value;
-            }, offset, 0, -1L);
-
-            while (bitset != 0)
-            {
-                int i = Long.numberOfTrailingZeros(bitset);
-                TxnId txnId = txnIds[offset + i];
-                forEach.accept(txnId);
-                bitset ^= Long.lowestOneBit(bitset);
-            }
-        }
-    }
-
-    public void forEach(Key key, Consumer<TxnId> forEach)
-    {
-        int keyIndex = keys.indexOf(key);
-        if (keyIndex < 0)
-            return;
-
-        int index = startOffset(keyIndex);
-        int end = endOffset(keyIndex);
-        while (index < end)
-            forEach.accept(txnIds[keyToTxnId[index++]]);
-    }
-
-    public Keys keys()
-    {
-        return keys;
+        return keyDeps.isEmpty() && rangeDeps.isEmpty();
     }
 
     public int txnIdCount()
     {
-        return txnIds.length;
-    }
-
-    public int totalCount()
-    {
-        return keyToTxnId.length - keys.size();
+        return keyDeps.txnIdCount() + rangeDeps.txnIdCount();
     }
 
     public TxnId txnId(int i)
     {
-        return txnIds[i];
+        return i < keyDeps.txnIdCount()
+                ? keyDeps.txnId(i)
+                : rangeDeps.txnId(i - keyDeps.txnIdCount());
     }
 
-    public Collection<TxnId> txnIds()
+    public List<TxnId> txnIds()
     {
-        return Arrays.asList(txnIds);
-    }
-
-    public List<TxnId> txnIds(Key key)
-    {
-        int keyIndex = keys.indexOf(key);
-        if (keyIndex < 0)
-            return Collections.emptyList();
-
-        int start = startOffset(keyIndex);
-        int end = endOffset(keyIndex);
-        int size = end - start;
-
-        return new AbstractList<TxnId>()
-        {
+        final int txnIdCount = txnIdCount();
+        final int keyDepsCount = keyDeps.txnIdCount();
+        return new AbstractList<TxnId>() {
             @Override
             public TxnId get(int index)
             {
-                if (index > end)
-                    throw new IndexOutOfBoundsException();
-                return txnIds[keyToTxnId[start + index]];
+                return index < keyDepsCount
+                        ? keyDeps.txnId(index)
+                        : rangeDeps.txnId(index - keyDepsCount);
             }
 
             @Override
-            public int size()
-            {
-                return size;
+            public int size() {
+                return txnIdCount;
             }
         };
     }
 
-    private int startOffset(int keyIndex)
-    {
-        return keyIndex == 0 ? keys.size() : keyToTxnId[keyIndex - 1];
-    }
-
-    private int endOffset(int keyIndex)
+    public List<TxnId> txnIds(Seekable keyOrRange)
     {
-        return keyToTxnId[keyIndex];
-    }
-
-    @Override
-    public Iterator<Map.Entry<Key, TxnId>> iterator()
-    {
-        return new Iterator<Map.Entry<Key, TxnId>>()
+        List<TxnId> keyIds, rangeIds;
+        switch (keyOrRange.domain())
         {
-            int i = keys.size(), k = 0;
-
-            @Override
-            public boolean hasNext()
+            default: throw new AssertionError();
+            case Key:
             {
-                return i < keyToTxnId.length;
+                Key key = keyOrRange.asKey();
+                keyIds = keyDeps.txnIds(key);
+                rangeIds = rangeDeps.txnIds(key);
+                break;
             }
-
-            @Override
-            public Map.Entry<Key, TxnId> next()
+            case Range:
             {
-                Entry result = new Entry(keys.get(k), txnIds[keyToTxnId[i++]]);
-                if (i == keyToTxnId[k])
-                    ++k;
-                return result;
+                Range range = keyOrRange.asRange();
+                keyIds = keyDeps.txnIds(range);
+                rangeIds = rangeDeps.txnIds(range);
             }
-        };
-    }
-
-    @Override
-    public String toString()
-    {
-        return toSimpleString();
-    }
+        }
 
-    public String toSimpleString()
-    {
-        if (keys.isEmpty())
-            return "{}";
+        if (rangeIds.isEmpty()) return keyIds;
+        if (keyIds.isEmpty()) return rangeIds;
 
-        StringBuilder builder = new StringBuilder("{");
-        for (int k = 0, t = keys.size(); k < keys.size() ; ++k)
+        List<TxnId> output = new ArrayList<>();
+        int ki = 0, ri = 0;
+        while (ki < keyIds.size() && ri < rangeIds.size())
         {
-            if (builder.length() > 1)
-                builder.append(", ");
-
-            builder.append(keys.get(k));
-            builder.append(":[");
-            boolean first = true;
-            while (t < keyToTxnId[k])
-            {
-                if (first) first = false;
-                else builder.append(", ");
-                builder.append(txnIds[keyToTxnId[t++]]);
-            }
-            builder.append("]");
+            int c = keyIds.get(ki).compareTo(rangeIds.get(ri));
+            Invariants.checkState(c != 0);
+            if (c < 0) output.add(keyIds.get(ki++));
+            else output.add(rangeIds.get(ri++));
         }
-        builder.append("}");
-        return builder.toString();
+        while (ki < keyIds.size())
+            output.add(keyIds.get(ki++));
+        while (ri < rangeIds.size())
+            output.add(rangeIds.get(ri++));
+        return output;
     }
 
-    @Override
-    public boolean equals(Object o)
+    public Unseekables<?, ?> someUnseekables(TxnId txnId)
     {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        return equals((Deps) o);
+        if (keyDeps.contains(txnId))
+            return keyDeps.someUnseekables(txnId);
+        else
+            return rangeDeps.someUnseekables(txnId);
     }
 
-    public boolean equals(Deps that)
+    // NOTE: filter only applied to keyDeps
+    public void forEach(Ranges ranges, Consumer<TxnId> forEach)
     {
-        return this.txnIds.length == that.txnIds.length
-               && this.keys.size() == that.keys.size()
-               && Arrays.equals(this.keyToTxnId, that.keyToTxnId)
-               && Arrays.equals(this.txnIds, that.txnIds)
-               && this.keys.equals(that.keys);
+        keyDeps.forEachUniqueTxnId(ranges, forEach);
+        rangeDeps.forEachUniqueTxnId(ranges, forEach);
     }
 
-    public static class Entry implements Map.Entry<Key, TxnId>
+    public static <T> Deps merge(List<T> list, Function<T, Deps> getter)
     {
-        final Key key;
-        final TxnId txnId;
-
-        public Entry(Key key, TxnId txnId)
-        {
-            this.key = key;
-            this.txnId = txnId;
-        }
-
-        @Override
-        public Key getKey()
-        {
-            return key;
-        }
-
-        @Override
-        public TxnId getValue()
-        {
-            return txnId;
-        }
-
-        @Override
-        public TxnId setValue(TxnId value)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public String toString()
-        {
-            return key + "->" + txnId;
-        }
+        return new Deps(KeyDeps.merge(list, getter, deps -> deps.keyDeps),
+                        RangeDeps.merge(list, getter, deps -> deps.rangeDeps));
     }
 
-    private void checkValid()
+    @Override
+    public String toString()
     {
-        int k = 0;
-        for (int i = keys.size() ; i < keyToTxnId.length ; ++i)
-        {
-            boolean first = true;
-            while (i < keyToTxnId[k])
-            {
-                if (first) first = false;
-                else if (keyToTxnId[i - 1] == keyToTxnId[i])
-                {
-                    Key key = keys.get(i);
-                    TxnId txnId = txnIds[keyToTxnId[i]];
-                    throw new IllegalStateException(String.format("Duplicate TxnId (%s) found for key %s", txnId, key));
-                }
-                i++;
-            }
-            ++k;
-        }
+        return keyDeps.toString() + ", " + rangeDeps.toString();
     }
 
-    private static void setBit(int[] array, int offset, int index)
+    public boolean equals(Object that)

Review Comment:
   Missing `@Override`?



##########
accord-core/src/main/java/accord/primitives/Deps.java:
##########
@@ -18,1311 +18,220 @@
 
 package accord.primitives;
 
-import java.util.*;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-import java.util.function.Function;
-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 accord.utils.Invariants;
 
-import static accord.utils.ArrayBuffers.*;
-import static accord.utils.SortedArrays.*;
-import static accord.utils.SortedArrays.Search.FAST;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
 
 /**
- * A collection of dependencies for a transaction, organised by the key the dependency is adopted via.
- * An inverse map from TxnId to Key may also be constructed and stored in this collection.
+ * A collection of transaction dependencies, keyed by the key or range on which they were adopted
  */
-// TODO (desired, consider): switch to RoutingKey? Would mean adopting execution dependencies less precisely, but saving ser/deser of large keys
-public class Deps implements Iterable<Map.Entry<Key, TxnId>>
+public class Deps
 {
-    private static final boolean DEBUG_CHECKS = true;
-
-    private static final TxnId[] NO_TXNIDS = new TxnId[0];
-    private static final int[] NO_INTS = new int[0];
-    public static final Deps NONE = new Deps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+    public static final Deps NONE = new Deps(KeyDeps.NONE, RangeDeps.NONE);
 
-    public static class SerializerSupport
+    public static Builder builder()
     {
-        private SerializerSupport() {}
-
-        public static int keyToTxnIdCount(Deps deps)
-        {
-            return deps.keyToTxnId.length;
-        }
-
-        public static int keyToTxnId(Deps deps, int idx)
-        {
-            return deps.keyToTxnId[idx];
-        }
-
-        public static Deps create(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
-        {
-            return new Deps(keys, txnIds, keyToTxnId);
-        }
-    }
-
-    public static Deps none(Keys keys)
-    {
-        int[] keysToTxnId = new int[keys.size()];
-        Arrays.fill(keysToTxnId, keys.size());
-        return new Deps(keys, NO_TXNIDS, keysToTxnId);
-    }
-
-    /**
-     * Expects Command to be provided in TxnId order
-     */
-    public static OrderedBuilder orderedBuilder(boolean hasOrderedTxnId)
-    {
-        return new OrderedBuilder(hasOrderedTxnId);
+        return new Builder();
     }
 
     // TODO (expected, efficiency): cache this object per thread
-    public static abstract class AbstractOrderedBuilder<T extends Deps> implements AutoCloseable
+    public static abstract class AbstractBuilder<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;
-        // txnId -> Offset
-        TxnId[] keyToTxnId;
-        int keyCount;
-        int keyOffset;
-        int totalCount;
-
-        public AbstractOrderedBuilder(boolean hasOrderedTxnId)
-        {
-            this.keys = cachedKeys.get(16);
-            this.keyLimits = cachedInts.getInts(keys.length);
-            this.hasOrderedTxnId = hasOrderedTxnId;
-            this.keyToTxnId = cachedTxnIds.get(16);
-        }
-
-        public boolean isEmpty()
-        {
-            return totalCount() == 0;
-        }
-
-        private int totalCount()
-        {
-            return totalCount;
-        }
+        final KeyDeps.Builder keyBuilder;
+        RangeDeps.Builder rangeBuilder;
 
-        public void nextKey(Key key)
+        AbstractBuilder()
         {
-            if (keyCount > 0 && keys[keyCount - 1].compareTo(key) >= 0)
-            {
-                throw new IllegalArgumentException("Key " + key + " has already been visited or was provided out of order ("
-                        + Arrays.toString(Arrays.copyOf(keys, keyCount)) + ")");
-            }
-
-            finishKey();
-
-            if (keyCount == keys.length)
-            {
-                Key[] newKeys = cachedKeys.get(keyCount * 2);
-                System.arraycopy(keys, 0, newKeys, 0, keyCount);
-                cachedKeys.forceDiscard(keys, keyCount);
-                keys = newKeys;
-                int[] newKeyLimits = cachedInts.getInts(keyCount * 2);
-                System.arraycopy(keyLimits, 0, newKeyLimits, 0, keyCount);
-                cachedInts.forceDiscard(keyLimits, keyCount);
-                keyLimits = newKeyLimits;
-            }
-            keys[keyCount++] = key;
+            this.keyBuilder = KeyDeps.builder();
         }
 
-        private void finishKey()
+        public AbstractBuilder<T> add(Seekable keyOrRange, TxnId txnId)
         {
-            if (totalCount == keyOffset && keyCount > 0)
-                --keyCount; // remove this key; no data
-
-            if (keyCount == 0)
-                return;
-
-            if (totalCount != keyOffset && !hasOrderedTxnId)
+            switch (keyOrRange.domain())
             {
-                // TODO (low priority, efficiency): this allocates a significant amount of memory: would be preferable to be able to sort using a pre-defined scratch buffer
-                Arrays.sort(keyToTxnId, keyOffset, totalCount);
-                for (int i = keyOffset + 1 ; i < totalCount ; ++i)
-                {
-                    if (keyToTxnId[i - 1].equals(keyToTxnId[i]))
-                        throw new IllegalArgumentException("TxnId for " + keys[keyCount - 1] + " are not unique: " + Arrays.asList(keyToTxnId).subList(keyOffset, totalCount));
-                }
+                default: throw new AssertionError();
+                case Key:
+                    keyBuilder.add(keyOrRange.asKey(), txnId);
+                    break;
+                case Range:
+                    if (rangeBuilder == null)
+                        rangeBuilder = RangeDeps.builder();
+                    rangeBuilder.add(keyOrRange.asRange(), txnId);
+                    break;
             }
-
-            keyLimits[keyCount - 1] = totalCount;
-            keyOffset = totalCount;
-        }
-
-        public void add(Key key, TxnId txnId)
-        {
-            if (keyCount == 0 || !keys[keyCount - 1].equals(key))
-                nextKey(key);
-            add(txnId);
-        }
-
-        /**
-         * Add this command as a dependency for each intersecting key
-         */
-        public void add(TxnId txnId)
-        {
-            if (hasOrderedTxnId && totalCount > keyOffset && keyToTxnId[totalCount - 1].compareTo(txnId) >= 0)
-                throw new IllegalArgumentException("TxnId provided out of order");
-
-            if (totalCount >= keyToTxnId.length)
-            {
-                TxnId[] newTxnIds = cachedTxnIds.get(keyToTxnId.length * 2);
-                System.arraycopy(keyToTxnId, 0, newTxnIds, 0, totalCount);
-                cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
-                keyToTxnId = newTxnIds;
-            }
-
-            keyToTxnId[totalCount++] = txnId;
-        }
-
-        public T build()
-        {
-            if (totalCount == 0)
-                return build(Keys.EMPTY, NO_TXNIDS, NO_INTS);
-
-            finishKey();
-
-            TxnId[] uniqueTxnId = cachedTxnIds.get(totalCount);
-            System.arraycopy(keyToTxnId, 0, uniqueTxnId, 0, totalCount);
-            Arrays.sort(uniqueTxnId, 0, totalCount);
-            int txnIdCount = 1;
-            for (int i = 1 ; i < totalCount ; ++i)
-            {
-                if (!uniqueTxnId[txnIdCount - 1].equals(uniqueTxnId[i]))
-                    uniqueTxnId[txnIdCount++] = uniqueTxnId[i];
-            }
-
-            TxnId[] txnIds = cachedTxnIds.complete(uniqueTxnId, txnIdCount);
-            cachedTxnIds.discard(uniqueTxnId, totalCount);
-
-            int[] result = new int[keyCount + totalCount];
-            int offset = keyCount;
-            for (int k = 0 ; k < keyCount ; ++k)
-            {
-                result[k] = keyCount + keyLimits[k];
-                int from = k == 0 ? 0 : keyLimits[k - 1];
-                int to = keyLimits[k];
-                offset = (int)SortedArrays.foldlIntersection(txnIds, 0, txnIdCount, keyToTxnId, from, to, (key, p, v, li, ri) -> {
-                    result[(int)v] = li;
-                    return v + 1;
-                }, keyCount, offset, -1);
-            }
-
-            return build(Keys.ofSortedUnchecked(cachedKeys.complete(keys, keyCount)), txnIds, result);
+            return this;
         }
 
-        abstract T build(Keys keys, TxnId[] txnIds, int[] keyToTxnId);
+        public abstract T build();
 
         @Override
         public void close()
         {
-            cachedKeys.discard(keys, keyCount);
-            cachedInts.forceDiscard(keyLimits, keyCount);
-            cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
-        }
-    }
-
-    public static class OrderedBuilder extends AbstractOrderedBuilder<Deps>
-    {
-        public OrderedBuilder(boolean hasOrderedTxnId)
-        {
-            super(hasOrderedTxnId);
-        }
-
-        @Override
-        Deps build(Keys keys, TxnId[] txnIds, int[] keysToTxnIds)
-        {
-            return new Deps(keys, txnIds, keysToTxnIds);
+            keyBuilder.close();
+            if (rangeBuilder != null)
+                rangeBuilder.close();
         }
     }
 
-    /**
-     * An object for managing a sequence of efficient linear merges Deps objects.
-     * Its primary purpose is to manage input and output buffers, so that we reuse output buffers
-     * as input to the next merge, and if any input is a superset of the other inputs that this input
-     * is returned unmodified.
-     *
-     * This is achieved by using PassThroughXBuffers so that the result buffers (and their sizes) are returned
-     * unmodified, and the buffers are cached as far as possible. In general, the buffers should be taken
-     * out of pre-existing caches, but if the buffers are too large then we cache any additional buffers we
-     * allocate for the duration of the merge.
-     */
-    private static class LinearMerger extends PassThroughObjectAndIntBuffers<TxnId> implements DepsConstructor<Key, TxnId, Object>
+    public static class Builder extends AbstractBuilder<Deps>
     {
-        final PassThroughObjectBuffers<Key> keyBuffers;
-        Key[] bufKeys;
-        TxnId[] bufTxnIds;
-        int[] buf = null;
-        int bufKeysLength, bufTxnIdsLength = 0, bufLength = 0;
-        Deps from = null;
-
-        LinearMerger()
-        {
-            super(cachedTxnIds(), cachedInts());
-            keyBuffers = new PassThroughObjectBuffers<>(cachedKeys());
-        }
-
-        @Override
-        public Object construct(Key[] keys, int keysLength, TxnId[] txnIds, int txnIdsLength, int[] out, int outLength)
-        {
-            if (from == null)
-            {
-                // if our input buffers were themselves buffers, we want to discard them unless they have been returned back to us
-                discard(keys, txnIds, out);
-            }
-            else if (buf != out)
-            {
-                // the output is not equal to a prior input
-                from = null;
-            }
-
-            if (from == null)
-            {
-                bufKeys = keys;
-                bufKeysLength = keysLength;
-                bufTxnIds = txnIds;
-                bufTxnIdsLength = txnIdsLength;
-                buf = out;
-                bufLength = outLength;
-            }
-            else
-            {
-                Invariants.checkState(keys == bufKeys && keysLength == bufKeysLength);
-                Invariants.checkState(txnIds == bufTxnIds && txnIdsLength == bufTxnIdsLength);
-                Invariants.checkState(outLength == bufLength);
-            }
-            return null;
-        }
-
-        void update(Deps deps)
+        public Builder()
         {
-            if (buf == null)
-            {
-                bufKeys = deps.keys.keys;
-                bufKeysLength = deps.keys.keys.length;
-                bufTxnIds = deps.txnIds;
-                bufTxnIdsLength = deps.txnIds.length;
-                buf = deps.keyToTxnId;
-                bufLength = deps.keyToTxnId.length;
-                from = deps;
-                return;
-            }
-
-            linearUnion(
-                    bufKeys, bufKeysLength, bufTxnIds, bufTxnIdsLength, buf, bufLength,
-                    deps.keys.keys, deps.keys.keys.length, deps.txnIds, deps.txnIds.length, deps.keyToTxnId, deps.keyToTxnId.length,
-                    keyBuffers, this, this, this
-            );
-            if (buf == deps.keyToTxnId)
-            {
-                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;
-            }
+            super();
         }
 
-        Deps get()
+        public Deps build()
         {
-            if (buf == null)
-                return NONE;
-
-            if (from != null)
-                return from;
-
-            return new Deps(
-                    Keys.ofSortedUnchecked(keyBuffers.realComplete(bufKeys, bufKeysLength)),
-                    realComplete(bufTxnIds, bufTxnIdsLength),
-                    realComplete(buf, bufLength));
-        }
-
-        /**
-         * Free any buffers we no longer need
-         */
-        void discard()
-        {
-            if (from == null)
-                discard(null, null, null);
-        }
-
-        /**
-         * Free buffers unless they are equal to the corresponding parameter
-         */
-        void discard(Key[] freeKeysIfNot, TxnId[] freeTxnIdsIfNot, int[] freeBufIfNot)
-        {
-            if (from != null)
-                return;
-
-            if (bufKeys != freeKeysIfNot)
-            {
-                keyBuffers.realDiscard(bufKeys, bufKeysLength);
-                bufKeys = null;
-            }
-            if (bufTxnIds != freeTxnIdsIfNot)
-            {
-                realDiscard(bufTxnIds, bufTxnIdsLength);
-                bufTxnIds = null;
-            }
-            if (buf != freeBufIfNot)
-            {
-                realDiscard(buf, bufLength);
-                buf = null;
-            }
+            return new Deps(keyBuilder.build(), rangeBuilder == null ? RangeDeps.NONE : rangeBuilder.build());
         }
     }
 
-    public static <T> Deps merge(List<T> merge, Function<T, Deps> getter)
-    {
-        LinearMerger linearMerger = new LinearMerger();
-        try
-        {
-            int mergeIndex = 0, mergeSize = merge.size();
-            while (mergeIndex < mergeSize)
-            {
-                Deps deps = getter.apply(merge.get(mergeIndex++));
-                if (deps == null || deps.isEmpty())
-                    continue;
-
-                linearMerger.update(deps);
-            }
-
-            return linearMerger.get();
-        }
-        finally
-        {
-            linearMerger.discard();
-        }
-    }
-
-    final Keys keys; // unique Keys
-    final TxnId[] txnIds; // unique TxnId TODO (low priority, efficiency): this could be a BTree?
+    public final KeyDeps keyDeps;
+    public final RangeDeps rangeDeps;
 
-    /**
-     * This represents a map of {@code Key -> [TxnId] } where each TxnId is actually a pointer into the txnIds array.
-     * The beginning of the array (the first keys.size() entries) are offsets into this array.
-     * <p/>
-     * Example:
-     * <p/>
-     * {@code
-     *   int keyIdx = keys.indexOf(key);
-     *   int startOfTxnOffset = keyIdx == 0 ? keys.size() : keyToTxnId[keyIdx - 1];
-     *   int endOfTxnOffset = keyToTxnId[keyIdx];
-     *   for (int i = startOfTxnOffset; i < endOfTxnOffset; i++)
-     *   {
-     *       TxnId id = txnIds[keyToTxnId[i]]
-     *       ...
-     *   }
-     * }
-     */
-    final int[] keyToTxnId; // Key -> [TxnId]
-    // Lazy loaded in ensureTxnIdToKey()
-    int[] txnIdToKey; // TxnId -> [Key]
-
-    Deps(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
+    public Deps(KeyDeps keyDeps, RangeDeps rangeDeps)
     {
-        this.keys = keys;
-        this.txnIds = txnIds;
-        this.keyToTxnId = keyToTxnId;
-        if (!(keys.isEmpty() || keyToTxnId[keys.size() - 1] == keyToTxnId.length))
-            throw new IllegalArgumentException(String.format("Last key (%s) in keyToTxnId does not point (%d) to the end of the array (%d);\nkeyToTxnId=%s", keys.get(keys.size() - 1), keyToTxnId[keys.size() - 1], keyToTxnId.length, Arrays.toString(keyToTxnId)));
-        if (DEBUG_CHECKS)
-            checkValid();
+        this.keyDeps = keyDeps;
+        this.rangeDeps = rangeDeps;
     }
 
-    public PartialDeps slice(Ranges ranges)
-    {
-        if (isEmpty())
-            return new PartialDeps(ranges, keys, txnIds, keyToTxnId);
-
-        Keys select = keys.slice(ranges);
-
-        if (select.isEmpty())
-            return new PartialDeps(ranges, Keys.EMPTY, NO_TXNIDS, NO_INTS);
-
-        if (select.size() == keys.size())
-            return new PartialDeps(ranges, keys, txnIds, keyToTxnId);
-
-        int i = 0;
-        int offset = select.size();
-        for (int j = 0 ; j < select.size() ; ++j)
-        {
-            int findi = keys.findNext(i, select.get(j), FAST);
-            if (findi < 0)
-                continue;
-
-            i = findi;
-            offset += keyToTxnId[i] - (i == 0 ? keys.size() : keyToTxnId[i - 1]);
-        }
-
-        int[] src = keyToTxnId;
-        int[] trg = new int[offset];
-
-        i = 0;
-        offset = select.size();
-        for (int j = 0 ; j < select.size() ; ++j)
-        {
-            int findi = keys.findNext(i, select.get(j), FAST);
-            if (findi >= 0)
-            {
-                i = findi;
-                int start = i == 0 ? keys.size() : src[i - 1];
-                int count = src[i] - start;
-                System.arraycopy(src, start, trg, offset, count);
-                offset += count;
-            }
-            trg[j] = offset;
-        }
-
-        TxnId[] txnIds = trimUnusedTxnId(select, this.txnIds, trg);
-        return new PartialDeps(ranges, select, txnIds, trg);
-    }
-
-    /**
-     * Returns the set of {@link TxnId}s that are referenced by {@code keysToTxnId}, and <strong>updates</strong>
-     * {@code keysToTxnId} to point to the new offsets in the returned set.
-     * @param keys object referenced by {@code keysToTxnId} index
-     * @param txnIds to trim to the seen {@link TxnId}s
-     * @param keysToTxnId to use as reference for trimming, this index will be updated to reflect the trimmed offsets.
-     * @return smallest set of {@link TxnId} seen in {@code keysToTxnId}
-     */
-    private static TxnId[] trimUnusedTxnId(Keys keys, TxnId[] txnIds, int[] keysToTxnId)
+    public boolean contains(TxnId txnId)
     {
-        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
-        {
-            Arrays.fill(remapTxnId, bitMapOffset, txnIds.length + 1, 0);
-            for (int i = keys.size() ; i < keysToTxnId.length ; ++i)
-                setBit(remapTxnId, bitMapOffset, keysToTxnId[i]);
-
-            int offset = 0;
-            for (int i = 0 ; i < txnIds.length ; ++i)
-            {
-                if (hasSetBit(remapTxnId, bitMapOffset, i)) remapTxnId[i] = offset++;
-                else remapTxnId[i] = -1;
-            }
-
-            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);
-        }
+        return keyDeps.contains(txnId) || rangeDeps.contains(txnId);
     }
 
     public Deps with(Deps that)
     {
-        if (isEmpty() || that.isEmpty())
-            return isEmpty() ? that : this;
-
-        return linearUnion(
-                this.keys.keys, this.keys.keys.length, this.txnIds, this.txnIds.length, this.keyToTxnId, this.keyToTxnId.length,
-                that.keys.keys, that.keys.keys.length, that.txnIds, that.txnIds.length, that.keyToTxnId, that.keyToTxnId.length,
-                cachedKeys(), cachedTxnIds(), cachedInts(),
-                (keys, keysLength, txnIds, txnIdsLength, out, outLength) ->
-                        new Deps(Keys.ofSortedUnchecked(cachedKeys().complete(keys, keysLength)),
-                                cachedTxnIds().complete(txnIds, txnIdsLength),
-                                cachedInts().complete(out, outLength))
-                );
-    }
-
-    /**
-     * Turn a set of key, value and mapping buffers into a merge result;
-     * K and V are either Key and TxnId, or vice versa, depending on which mapping direction was present
-     */
-    interface DepsConstructor<K, V, T>
-    {
-        T construct(K[] keys, int keysLength, V[] values, int valuesLength, int[] out, int outLength);
-    }
-
-    private static boolean arraysEqual(int[] left, int[] right, int length)
-    {
-        if (left.length < length || right.length < length)
-            return false;
-
-        for (int i=0; i<length; i++)
-            if (left[i] !=right[i])
-                return false;
-
-        return true;
-    }
-
-    private static <T> boolean arraysEqual(T[] left, T[] right, int length)
-    {
-        if (left.length < length || right.length < length)
-            return false;
-
-        for (int i=0; i<length; i++)
-            if (!Objects.equals(left[i], right[i]))
-                return false;
-
-        return true;
-    }
-
-    // TODO (low priority, efficiency): this method supports merging keyToTxnId OR txnIdToKey; we can perhaps save time
-    //  and effort when constructing Deps on remote hosts by only producing txnIdToKey with OrderedCollector and serializing
-    //  only this, and merging on the recipient before inverting, so that we only have to invert the final assembled deps
-    private static <K extends Comparable<? super K>, V extends Comparable<? super V>, T>
-    T linearUnion(K[] leftKeys, int leftKeysLength, V[] leftValues, int leftValuesLength, int[] left, int leftLength,
-                  K[] rightKeys, int rightKeysLength, V[] rightValues, int rightValuesLength, int[] right, int rightLength,
-                  ObjectBuffers<K> keyBuffers, ObjectBuffers<V> valueBuffers, IntBuffers intBuffers, DepsConstructor<K, V, T> constructor)
-    {
-        K[] outKeys = null;
-        V[] outValues = null;
-        int[] remapLeft = null, remapRight = null, out = null;
-        int outLength = 0, outKeysLength = 0, outTxnIdsLength = 0;
-
-        try
-        {
-            outKeys = SortedArrays.linearUnion(leftKeys, leftKeysLength, rightKeys, rightKeysLength, keyBuffers);
-            outKeysLength = keyBuffers.lengthOfLast(outKeys);
-            outValues = SortedArrays.linearUnion(leftValues, leftValuesLength, rightValues, rightValuesLength, valueBuffers);
-            outTxnIdsLength = valueBuffers.lengthOfLast(outValues);
-
-            remapLeft = remapToSuperset(leftValues, leftValuesLength, outValues, outTxnIdsLength, intBuffers);
-            remapRight = remapToSuperset(rightValues, rightValuesLength, outValues, outTxnIdsLength, intBuffers);
-
-            if (remapLeft == null && remapRight == null && leftLength == rightLength && leftKeysLength == rightKeysLength
-                    && arraysEqual(left, right, rightLength)
-                    && arraysEqual(leftKeys, rightKeys, rightKeysLength)
-                )
-            {
-                return constructor.construct(leftKeys, leftKeysLength, leftValues, leftValuesLength, left, leftLength);
-            }
-
-            int lk = 0, rk = 0, ok = 0, l = leftKeysLength, r = rightKeysLength;
-            outLength = outKeysLength;
-
-            if (remapLeft == null && outKeys == leftKeys)
-            {
-                // "this" knows all the TxnId and Keys already, but do both agree on what Keys map to TxnIds?
-                noOp: while (lk < leftKeysLength && rk < rightKeysLength)
-                {
-                    int ck = leftKeys[lk].compareTo(rightKeys[rk]);
-                    if (ck < 0)
-                    {
-                        // "this" knows of a key not present in "that"
-                        outLength += left[lk] - l; // logically append the key's TxnIds to the size
-                        l = left[lk];
-                        assert outLength == l && ok == lk && left[ok] == outLength;
-                        ok++;
-                        lk++;
-                    }
-                    else if (ck > 0)
-                    {
-                        // if this happened there is a bug with keys.union or keys are not actually sorted
-                        throwUnexpectedMissingKeyException(leftKeys, lk, leftKeysLength, rightKeys, rk, rightKeysLength, true);
-                    }
-                    else
-                    {
-                        // both "this" and "that" know of the key
-                        while (l < left[lk] && r < right[rk])
-                        {
-                            int nextLeft = left[l];
-                            int nextRight = remap(right[r], remapRight);
-
-                            if (nextLeft < nextRight)
-                            {
-                                // "this" knows of the txn that "that" didn't
-                                outLength++;
-                                l++;
-                            }
-                            else if (nextRight < nextLeft)
-                            {
-                                out = copy(left, outLength, leftLength + rightLength - r, intBuffers);
-                                break noOp;
-                            }
-                            else
-                            {
-                                outLength++;
-                                l++;
-                                r++;
-                            }
-                        }
-
-                        if (l < left[lk])
-                        {
-                            outLength += left[lk] - l;
-                            l = left[lk];
-                        }
-                        else if (r < right[rk])
-                        {
-                            // "that" thinks a key includes a TxnId as a dependency but "this" doesn't, need to include this knowledge
-                            out = copy(left, outLength, leftLength + rightLength - r, intBuffers);
-                            break;
-                        }
-
-                        assert outLength == l && ok == lk && left[ok] == outLength;
-                        ok++;
-                        rk++;
-                        lk++;
-                    }
-                }
-
-                if (out == null)
-                    return constructor.construct(leftKeys, leftKeysLength, leftValues, leftValuesLength, left, leftLength);
-            }
-            else if (remapRight == null && outKeys == rightKeys)
-            {
-                // "that" knows all the TxnId and keys already, but "this" does not
-                noOp: while (lk < leftKeysLength && rk < rightKeysLength)
-                {
-                    int ck = leftKeys[lk].compareTo(rightKeys[rk]);
-                    if (ck < 0)
-                    {
-                        // if this happened there is a bug with keys.union or keys are not actually sorted
-                        throwUnexpectedMissingKeyException(leftKeys, lk, leftKeysLength, rightKeys, rk, rightKeysLength, false);
-                    }
-                    else if (ck > 0)
-                    {
-                        outLength += right[rk] - r;
-                        r = right[rk];
-                        assert outLength == r && ok == rk && right[ok] == outLength;
-                        ok++;
-                        rk++;
-                    }
-                    else
-                    {
-                        // both "this" and "that" know of the key
-                        while (l < left[lk] && r < right[rk])
-                        {
-                            int nextLeft = remap(left[l], remapLeft);
-                            int nextRight = right[r];
-
-                            if (nextLeft < nextRight)
-                            {
-                                // "this" thinks a TxnID depends on Key but "that" doesn't, need to include this knowledge
-                                out = copy(right, outLength, rightLength + leftLength - l, intBuffers);
-                                break noOp;
-                            }
-                            else if (nextRight < nextLeft)
-                            {
-                                // "that" knows of the txn that "this" didn't
-                                outLength++;
-                                r++;
-                            }
-                            else
-                            {
-                                outLength++;
-                                l++;
-                                r++;
-                            }
-                        }
-
-                        if (l < left[lk])
-                        {
-                            out = copy(right, outLength, rightLength + leftLength - l, intBuffers);
-                            break;
-                        }
-                        else if (r < right[rk])
-                        {
-                            outLength += right[rk] - r;
-                            r = right[rk];
-                        }
-
-                        assert outLength == r && ok == rk && right[ok] == outLength;
-                        ok++;
-                        rk++;
-                        lk++;
-                    }
-                }
-
-                if (out == null)
-                    return constructor.construct(rightKeys, rightKeysLength, rightValues, rightValuesLength, right, rightLength);
-            }
-            else
-            {
-                out = intBuffers.getInts(leftLength + rightLength);
-            }
-
-            while (lk < leftKeysLength && rk < rightKeysLength)
-            {
-                int ck = leftKeys[lk].compareTo(rightKeys[rk]);
-                if (ck < 0)
-                {
-                    while (l < left[lk])
-                        out[outLength++] = remap(left[l++], remapLeft);
-                    out[ok++] = outLength;
-                    lk++;
-                }
-                else if (ck > 0)
-                {
-                    while (r < right[rk])
-                        out[outLength++] = remap(right[r++], remapRight);
-                    out[ok++] = outLength;
-                    rk++;
-                }
-                else
-                {
-                    while (l < left[lk] && r < right[rk])
-                    {
-                        int nextLeft = remap(left[l], remapLeft);
-                        int nextRight = remap(right[r], remapRight);
-
-                        if (nextLeft <= nextRight)
-                        {
-                            out[outLength++] = nextLeft;
-                            l += 1;
-                            r += nextLeft == nextRight ? 1 : 0;
-                        }
-                        else
-                        {
-                            out[outLength++] = nextRight;
-                            ++r;
-                        }
-                    }
-
-                    while (l < left[lk])
-                        out[outLength++] = remap(left[l++], remapLeft);
-
-                    while (r < right[rk])
-                        out[outLength++] = remap(right[r++], remapRight);
-
-                    out[ok++] = outLength;
-                    rk++;
-                    lk++;
-                }
-            }
-
-            while (lk < leftKeysLength)
-            {
-                while (l < left[lk])
-                    out[outLength++] = remap(left[l++], remapLeft);
-                out[ok++] = outLength;
-                lk++;
-            }
-
-            while (rk < rightKeysLength)
-            {
-                while (r < right[rk])
-                    out[outLength++] = remap(right[r++], remapRight);
-                out[ok++] = outLength;
-                rk++;
-            }
-
-            return constructor.construct(outKeys, outKeysLength, outValues, outTxnIdsLength, out, outLength);
-        }
-        finally
-        {
-            if (outKeys != null)
-                keyBuffers.discard(outKeys, outKeysLength);
-            if (outValues != null)
-                valueBuffers.discard(outValues, outTxnIdsLength);
-            if (out != null)
-                intBuffers.discard(out, outLength);
-            if (remapLeft != null)
-                intBuffers.forceDiscard(remapLeft, leftValuesLength);
-            if (remapRight != null)
-                intBuffers.forceDiscard(remapRight, rightValuesLength);
-        }
-    }
-
-    private static <A> void throwUnexpectedMissingKeyException(A[] leftKeys, int leftKeyIndex, int leftKeyLength, A[] rightKeys, int rightKeyIndex, int rightKeyLength, boolean isMissingLeft)
-    {
-        StringBuilder sb = new StringBuilder();
-        String missing = isMissingLeft ? "left" : "right";
-        String extra = isMissingLeft ? "right" : "left";
-        sb.append(missing).append(" knows all keys, yet ").append(extra).append(" knew of an extra key at indexes left[")
-                .append(leftKeyIndex).append("] = ").append(leftKeys[leftKeyIndex])
-                .append(", right[").append(rightKeyIndex).append("] = ").append(rightKeys[rightKeyIndex]).append("\n");
-        sb.append("leftKeys = ").append(Arrays.stream(leftKeys, 0, leftKeyLength).map(Object::toString).collect(Collectors.joining())).append('\n');
-        sb.append("rightKeys = ").append(Arrays.stream(rightKeys, 0, rightKeyLength).map(Object::toString).collect(Collectors.joining())).append('\n');
-        throw new IllegalStateException(sb.toString());
-    }
-
-    private static int[] copy(int[] src, int to, int length, IntBuffers bufferManager)
-    {
-        if (length == 0)
-            return NO_INTS;
-
-        int[] result = bufferManager.getInts(length);
-        if (result.length < length)
-            throw new IllegalStateException();
-        System.arraycopy(src, 0, result, 0, to);
-        return result;
+        return new Deps(this.keyDeps.with(that.keyDeps), this.rangeDeps.with(that.rangeDeps));
     }
 
     public Deps without(Predicate<TxnId> remove)
     {
-        if (isEmpty())
-            return this;
-
-        IntBuffers cache = ArrayBuffers.cachedInts();
-        TxnId[] oldTxnIds = txnIds;
-        int[] oldKeyToTxnId = keyToTxnId;
-        int[] remapTxnIds = cache.getInts(oldTxnIds.length);
-        int[] newKeyToTxnId = null;
-        TxnId[] newTxnIds;
-        int o = 0;
-        try
-        {
-            int count = 0;
-            for (int i = 0 ; i < oldTxnIds.length ; ++i)
-            {
-                if (remove.test(oldTxnIds[i])) remapTxnIds[i] = -1;
-                else remapTxnIds[i] = count++;
-            }
-
-            if (count == oldTxnIds.length)
-                return this;
-
-            if (count == 0)
-                return NONE;
-
-            newTxnIds = new TxnId[count];
-            for (int i = 0 ; i < oldTxnIds.length ; ++i)
-            {
-                if (remapTxnIds[i] >= 0)
-                    newTxnIds[remapTxnIds[i]] = oldTxnIds[i];
-            }
-
-            newKeyToTxnId = cache.getInts(oldKeyToTxnId.length);
-            int k = 0, i = keys.size();
-            o = i;
-            while (i < oldKeyToTxnId.length)
-            {
-                while (oldKeyToTxnId[k] == i)
-                    newKeyToTxnId[k++] = o;
-
-                int remapped = remapTxnIds[oldKeyToTxnId[i]];
-                if (remapped >= 0)
-                    newKeyToTxnId[o++] = remapped;
-                ++i;
-            }
-
-            while (k < keys.size())
-                newKeyToTxnId[k++] = o;
-        }
-        catch (Throwable t)
-        {
-            cache.forceDiscard(newKeyToTxnId, o);
-            throw t;
-        }
-        finally
-        {
-            cache.forceDiscard(remapTxnIds, oldTxnIds.length);
-        }
-
-        newKeyToTxnId = cache.completeAndDiscard(newKeyToTxnId, o);
-        return new Deps(keys, newTxnIds, newKeyToTxnId);
+        return new Deps(keyDeps.without(remove), rangeDeps.without(remove));
     }
 
-    public boolean contains(TxnId txnId)
+    public PartialDeps slice(Ranges covering)
     {
-        return Arrays.binarySearch(txnIds, txnId) >= 0;
+        return new PartialDeps(covering, keyDeps.slice(covering), rangeDeps.slice(covering));
     }
 
-    // return true iff we map any keys to any txnId
-    // if the mapping is empty we return false, whether or not we have any keys or txnId by themselves
     public boolean isEmpty()
     {
-        return keyToTxnId.length == keys.size();
-    }
-
-    public Keys someKeys(TxnId txnId)
-    {
-        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
-        if (txnIdIndex < 0)
-            return Keys.EMPTY;
-
-        ensureTxnIdToKey();
-
-        int start = txnIdIndex == 0 ? txnIds.length : txnIdToKey[txnIdIndex - 1];
-        int end = txnIdToKey[txnIdIndex];
-        if (start == end)
-            return Keys.EMPTY;
-
-        Key[] result = new Key[end - start];
-        for (int i = start ; i < end ; ++i)
-            result[i - start] = keys.get(txnIdToKey[i]);
-        return Keys.of(result);
-    }
-
-    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)
-            constructor.apply(RoutingKeys.EMPTY.keys);
-
-        ensureTxnIdToKey();
-
-        int start = txnIdIndex == 0 ? txnIds.length : txnIdToKey[txnIdIndex - 1];
-        int end = txnIdToKey[txnIdIndex];
-        RoutingKey[] result = new RoutingKey[end - start];
-        if (start == end)
-            constructor.apply(RoutingKeys.EMPTY.keys);
-
-        result[0] = keys.get(txnIdToKey[start]).toUnseekable();
-        int resultCount = 1;
-        for (int i = start + 1 ; i < end ; ++i)
-        {
-            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 constructor.apply(result);
-    }
-
-    void ensureTxnIdToKey()
-    {
-        if (txnIdToKey != null)
-            return;
-
-        txnIdToKey = invert(keyToTxnId, keyToTxnId.length, keys.size(), txnIds.length);
-    }
-
-    private static int[] invert(int[] src, int srcLength, int srcKeyCount, int trgKeyCount)
-    {
-        int[] trg = new int[trgKeyCount + srcLength - srcKeyCount];
-
-        // first pass, count number of txnId per key
-        for (int i = srcKeyCount ; i < srcLength ; ++i)
-            trg[src[i]]++;
-
-        // turn into offsets (i.e. add txnIds.size() and then sum them)
-        trg[0] += trgKeyCount;
-        for (int i = 1; i < trgKeyCount ; ++i)
-            trg[i] += trg[i - 1];
-
-        // shuffle forwards one, so we have the start index rather than end
-        System.arraycopy(trg, 0, trg, 1, trgKeyCount - 1);
-        trg[0] = trgKeyCount;
-
-        // convert the offsets to end, and set the key at the target positions
-        int k = 0;
-        for (int i = srcKeyCount ; i < srcLength ; ++i)
-        {
-            // if at the end offset, switch to the next key
-            while (i == src[k])
-                ++k;
-
-            // find the next key offset for the TxnId and set the offset to this key
-            trg[trg[src[i]]++] = k;
-        }
-
-        return trg;
-    }
-
-    public void forEachOn(Ranges ranges, Predicate<Key> include, BiConsumer<Key, TxnId> forEach)
-    {
-        Routables.foldl(keys, ranges, (key, value, index) -> {
-            if (!include.test(key))
-                return null;
-
-            for (int t = startOffset(index), end = endOffset(index); t < end ; ++t)
-            {
-                TxnId txnId = txnIds[keyToTxnId[t]];
-                forEach.accept(key, txnId);
-            }
-            return null;
-        }, null);
-    }
-
-    /**
-     * 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 forEach function to call on each unique {@link TxnId}
-     */
-    public void forEachOn(Ranges ranges, 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
-        // the property that forEach is called in TxnId order.
-        //TODO (expected, efficiency): reconsider this, probably not worth trying to save allocations at cost of multiple loop
-        //                             use BitSet, or perhaps extend so we can have no nested allocations when few bits
-        for (int offset = 0 ; offset < txnIds.length ; offset += 64)
-        {
-            long bitset = Routables.foldl(keys, ranges, (key, off, value, keyIndex) -> {
-                int index = startOffset(keyIndex);
-                int end = endOffset(keyIndex);
-                if (off > 0)
-                {
-                    // TODO (low priority, efficiency): interpolation search probably great here
-                    index = Arrays.binarySearch(keyToTxnId, index, end, (int)off);
-                    if (index < 0)
-                        index = -1 - index;
-                }
-
-                while (index < end)
-                {
-                    long next = keyToTxnId[index++] - off;
-                    if (next >= 64)
-                        break;
-                    value |= 1L << next;
-                }
-
-                return value;
-            }, offset, 0, -1L);
-
-            while (bitset != 0)
-            {
-                int i = Long.numberOfTrailingZeros(bitset);
-                TxnId txnId = txnIds[offset + i];
-                forEach.accept(txnId);
-                bitset ^= Long.lowestOneBit(bitset);
-            }
-        }
-    }
-
-    public void forEach(Key key, Consumer<TxnId> forEach)
-    {
-        int keyIndex = keys.indexOf(key);
-        if (keyIndex < 0)
-            return;
-
-        int index = startOffset(keyIndex);
-        int end = endOffset(keyIndex);
-        while (index < end)
-            forEach.accept(txnIds[keyToTxnId[index++]]);
-    }
-
-    public Keys keys()
-    {
-        return keys;
+        return keyDeps.isEmpty() && rangeDeps.isEmpty();
     }
 
     public int txnIdCount()
     {
-        return txnIds.length;
-    }
-
-    public int totalCount()
-    {
-        return keyToTxnId.length - keys.size();
+        return keyDeps.txnIdCount() + rangeDeps.txnIdCount();
     }
 
     public TxnId txnId(int i)
     {
-        return txnIds[i];
+        return i < keyDeps.txnIdCount()
+                ? keyDeps.txnId(i)
+                : rangeDeps.txnId(i - keyDeps.txnIdCount());
     }
 
-    public Collection<TxnId> txnIds()
+    public List<TxnId> txnIds()
     {
-        return Arrays.asList(txnIds);
-    }
-
-    public List<TxnId> txnIds(Key key)
-    {
-        int keyIndex = keys.indexOf(key);
-        if (keyIndex < 0)
-            return Collections.emptyList();
-
-        int start = startOffset(keyIndex);
-        int end = endOffset(keyIndex);
-        int size = end - start;
-
-        return new AbstractList<TxnId>()
-        {
+        final int txnIdCount = txnIdCount();
+        final int keyDepsCount = keyDeps.txnIdCount();
+        return new AbstractList<TxnId>() {
             @Override
             public TxnId get(int index)
             {
-                if (index > end)
-                    throw new IndexOutOfBoundsException();
-                return txnIds[keyToTxnId[start + index]];
+                return index < keyDepsCount
+                        ? keyDeps.txnId(index)
+                        : rangeDeps.txnId(index - keyDepsCount);
             }
 
             @Override
-            public int size()
-            {
-                return size;
+            public int size() {
+                return txnIdCount;
             }
         };
     }
 
-    private int startOffset(int keyIndex)
-    {
-        return keyIndex == 0 ? keys.size() : keyToTxnId[keyIndex - 1];
-    }
-
-    private int endOffset(int keyIndex)
+    public List<TxnId> txnIds(Seekable keyOrRange)
     {
-        return keyToTxnId[keyIndex];
-    }
-
-    @Override
-    public Iterator<Map.Entry<Key, TxnId>> iterator()
-    {
-        return new Iterator<Map.Entry<Key, TxnId>>()
+        List<TxnId> keyIds, rangeIds;
+        switch (keyOrRange.domain())
         {
-            int i = keys.size(), k = 0;
-
-            @Override
-            public boolean hasNext()
+            default: throw new AssertionError();
+            case Key:
             {
-                return i < keyToTxnId.length;
+                Key key = keyOrRange.asKey();
+                keyIds = keyDeps.txnIds(key);
+                rangeIds = rangeDeps.txnIds(key);
+                break;
             }
-
-            @Override
-            public Map.Entry<Key, TxnId> next()
+            case Range:
             {
-                Entry result = new Entry(keys.get(k), txnIds[keyToTxnId[i++]]);
-                if (i == keyToTxnId[k])
-                    ++k;
-                return result;
+                Range range = keyOrRange.asRange();
+                keyIds = keyDeps.txnIds(range);
+                rangeIds = rangeDeps.txnIds(range);
             }
-        };
-    }
-
-    @Override
-    public String toString()
-    {
-        return toSimpleString();
-    }
+        }
 
-    public String toSimpleString()
-    {
-        if (keys.isEmpty())
-            return "{}";
+        if (rangeIds.isEmpty()) return keyIds;
+        if (keyIds.isEmpty()) return rangeIds;
 
-        StringBuilder builder = new StringBuilder("{");
-        for (int k = 0, t = keys.size(); k < keys.size() ; ++k)
+        List<TxnId> output = new ArrayList<>();

Review Comment:
   Pre-size to the sum of input list sizes?



##########
accord-core/src/main/java/accord/primitives/Deps.java:
##########
@@ -18,1311 +18,220 @@
 
 package accord.primitives;
 
-import java.util.*;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-import java.util.function.Function;
-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 accord.utils.Invariants;
 
-import static accord.utils.ArrayBuffers.*;
-import static accord.utils.SortedArrays.*;
-import static accord.utils.SortedArrays.Search.FAST;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
 
 /**
- * A collection of dependencies for a transaction, organised by the key the dependency is adopted via.
- * An inverse map from TxnId to Key may also be constructed and stored in this collection.
+ * A collection of transaction dependencies, keyed by the key or range on which they were adopted
  */
-// TODO (desired, consider): switch to RoutingKey? Would mean adopting execution dependencies less precisely, but saving ser/deser of large keys
-public class Deps implements Iterable<Map.Entry<Key, TxnId>>
+public class Deps
 {
-    private static final boolean DEBUG_CHECKS = true;
-
-    private static final TxnId[] NO_TXNIDS = new TxnId[0];
-    private static final int[] NO_INTS = new int[0];
-    public static final Deps NONE = new Deps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+    public static final Deps NONE = new Deps(KeyDeps.NONE, RangeDeps.NONE);
 
-    public static class SerializerSupport
+    public static Builder builder()
     {
-        private SerializerSupport() {}
-
-        public static int keyToTxnIdCount(Deps deps)
-        {
-            return deps.keyToTxnId.length;
-        }
-
-        public static int keyToTxnId(Deps deps, int idx)
-        {
-            return deps.keyToTxnId[idx];
-        }
-
-        public static Deps create(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
-        {
-            return new Deps(keys, txnIds, keyToTxnId);
-        }
-    }
-
-    public static Deps none(Keys keys)
-    {
-        int[] keysToTxnId = new int[keys.size()];
-        Arrays.fill(keysToTxnId, keys.size());
-        return new Deps(keys, NO_TXNIDS, keysToTxnId);
-    }
-
-    /**
-     * Expects Command to be provided in TxnId order
-     */
-    public static OrderedBuilder orderedBuilder(boolean hasOrderedTxnId)
-    {
-        return new OrderedBuilder(hasOrderedTxnId);
+        return new Builder();
     }
 
     // TODO (expected, efficiency): cache this object per thread
-    public static abstract class AbstractOrderedBuilder<T extends Deps> implements AutoCloseable
+    public static abstract class AbstractBuilder<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;
-        // txnId -> Offset
-        TxnId[] keyToTxnId;
-        int keyCount;
-        int keyOffset;
-        int totalCount;
-
-        public AbstractOrderedBuilder(boolean hasOrderedTxnId)
-        {
-            this.keys = cachedKeys.get(16);
-            this.keyLimits = cachedInts.getInts(keys.length);
-            this.hasOrderedTxnId = hasOrderedTxnId;
-            this.keyToTxnId = cachedTxnIds.get(16);
-        }
-
-        public boolean isEmpty()
-        {
-            return totalCount() == 0;
-        }
-
-        private int totalCount()
-        {
-            return totalCount;
-        }
+        final KeyDeps.Builder keyBuilder;
+        RangeDeps.Builder rangeBuilder;
 
-        public void nextKey(Key key)
+        AbstractBuilder()
         {
-            if (keyCount > 0 && keys[keyCount - 1].compareTo(key) >= 0)
-            {
-                throw new IllegalArgumentException("Key " + key + " has already been visited or was provided out of order ("
-                        + Arrays.toString(Arrays.copyOf(keys, keyCount)) + ")");
-            }
-
-            finishKey();
-
-            if (keyCount == keys.length)
-            {
-                Key[] newKeys = cachedKeys.get(keyCount * 2);
-                System.arraycopy(keys, 0, newKeys, 0, keyCount);
-                cachedKeys.forceDiscard(keys, keyCount);
-                keys = newKeys;
-                int[] newKeyLimits = cachedInts.getInts(keyCount * 2);
-                System.arraycopy(keyLimits, 0, newKeyLimits, 0, keyCount);
-                cachedInts.forceDiscard(keyLimits, keyCount);
-                keyLimits = newKeyLimits;
-            }
-            keys[keyCount++] = key;
+            this.keyBuilder = KeyDeps.builder();
         }
 
-        private void finishKey()
+        public AbstractBuilder<T> add(Seekable keyOrRange, TxnId txnId)
         {
-            if (totalCount == keyOffset && keyCount > 0)
-                --keyCount; // remove this key; no data
-
-            if (keyCount == 0)
-                return;
-
-            if (totalCount != keyOffset && !hasOrderedTxnId)
+            switch (keyOrRange.domain())
             {
-                // TODO (low priority, efficiency): this allocates a significant amount of memory: would be preferable to be able to sort using a pre-defined scratch buffer
-                Arrays.sort(keyToTxnId, keyOffset, totalCount);
-                for (int i = keyOffset + 1 ; i < totalCount ; ++i)
-                {
-                    if (keyToTxnId[i - 1].equals(keyToTxnId[i]))
-                        throw new IllegalArgumentException("TxnId for " + keys[keyCount - 1] + " are not unique: " + Arrays.asList(keyToTxnId).subList(keyOffset, totalCount));
-                }
+                default: throw new AssertionError();
+                case Key:
+                    keyBuilder.add(keyOrRange.asKey(), txnId);
+                    break;
+                case Range:
+                    if (rangeBuilder == null)
+                        rangeBuilder = RangeDeps.builder();
+                    rangeBuilder.add(keyOrRange.asRange(), txnId);
+                    break;
             }
-
-            keyLimits[keyCount - 1] = totalCount;
-            keyOffset = totalCount;
-        }
-
-        public void add(Key key, TxnId txnId)
-        {
-            if (keyCount == 0 || !keys[keyCount - 1].equals(key))
-                nextKey(key);
-            add(txnId);
-        }
-
-        /**
-         * Add this command as a dependency for each intersecting key
-         */
-        public void add(TxnId txnId)
-        {
-            if (hasOrderedTxnId && totalCount > keyOffset && keyToTxnId[totalCount - 1].compareTo(txnId) >= 0)
-                throw new IllegalArgumentException("TxnId provided out of order");
-
-            if (totalCount >= keyToTxnId.length)
-            {
-                TxnId[] newTxnIds = cachedTxnIds.get(keyToTxnId.length * 2);
-                System.arraycopy(keyToTxnId, 0, newTxnIds, 0, totalCount);
-                cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
-                keyToTxnId = newTxnIds;
-            }
-
-            keyToTxnId[totalCount++] = txnId;
-        }
-
-        public T build()
-        {
-            if (totalCount == 0)
-                return build(Keys.EMPTY, NO_TXNIDS, NO_INTS);
-
-            finishKey();
-
-            TxnId[] uniqueTxnId = cachedTxnIds.get(totalCount);
-            System.arraycopy(keyToTxnId, 0, uniqueTxnId, 0, totalCount);
-            Arrays.sort(uniqueTxnId, 0, totalCount);
-            int txnIdCount = 1;
-            for (int i = 1 ; i < totalCount ; ++i)
-            {
-                if (!uniqueTxnId[txnIdCount - 1].equals(uniqueTxnId[i]))
-                    uniqueTxnId[txnIdCount++] = uniqueTxnId[i];
-            }
-
-            TxnId[] txnIds = cachedTxnIds.complete(uniqueTxnId, txnIdCount);
-            cachedTxnIds.discard(uniqueTxnId, totalCount);
-
-            int[] result = new int[keyCount + totalCount];
-            int offset = keyCount;
-            for (int k = 0 ; k < keyCount ; ++k)
-            {
-                result[k] = keyCount + keyLimits[k];
-                int from = k == 0 ? 0 : keyLimits[k - 1];
-                int to = keyLimits[k];
-                offset = (int)SortedArrays.foldlIntersection(txnIds, 0, txnIdCount, keyToTxnId, from, to, (key, p, v, li, ri) -> {
-                    result[(int)v] = li;
-                    return v + 1;
-                }, keyCount, offset, -1);
-            }
-
-            return build(Keys.ofSortedUnchecked(cachedKeys.complete(keys, keyCount)), txnIds, result);
+            return this;
         }
 
-        abstract T build(Keys keys, TxnId[] txnIds, int[] keyToTxnId);
+        public abstract T build();
 
         @Override
         public void close()
         {
-            cachedKeys.discard(keys, keyCount);
-            cachedInts.forceDiscard(keyLimits, keyCount);
-            cachedTxnIds.forceDiscard(keyToTxnId, totalCount);
-        }
-    }
-
-    public static class OrderedBuilder extends AbstractOrderedBuilder<Deps>
-    {
-        public OrderedBuilder(boolean hasOrderedTxnId)
-        {
-            super(hasOrderedTxnId);
-        }
-
-        @Override
-        Deps build(Keys keys, TxnId[] txnIds, int[] keysToTxnIds)
-        {
-            return new Deps(keys, txnIds, keysToTxnIds);
+            keyBuilder.close();
+            if (rangeBuilder != null)
+                rangeBuilder.close();
         }
     }
 
-    /**
-     * An object for managing a sequence of efficient linear merges Deps objects.
-     * Its primary purpose is to manage input and output buffers, so that we reuse output buffers
-     * as input to the next merge, and if any input is a superset of the other inputs that this input
-     * is returned unmodified.
-     *
-     * This is achieved by using PassThroughXBuffers so that the result buffers (and their sizes) are returned
-     * unmodified, and the buffers are cached as far as possible. In general, the buffers should be taken
-     * out of pre-existing caches, but if the buffers are too large then we cache any additional buffers we
-     * allocate for the duration of the merge.
-     */
-    private static class LinearMerger extends PassThroughObjectAndIntBuffers<TxnId> implements DepsConstructor<Key, TxnId, Object>
+    public static class Builder extends AbstractBuilder<Deps>
     {
-        final PassThroughObjectBuffers<Key> keyBuffers;
-        Key[] bufKeys;
-        TxnId[] bufTxnIds;
-        int[] buf = null;
-        int bufKeysLength, bufTxnIdsLength = 0, bufLength = 0;
-        Deps from = null;
-
-        LinearMerger()
-        {
-            super(cachedTxnIds(), cachedInts());
-            keyBuffers = new PassThroughObjectBuffers<>(cachedKeys());
-        }
-
-        @Override
-        public Object construct(Key[] keys, int keysLength, TxnId[] txnIds, int txnIdsLength, int[] out, int outLength)
-        {
-            if (from == null)
-            {
-                // if our input buffers were themselves buffers, we want to discard them unless they have been returned back to us
-                discard(keys, txnIds, out);
-            }
-            else if (buf != out)
-            {
-                // the output is not equal to a prior input
-                from = null;
-            }
-
-            if (from == null)
-            {
-                bufKeys = keys;
-                bufKeysLength = keysLength;
-                bufTxnIds = txnIds;
-                bufTxnIdsLength = txnIdsLength;
-                buf = out;
-                bufLength = outLength;
-            }
-            else
-            {
-                Invariants.checkState(keys == bufKeys && keysLength == bufKeysLength);
-                Invariants.checkState(txnIds == bufTxnIds && txnIdsLength == bufTxnIdsLength);
-                Invariants.checkState(outLength == bufLength);
-            }
-            return null;
-        }
-
-        void update(Deps deps)
+        public Builder()
         {
-            if (buf == null)
-            {
-                bufKeys = deps.keys.keys;
-                bufKeysLength = deps.keys.keys.length;
-                bufTxnIds = deps.txnIds;
-                bufTxnIdsLength = deps.txnIds.length;
-                buf = deps.keyToTxnId;
-                bufLength = deps.keyToTxnId.length;
-                from = deps;
-                return;
-            }
-
-            linearUnion(
-                    bufKeys, bufKeysLength, bufTxnIds, bufTxnIdsLength, buf, bufLength,
-                    deps.keys.keys, deps.keys.keys.length, deps.txnIds, deps.txnIds.length, deps.keyToTxnId, deps.keyToTxnId.length,
-                    keyBuffers, this, this, this
-            );
-            if (buf == deps.keyToTxnId)
-            {
-                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;
-            }
+            super();
         }
 
-        Deps get()
+        public Deps build()

Review Comment:
   Missing `@Override`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] belliottsmith merged pull request #21: Introduce RangeDeps

Posted by "belliottsmith (via GitHub)" <gi...@apache.org>.
belliottsmith merged PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] iamaleksey commented on a diff in pull request #21: Introduce RangeDeps

Posted by "iamaleksey (via GitHub)" <gi...@apache.org>.
iamaleksey commented on code in PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21#discussion_r1085662539


##########
accord-core/src/main/java/accord/primitives/RangeDeps.java:
##########
@@ -0,0 +1,672 @@
+package accord.primitives;
+
+import accord.api.Key;
+import accord.utils.*;
+import accord.utils.RelationMultiMap.AbstractBuilder;
+import accord.utils.RelationMultiMap.Adapter;
+import net.nicoulaj.compilecommand.annotations.DontInline;
+import net.nicoulaj.compilecommand.annotations.Inline;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.RelationMultiMap.remove;
+import static accord.utils.SearchableRangeListBuilder.Links.LINKS;
+import static accord.utils.SearchableRangeListBuilder.Strategy.ACCURATE;

Review Comment:
   Unused imports



##########
accord-core/src/main/java/accord/primitives/RangeDeps.java:
##########
@@ -0,0 +1,672 @@
+package accord.primitives;
+
+import accord.api.Key;
+import accord.utils.*;
+import accord.utils.RelationMultiMap.AbstractBuilder;
+import accord.utils.RelationMultiMap.Adapter;
+import net.nicoulaj.compilecommand.annotations.DontInline;
+import net.nicoulaj.compilecommand.annotations.Inline;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.RelationMultiMap.remove;
+import static accord.utils.SearchableRangeListBuilder.Links.LINKS;
+import static accord.utils.SearchableRangeListBuilder.Strategy.ACCURATE;
+import static accord.utils.SortedArrays.Search.CEIL;
+
+/**
+ * <p>Maintains a lazily-constructed, bidirectional map between Range and TxnId.
+ * <p>Ranges are stored sorted by start then end, and indexed by a secondary {@link SearchableRangeList} structure.
+ * <p>The relationship between Range and TxnId is maintained via {@code int[]} utilising {@link RelationMultiMap}
+ * functionality.
+ */
+public class RangeDeps implements Iterable<Map.Entry<Range, TxnId>>
+{
+    public static class SerializerSupport
+    {
+        private SerializerSupport() {}
+
+        public static int rangesToTxnIdsCount(RangeDeps deps)
+        {
+            return deps.rangesToTxnIds.length;
+        }
+
+        public static int rangesToTxnIds(RangeDeps deps, int idx)
+        {
+            return deps.rangesToTxnIds[idx];
+        }
+
+        public static RangeDeps create(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+        {
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    private static final Range[] NO_RANGES = new Range[0];
+    public static final RangeDeps NONE = new RangeDeps(new Range[0], new TxnId[0], new int[0], new int[0]);
+
+    final TxnId[] txnIds;
+    // the list of ranges and their mappings to txnIds
+    // unique, and sorted by start()
+    final Range[] ranges;
+    /**
+     * See {@link RelationMultiMap}.
+     * TODO consider alternative layout depending on real-world data distributions:
+     *      if most ranges have at most TxnId (or vice-versa) might be better to use negative values
+     *      to index into the dynamic portion of the array. We started with this, but decided it was
+     *      hard to justify the extra work for two layouts for the moment.
+     */
+    final int[] rangesToTxnIds;
+    int[] txnIdsToRanges;
+
+    private SearchableRangeList searchable;
+
+    public static <T1, T2> RangeDeps merge(List<T1> merge, Function<T1, T2> getter1, Function<T2, RangeDeps> getter2)
+    {
+        try (LinearMerger<Range, TxnId, RangeDeps> linearMerger = new LinearMerger<>(ADAPTER))
+        {
+            int mergeIndex = 0, mergeSize = merge.size();
+            while (mergeIndex < mergeSize)
+            {
+                T2 intermediate = getter1.apply(merge.get(mergeIndex++));
+                if (intermediate == null)
+                    continue;
+
+                RangeDeps deps = getter2.apply(intermediate);
+                if (deps == null || deps.isEmpty())
+                    continue;
+
+                linearMerger.update(deps, deps.ranges, deps.txnIds, deps.rangesToTxnIds);
+            }
+
+            return linearMerger.get(RangeDeps::new, NONE);
+        }
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+    {
+        this(ranges, txnIds, rangesToTxnIds, null);
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds, int[] txnIdsToRanges)
+    {
+        Invariants.checkArgument(rangesToTxnIds.length >= ranges.length);
+        Invariants.checkArgument(ranges.length > 0 || rangesToTxnIds.length == 0);
+        Invariants.paranoid(SortedArrays.isSorted(ranges, Range::compare));
+        this.ranges = ranges;
+        this.txnIds = txnIds;
+        this.rangesToTxnIds = rangesToTxnIds;
+        this.txnIdsToRanges = txnIdsToRanges;
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Key key, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(key, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Key key, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(key, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Range range, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(range, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private <P1, P2, P3> void forEach(Ranges ranges, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Range range, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(range, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int rangeIndex)
+    {
+        for (int i = startOffset(ranges, rangesToTxnIds, rangeIndex), end = endOffset(rangesToTxnIds, rangeIndex) ; i < end ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int start, int end)
+    {
+        if (end == 0)
+            return;
+        for (int i = startOffset(ranges, rangesToTxnIds, start) ; i < endOffset(rangesToTxnIds, end - 1) ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    // TODO (low priority, efficiency): ideally we would accept something like a BitHashSet or IntegerTrie
+    //   as O(N) space needed for BitSet here (but with a very low constant multiplier)
+    private void visitTxnIdx(int txnIdx, Consumer<TxnId> forEach, @Nullable BitSet visited)
+    {
+        if (visited == null || !visited.get(txnIdx))
+        {
+            if (visited != null)
+                visited.set(txnIdx);
+            forEach.accept(txnIds[txnIdx]);
+        }
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Key key, Consumer<TxnId> forEach)
+    {
+        forEach(key, forEach, 0, new BitSet());
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, null);
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, null);
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, new BitSet());
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     *
+     * @param ranges to match on
+     * @param forEach function to call on each unique {@link TxnId}
+     */
+    public void forEachUniqueTxnId(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, new BitSet());
+    }
+
+    // return true iff we map any ranges to any txnId
+    // if the mapping is empty we return false, whether or not we have any ranges or txnId by themselves
+    public boolean isEmpty()
+    {
+        return RelationMultiMap.isEmpty(ranges, rangesToTxnIds);
+    }
+
+    public Unseekables<Range, ?> someUnseekables(TxnId txnId)
+    {
+        return toUnseekables(txnId, array -> {
+            if (array.length == 0)
+                throw new IllegalStateException("Cannot create a RouteFragment without any keys");
+            return new Ranges(array);
+        });
+    }
+
+    private <R> R toUnseekables(TxnId txnId, Function<Range[], R> constructor)
+    {
+        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
+        if (txnIdIndex < 0)
+            constructor.apply(NO_RANGES);
+
+        ensureTxnIdToRange();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToRanges[txnIdIndex - 1];
+        int end = txnIdsToRanges[txnIdIndex];
+        Range[] result = new Range[end - start];
+        if (start == end)
+            constructor.apply(NO_RANGES);
+
+        result[0] = ranges[txnIdsToRanges[start]].toUnseekable();
+        int resultCount = 1;
+        for (int i = start + 1 ; i < end ; ++i)
+        {
+            Range next = ranges[txnIdsToRanges[i]];
+            if (!next.equals(result[resultCount - 1]))
+                result[resultCount++] = next;
+        }
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+        return constructor.apply(result);
+    }
+
+    void ensureTxnIdToRange()
+    {
+        if (txnIdsToRanges != null)
+            return;
+
+        txnIdsToRanges = invert(rangesToTxnIds, rangesToTxnIds.length, ranges.length, txnIds.length);
+    }
+
+    public RangeDeps slice(Ranges select)
+    {
+        if (isEmpty())
+            return new RangeDeps(NO_RANGES, txnIds, NO_INTS);
+
+        try (RangeAndMapCollector collector = new RangeAndMapCollector(ensureSearchable().maxScanAndCheckpointMatches))
+        {
+            forEach(select, collector, collector, ranges, rangesToTxnIds, null);
+
+            if (collector.rangesCount == 0)
+                return new RangeDeps(NO_RANGES, NO_TXNIDS, NO_INTS);
+
+            if (collector.rangesCount == this.ranges.length)
+                return this;
+
+            Range[] ranges = collector.getRanges();
+            int[] rangesToTxnIds = collector.getRangesToTxnIds();
+            TxnId[] txnIds = trimUnusedValues(ranges, this.txnIds, rangesToTxnIds, TxnId[]::new);
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    public RangeDeps with(RangeDeps that)
+    {
+        if (isEmpty() || that.isEmpty())
+            return isEmpty() ? that : this;
+
+        return linearUnion(
+                this.ranges, this.ranges.length, this.txnIds, this.txnIds.length, this.rangesToTxnIds, this.rangesToTxnIds.length,
+                that.ranges, that.ranges.length, that.txnIds, that.txnIds.length, that.rangesToTxnIds, that.rangesToTxnIds.length,
+                rangeComparator(), TxnId::compareTo,
+                cachedRanges(), cachedTxnIds(), cachedInts(),
+                (ranges, rangesLength, txnIds, txnIdsLength, out, outLength) ->
+                        new RangeDeps(cachedRanges().complete(ranges, rangesLength),
+                                cachedTxnIds().complete(txnIds, txnIdsLength),
+                                cachedInts().complete(out, outLength))
+        );
+    }
+
+    public RangeDeps without(Predicate<TxnId> remove)
+    {
+        return remove(this, ranges, txnIds, rangesToTxnIds, remove,
+                NONE, TxnId[]::new, ranges, RangeDeps::new);
+    }
+
+    public boolean contains(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId) >= 0;
+    }
+
+    public boolean isCoveredBy(Ranges covering)
+    {
+        // check that every entry intersects with some entry in covering
+        int prev = 0;
+        for (Range range : covering)
+        {
+            int start = SortedArrays.binarySearch(ranges, 0, ranges.length, range.start(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (start < 0) start = -1 - start;
+            int end = SortedArrays.binarySearch(ranges, 0, ranges.length, range.end(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (end < 0) end = -1 - end;
+            for (int i = prev; i < start ; ++i)
+            {
+                if (range.compareIntersecting(ranges[i]) != 0)
+                    return false;
+            }
+            prev = end;
+        }
+        return prev == ranges.length;
+    }
+
+    public List<TxnId> txnIds(Key key)
+    {
+        List<TxnId> result = new ArrayList<>();
+        forEachUniqueTxnId(key, result::add);
+        result.sort(TxnId::compareTo);
+        return result;
+    }
+
+    public List<TxnId> txnIds(Range key)
+    {
+        List<TxnId> result = new ArrayList<>();
+        forEachUniqueTxnId(key, result::add);
+        result.sort(TxnId::compareTo);
+        return result;
+    }
+
+    public TxnId txnId(int i)
+    {
+        return txnIds[i];
+    }
+
+    public int txnIdCount()
+    {
+        return txnIds.length;
+    }
+
+    public Range range(int i)
+    {
+        return ranges[i];
+    }
+
+    public int rangeCount()
+    {
+        return ranges.length;
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return this == that || (that instanceof RangeDeps && equals((RangeDeps)that));
+    }
+
+    public boolean equals(RangeDeps that)
+    {
+        return testEquality(this.ranges, this.txnIds, this.rangesToTxnIds, that.ranges, that.txnIds, that.rangesToTxnIds);
+    }
+
+    @Override
+    public String toString()
+    {
+        return RelationMultiMap.toSimpleString(ranges, txnIds, rangesToTxnIds);
+    }
+
+    @Nonnull
+    @Override
+    public Iterator<Map.Entry<Range, TxnId>> iterator()
+    {
+        return newIterator(ranges, txnIds, rangesToTxnIds);
+    }
+
+    private SearchableRangeList ensureSearchable()
+    {
+        if (searchable == null)
+            buildSearchable();
+        return searchable;
+    }
+
+    @DontInline
+    private void buildSearchable()
+    {
+        searchable = SearchableRangeList.build(ranges);
+    }
+
+    public boolean isSearchable()
+    {
+        return searchable != null;
+    }
+
+    static class RangeCollector implements
+            IndexedRangeTriConsumer<Range[], int[], Object>,
+            IndexedTriConsumer<Range[], int[], Object>,
+            AutoCloseable
+    {
+        int[] oooBuffer;
+        Range[] rangesOut;
+        int oooCount, rangesCount;
+
+        RangeCollector(int maxScanAndCheckpointCount)
+        {
+            oooBuffer = cachedInts().getInts(maxScanAndCheckpointCount);
+            rangesOut = cachedRanges().get(32);
+        }
+
+        @Override
+        public void accept(Range[] o, int[] o2, Object o3, int index)
+        {
+            oooBuffer[oooCount++] = index;
+        }
+
+        @Override
+        public void accept(Range[] ranges, int[] rangesToTxnIds, Object o3, int fromIndex, int toIndex)
+        {
+            if (oooCount > 0)
+            {
+                Arrays.sort(oooBuffer, 0, oooCount);
+                int oooCount = Arrays.binarySearch(oooBuffer, 0, this.oooCount, fromIndex);
+                if (oooCount < 0) oooCount = -1 - oooCount;
+                copy(ranges, rangesToTxnIds, oooCount, fromIndex, toIndex);
+                this.oooCount = 0;
+            }
+            else if (fromIndex < toIndex)
+            {
+                copy(ranges, rangesToTxnIds, 0, fromIndex, toIndex);
+            }
+        }
+
+        protected void copy(Range[] ranges, int[] rangesToTxnIds, int oooCount, int start, int end)
+        {
+            int count = oooCount + (end - start);
+            if (rangesCount + count >= rangesOut.length)
+                rangesOut = cachedRanges().resize(rangesOut, rangesCount, rangesCount + count + (rangesCount /2));
+            for (int i = 0 ; i < oooCount ; ++i)
+                rangesOut[rangesCount++] = ranges[oooBuffer[i]];
+            for (int i = start ; i < end ; ++i)
+                rangesOut[rangesCount++] = ranges[i];
+        }
+
+        Range[] getRanges()
+        {
+            Range[] result = cachedRanges().completeAndDiscard(rangesOut, rangesCount);
+            rangesOut = null;
+            return result;
+        }
+
+        @Override
+        public void close()
+        {
+            if (oooBuffer != null)
+            {
+                cachedInts().forceDiscard(oooBuffer);
+                oooBuffer = null;
+            }
+            if (rangesOut != null)
+            {
+                cachedRanges().forceDiscard(rangesOut, rangesCount);
+                rangesOut = null;
+            }
+        }
+    }
+
+    static class RangeAndMapCollector extends RangeCollector
+    {
+        int[] headers;
+        int[] lists;
+        int headerCount, listOffset;
+
+        RangeAndMapCollector(int maxScanAndCheckpointCount)
+        {
+            super(maxScanAndCheckpointCount);
+            headers = cachedInts().getInts(32);
+            lists = cachedInts().getInts(32);
+        }
+
+        @Override
+        protected void copy(Range[] ranges, int[] rangesToTxnIds, int oooCount, int start, int end)
+        {
+            super.copy(ranges, rangesToTxnIds, oooCount, start, end);
+            int count = oooCount + (end - start);
+            if (headerCount + count >= headers.length)
+                headers = cachedInts().resize(headers, headerCount, headerCount + count + (headerCount /2));
+            for (int i = 0 ; i < oooCount ; ++i)
+            {
+                int ri = oooBuffer[i];
+                copyToDynamic(rangesToTxnIds, startOffset(ranges, rangesToTxnIds, ri), endOffset(rangesToTxnIds, ri));
+                headers[headerCount++] = listOffset;
+            }
+            int startOffset = startOffset(ranges, rangesToTxnIds, start);
+            for (int i = start ; i < end ; ++i)
+                headers[this.headerCount++] = listOffset + rangesToTxnIds[i] - startOffset;
+            copyToDynamic(rangesToTxnIds, startOffset, startOffset(ranges, rangesToTxnIds, end));
+        }
+
+        protected void copyToDynamic(int[] rangesToTxnIds, int start, int end)
+        {
+            int count = end - start;
+            if (count + listOffset >= lists.length)
+                lists = cachedInts().resize(lists, listOffset, listOffset + (listOffset /2) + count);
+            System.arraycopy(rangesToTxnIds, start, lists, listOffset, count);
+            listOffset += count;
+        }
+
+        public int[] getRangesToTxnIds()
+        {
+            int[] out = new int[headerCount + listOffset];
+            for (int i = 0; i < headerCount; ++i)
+                out[i] = headers[i] + headerCount;
+            System.arraycopy(lists, 0, out, headerCount, listOffset);
+            return out;
+        }
+    }
+
+    public static RangeDeps of(Map<TxnId, Ranges> txnIdRanges)
+    {
+        if (txnIdRanges.isEmpty())
+            return NONE;
+
+        try (BuilderByTxnId builder = new BuilderByTxnId())
+        {
+            for (Map.Entry<TxnId, Ranges> e : txnIdRanges.entrySet())
+            {
+                builder.nextKey(e.getKey());
+                Ranges ranges = e.getValue();
+                for (int i = 0 ; i < ranges.size() ; ++i)
+                    builder.add(ranges.get(i));
+            }
+            return builder.build();
+        }
+    }
+
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static class Builder extends AbstractBuilder<Range, TxnId, RangeDeps>
+    {
+        public Builder()
+        {
+            super(ADAPTER);
+        }
+
+        @Override
+        protected RangeDeps none()
+        {
+            return RangeDeps.NONE;
+        }
+
+        @Override
+        protected RangeDeps build(Range[] ranges, TxnId[] txnIds, int[] keyToValue)
+        {
+            return new RangeDeps(ranges, txnIds, keyToValue);
+        }
+    }
+
+    public static BuilderByTxnId byTxnIdBuilder()

Review Comment:
   Unused method



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] iamaleksey commented on a diff in pull request #21: Introduce RangeDeps

Posted by "iamaleksey (via GitHub)" <gi...@apache.org>.
iamaleksey commented on code in PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21#discussion_r1084129076


##########
accord-core/src/main/java/accord/primitives/RangeDeps.java:
##########
@@ -0,0 +1,677 @@
+package accord.primitives;
+
+import accord.api.Key;
+import accord.utils.*;
+import accord.utils.RelationMultiMap.AbstractBuilder;
+import accord.utils.RelationMultiMap.Adapter;
+import net.nicoulaj.compilecommand.annotations.DontInline;
+import net.nicoulaj.compilecommand.annotations.Inline;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.RelationMultiMap.remove;
+import static accord.utils.SearchableRangeListBuilder.Links.LINKS;
+import static accord.utils.SearchableRangeListBuilder.Strategy.ACCURATE;
+import static accord.utils.SortedArrays.Search.CEIL;
+
+/**
+ * <p>Maintains a lazily-constructed, bidirectional map between Range and TxnId.
+ * <p>Ranges are stored sorted by start then end, and indexed by a secondary {@link SearchableRangeList} structure.
+ * <p>The relationship between Range and TxnId is maintained via {@code int[]} utilising {@link RelationMultiMap}
+ * functionality.
+ */
+public class RangeDeps implements Iterable<Map.Entry<Range, TxnId>>
+{
+    public static class SerializerSupport
+    {
+        private SerializerSupport() {}
+
+        public static int rangesToTxnIdsCount(RangeDeps deps)
+        {
+            return deps.rangesToTxnIds.length;
+        }
+
+        public static int rangesToTxnIds(RangeDeps deps, int idx)
+        {
+            return deps.rangesToTxnIds[idx];
+        }
+
+        public static RangeDeps create(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+        {
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    private static final Range[] NO_RANGES = new Range[0];
+    public static final RangeDeps NONE = new RangeDeps(new Range[0], new TxnId[0], new int[0], new int[0]);
+
+    final TxnId[] txnIds;
+    // the list of ranges and their mappings to txnIds
+    // unique, and sorted by start()
+    final Range[] ranges;
+    /**
+     * See {@link RelationMultiMap}.
+     * TODO consider alternative layout depending on real-world data distributions:
+     *      if most ranges have at most TxnId (or vice-versa) might be better to use negative values
+     *      to index into the dynamic portion of the array. We started with this, but decided it was
+     *      hard to justify the extra work for two layouts for the moment.
+     */
+    final int[] rangesToTxnIds;
+    int[] txnIdsToRanges;
+
+    private SearchableRangeList searchable;
+
+    public static <T1, T2> RangeDeps merge(List<T1> merge, Function<T1, T2> getter1, Function<T2, RangeDeps> getter2)
+    {
+        try (LinearMerger<Range, TxnId, RangeDeps> linearMerger = new LinearMerger<>(ADAPTER))
+        {
+            int mergeIndex = 0, mergeSize = merge.size();
+            while (mergeIndex < mergeSize)
+            {
+                T2 intermediate = getter1.apply(merge.get(mergeIndex++));
+                if (intermediate == null)
+                    continue;
+
+                RangeDeps deps = getter2.apply(intermediate);
+                if (deps == null || deps.isEmpty())
+                    continue;
+
+                linearMerger.update(deps, deps.ranges, deps.txnIds, deps.rangesToTxnIds);
+            }
+
+            return linearMerger.get(RangeDeps::new, NONE);
+        }
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+    {
+        this(ranges, txnIds, rangesToTxnIds, null);
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds, int[] txnIdsToRanges)
+    {
+        Invariants.checkArgument(rangesToTxnIds.length >= ranges.length);
+        Invariants.checkArgument(ranges.length > 0 || rangesToTxnIds.length == 0);
+        Invariants.paranoid(SortedArrays.isSorted(ranges, Range::compare));
+        this.ranges = ranges;
+        this.txnIds = txnIds;
+        this.rangesToTxnIds = rangesToTxnIds;
+        this.txnIdsToRanges = txnIdsToRanges;
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Key key, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(key, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Key key, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(key, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Range range, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(range, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private <P1, P2, P3> void forEach(Ranges ranges, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Range range, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(range, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int rangeIndex)
+    {
+        for (int i = startOffset(ranges, rangesToTxnIds, rangeIndex), end = endOffset(rangesToTxnIds, rangeIndex) ; i < end ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int start, int end)
+    {
+        if (end == 0)
+            return;
+        for (int i = startOffset(ranges, rangesToTxnIds, start) ; i < endOffset(rangesToTxnIds, end - 1) ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    // TODO (low priority, efficiency): ideally we would accept something like a BitHashSet or IntegerTrie
+    //   as O(N) space needed for BitSet here (but with a very low constant multiplier)
+    private void visitTxnIdx(int txnIdx, Consumer<TxnId> forEach, @Nullable BitSet visited)
+    {
+        if (visited == null || !visited.get(txnIdx))
+        {
+            if (visited != null)
+                visited.set(txnIdx);
+            forEach.accept(txnIds[txnIdx]);
+        }
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Key key, Consumer<TxnId> forEach)
+    {
+        forEach(key, forEach, 0, new BitSet());
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, null);
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, null);
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, new BitSet());
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     *
+     * @param ranges to match on
+     * @param forEach function to call on each unique {@link TxnId}
+     */
+    public void forEachUniqueTxnId(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, new BitSet());
+    }
+
+    // return true iff we map any ranges to any txnId
+    // if the mapping is empty we return false, whether or not we have any ranges or txnId by themselves
+    public boolean isEmpty()
+    {
+        return RelationMultiMap.isEmpty(ranges, rangesToTxnIds);
+    }
+
+    public Unseekables<Range, ?> someUnseekables(TxnId txnId)
+    {
+        return toUnseekables(txnId, array -> {
+            if (array.length == 0)
+                throw new IllegalStateException("Cannot create a RouteFragment without any keys");
+            return new Ranges(array);
+        });
+    }
+
+    private <R> R toUnseekables(TxnId txnId, Function<Range[], R> constructor)
+    {
+        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
+        if (txnIdIndex < 0)
+            constructor.apply(NO_RANGES);
+
+        ensureTxnIdToRange();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToRanges[txnIdIndex - 1];
+        int end = txnIdsToRanges[txnIdIndex];
+        Range[] result = new Range[end - start];
+        if (start == end)
+            constructor.apply(NO_RANGES);
+
+        result[0] = ranges[txnIdsToRanges[start]].toUnseekable();
+        int resultCount = 1;
+        for (int i = start + 1 ; i < end ; ++i)
+        {
+            Range next = ranges[txnIdsToRanges[i]];
+            if (!next.equals(result[resultCount - 1]))
+                result[resultCount++] = next;
+        }
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+        return constructor.apply(result);
+    }
+
+    void ensureTxnIdToRange()
+    {
+        if (txnIdsToRanges != null)
+            return;
+
+        txnIdsToRanges = invert(rangesToTxnIds, rangesToTxnIds.length, ranges.length, txnIds.length);
+    }
+
+    public RangeDeps slice(Ranges select)
+    {
+        if (isEmpty())
+            return new RangeDeps(NO_RANGES, txnIds, NO_INTS);
+
+        try (RangeAndMapCollector collector = new RangeAndMapCollector(ensureSearchable().maxScanAndCheckpointMatches))
+        {
+            forEach(select, collector, collector, ranges, rangesToTxnIds, null);
+
+            if (collector.rangesCount == 0)
+                return new RangeDeps(NO_RANGES, NO_TXNIDS, NO_INTS);
+
+            if (collector.rangesCount == this.ranges.length)
+                return this;
+
+            Range[] ranges = collector.getRanges();
+            int[] rangesToTxnIds = collector.getRangesToTxnIds();
+            TxnId[] txnIds = trimUnusedValues(ranges, this.txnIds, rangesToTxnIds, TxnId[]::new);
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    public RangeDeps with(RangeDeps that)
+    {
+        if (isEmpty() || that.isEmpty())
+            return isEmpty() ? that : this;
+
+        return linearUnion(
+                this.ranges, this.ranges.length, this.txnIds, this.txnIds.length, this.rangesToTxnIds, this.rangesToTxnIds.length,
+                that.ranges, that.ranges.length, that.txnIds, that.txnIds.length, that.rangesToTxnIds, that.rangesToTxnIds.length,
+                rangeComparator(), TxnId::compareTo,
+                cachedRanges(), cachedTxnIds(), cachedInts(),
+                (ranges, rangesLength, txnIds, txnIdsLength, out, outLength) ->
+                        new RangeDeps(cachedRanges().complete(ranges, rangesLength),
+                                cachedTxnIds().complete(txnIds, txnIdsLength),
+                                cachedInts().complete(out, outLength))
+        );
+    }
+
+    public RangeDeps without(Predicate<TxnId> remove)
+    {
+        return remove(this, ranges, txnIds, rangesToTxnIds, remove,
+                NONE, TxnId[]::new, ranges, RangeDeps::new);
+    }
+
+    public boolean contains(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId) >= 0;
+    }
+
+    public boolean isCoveredBy(Ranges covering)
+    {
+        // check that every entry intersects with some entry in covering
+        int prev = 0;
+        for (Range range : covering)
+        {
+            int start = SortedArrays.binarySearch(ranges, 0, ranges.length, range.start(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (start < 0) start = -1 - start;
+            int end = SortedArrays.binarySearch(ranges, 0, ranges.length, range.end(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (end < 0) end = -1 - end;
+            for (int i = prev; i < start ; ++i)
+            {
+                if (range.compareIntersecting(ranges[i]) != 0)
+                    return false;
+            }
+            prev = end;
+        }
+        return prev == ranges.length;
+    }
+
+    public List<TxnId> txnIds(Key key)
+    {
+        List<TxnId> result = new ArrayList<>();
+        forEachUniqueTxnId(key, result::add);
+        result.sort(TxnId::compareTo);
+        return result;
+    }
+
+    public List<TxnId> txnIds(Range key)
+    {
+        List<TxnId> result = new ArrayList<>();
+        forEachUniqueTxnId(key, result::add);
+        result.sort(TxnId::compareTo);
+        return result;
+    }
+
+    public TxnId txnId(int i)
+    {
+        return txnIds[i];
+    }
+
+    public int txnIdCount()
+    {
+        return txnIds.length;
+    }
+
+    public Range range(int i)
+    {
+        return ranges[i];
+    }
+
+    public int rangeCount()
+    {
+        return ranges.length;
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return this == that || (that instanceof RangeDeps && equals((RangeDeps)that));
+    }
+
+    public boolean equals(RangeDeps that)
+    {
+        return testEquality(this.ranges, this.txnIds, this.rangesToTxnIds, that.ranges, that.txnIds, that.rangesToTxnIds);
+    }
+
+    @Override
+    public String toString()
+    {
+        return RelationMultiMap.toSimpleString(ranges, txnIds, rangesToTxnIds);
+    }
+
+    @Nonnull
+    @Override
+    public Iterator<Map.Entry<Range, TxnId>> iterator()
+    {
+        return newIterator(ranges, txnIds, rangesToTxnIds);
+    }
+
+    private SearchableRangeList ensureSearchable()
+    {
+        if (searchable == null)
+            buildSearchable();
+        return searchable;
+    }
+
+    @DontInline
+    private void buildSearchable()
+    {
+        searchable = SearchableRangeList.build(ranges);
+    }
+
+    public boolean isSearchable()
+    {
+        return searchable != null;
+    }
+
+    public void buildSearchable(int maxScanDistance)
+    {
+        searchable = SearchableRangeList.build(ranges, maxScanDistance, ACCURATE, LINKS);
+    }
+
+    static class RangeCollector implements
+            IndexedRangeTriConsumer<Range[], int[], Object>,
+            IndexedTriConsumer<Range[], int[], Object>,
+            AutoCloseable
+    {
+        int[] oooBuffer;
+        Range[] rangesOut;
+        int oooCount, rangesCount;
+
+        RangeCollector(int maxScanAndCheckpointCount)
+        {
+            oooBuffer = cachedInts().getInts(maxScanAndCheckpointCount);
+            rangesOut = cachedRanges().get(32);
+        }
+
+        @Override
+        public void accept(Range[] o, int[] o2, Object o3, int index)
+        {
+            oooBuffer[oooCount++] = index;
+        }
+
+        @Override
+        public void accept(Range[] ranges, int[] rangesToTxnIds, Object o3, int fromIndex, int toIndex)
+        {
+            if (oooCount > 0)
+            {
+                Arrays.sort(oooBuffer, 0, oooCount);
+                int oooCount = Arrays.binarySearch(oooBuffer, 0, this.oooCount, fromIndex);
+                if (oooCount < 0) oooCount = -1 - oooCount;
+                copy(ranges, rangesToTxnIds, oooCount, fromIndex, toIndex);
+                this.oooCount = 0;
+            }
+            else if (fromIndex < toIndex)
+            {
+                copy(ranges, rangesToTxnIds, 0, fromIndex, toIndex);
+            }
+        }
+
+        protected void copy(Range[] ranges, int[] rangesToTxnIds, int oooCount, int start, int end)
+        {
+            int count = oooCount + (end - start);
+            if (rangesCount + count >= rangesOut.length)
+                rangesOut = cachedRanges().resize(rangesOut, rangesCount, rangesCount + count + (rangesCount /2));
+            for (int i = 0 ; i < oooCount ; ++i)
+                rangesOut[rangesCount++] = ranges[oooBuffer[i]];
+            for (int i = start ; i < end ; ++i)
+                rangesOut[rangesCount++] = ranges[i];
+        }
+
+        Range[] getRanges()
+        {
+            Range[] result = cachedRanges().completeAndDiscard(rangesOut, rangesCount);
+            rangesOut = null;
+            return result;
+        }
+
+        @Override
+        public void close()
+        {
+            if (oooBuffer != null)
+            {
+                cachedInts().forceDiscard(oooBuffer);
+                oooBuffer = null;
+            }
+            if (rangesOut != null)
+            {
+                cachedRanges().forceDiscard(rangesOut, rangesCount);
+                rangesOut = null;
+            }
+        }
+    }
+
+    static class RangeAndMapCollector extends RangeCollector
+    {
+        int[] headers;
+        int[] lists;
+        int headerCount, listOffset;
+
+        RangeAndMapCollector(int maxScanAndCheckpointCount)
+        {
+            super(maxScanAndCheckpointCount);
+            headers = cachedInts().getInts(32);
+            lists = cachedInts().getInts(32);
+        }
+
+        @Override
+        protected void copy(Range[] ranges, int[] rangesToTxnIds, int oooCount, int start, int end)
+        {
+            super.copy(ranges, rangesToTxnIds, oooCount, start, end);
+            int count = oooCount + (end - start);
+            if (headerCount + count >= headers.length)
+                headers = cachedInts().resize(headers, headerCount, headerCount + count + (headerCount /2));
+            for (int i = 0 ; i < oooCount ; ++i)
+            {
+                int ri = oooBuffer[i];
+                copyToDynamic(rangesToTxnIds, startOffset(ranges, rangesToTxnIds, ri), endOffset(rangesToTxnIds, ri));
+                headers[headerCount++] = listOffset;
+            }
+            int startOffset = startOffset(ranges, rangesToTxnIds, start);
+            for (int i = start ; i < end ; ++i)
+                headers[this.headerCount++] = listOffset + rangesToTxnIds[i] - startOffset;
+            copyToDynamic(rangesToTxnIds, startOffset, startOffset(ranges, rangesToTxnIds, end));
+        }
+
+        protected void copyToDynamic(int[] rangesToTxnIds, int start, int end)
+        {
+            int count = end - start;
+            if (count + listOffset >= lists.length)
+                lists = cachedInts().resize(lists, listOffset, listOffset + (listOffset /2) + count);
+            System.arraycopy(rangesToTxnIds, start, lists, listOffset, count);
+            listOffset += count;
+        }
+
+        public int[] getRangesToTxnIds()
+        {
+            int[] out = new int[headerCount + listOffset];
+            for (int i = 0; i < headerCount; ++i)
+                out[i] = headers[i] + headerCount;
+            System.arraycopy(lists, 0, out, headerCount, listOffset);
+            return out;
+        }
+    }
+
+    public static RangeDeps of(Map<TxnId, Ranges> txnIdRanges)
+    {
+        if (txnIdRanges.isEmpty())
+            return NONE;
+
+        try (BuilderByTxnId builder = new BuilderByTxnId(true))
+        {
+            for (Map.Entry<TxnId, Ranges> e : txnIdRanges.entrySet())
+            {
+                builder.nextKey(e.getKey());
+                Ranges ranges = e.getValue();
+                for (int i = 0 ; i < ranges.size() ; ++i)
+                    builder.add(ranges.get(i));
+            }
+            return builder.build();
+        }
+    }
+
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static class Builder extends AbstractBuilder<Range, TxnId, RangeDeps>
+    {
+        public Builder()
+        {
+            super(ADAPTER);
+        }
+
+        @Override
+        protected RangeDeps none()
+        {
+            return RangeDeps.NONE;
+        }
+
+        @Override
+        protected RangeDeps build(Range[] ranges, TxnId[] txnIds, int[] keyToValue)
+        {
+            return new RangeDeps(ranges, txnIds, keyToValue);
+        }
+    }
+
+    public static BuilderByTxnId byTxnIdBuilder(boolean hasOrderedRanges)

Review Comment:
   `hasOrderedRanges` - vestige of some older code?



##########
accord-core/src/main/java/accord/primitives/RangeDeps.java:
##########
@@ -0,0 +1,677 @@
+package accord.primitives;
+
+import accord.api.Key;
+import accord.utils.*;
+import accord.utils.RelationMultiMap.AbstractBuilder;
+import accord.utils.RelationMultiMap.Adapter;
+import net.nicoulaj.compilecommand.annotations.DontInline;
+import net.nicoulaj.compilecommand.annotations.Inline;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.*;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.RelationMultiMap.remove;
+import static accord.utils.SearchableRangeListBuilder.Links.LINKS;
+import static accord.utils.SearchableRangeListBuilder.Strategy.ACCURATE;
+import static accord.utils.SortedArrays.Search.CEIL;
+
+/**
+ * <p>Maintains a lazily-constructed, bidirectional map between Range and TxnId.
+ * <p>Ranges are stored sorted by start then end, and indexed by a secondary {@link SearchableRangeList} structure.
+ * <p>The relationship between Range and TxnId is maintained via {@code int[]} utilising {@link RelationMultiMap}
+ * functionality.
+ */
+public class RangeDeps implements Iterable<Map.Entry<Range, TxnId>>
+{
+    public static class SerializerSupport
+    {
+        private SerializerSupport() {}
+
+        public static int rangesToTxnIdsCount(RangeDeps deps)
+        {
+            return deps.rangesToTxnIds.length;
+        }
+
+        public static int rangesToTxnIds(RangeDeps deps, int idx)
+        {
+            return deps.rangesToTxnIds[idx];
+        }
+
+        public static RangeDeps create(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+        {
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    private static final Range[] NO_RANGES = new Range[0];
+    public static final RangeDeps NONE = new RangeDeps(new Range[0], new TxnId[0], new int[0], new int[0]);
+
+    final TxnId[] txnIds;
+    // the list of ranges and their mappings to txnIds
+    // unique, and sorted by start()
+    final Range[] ranges;
+    /**
+     * See {@link RelationMultiMap}.
+     * TODO consider alternative layout depending on real-world data distributions:
+     *      if most ranges have at most TxnId (or vice-versa) might be better to use negative values
+     *      to index into the dynamic portion of the array. We started with this, but decided it was
+     *      hard to justify the extra work for two layouts for the moment.
+     */
+    final int[] rangesToTxnIds;
+    int[] txnIdsToRanges;
+
+    private SearchableRangeList searchable;
+
+    public static <T1, T2> RangeDeps merge(List<T1> merge, Function<T1, T2> getter1, Function<T2, RangeDeps> getter2)
+    {
+        try (LinearMerger<Range, TxnId, RangeDeps> linearMerger = new LinearMerger<>(ADAPTER))
+        {
+            int mergeIndex = 0, mergeSize = merge.size();
+            while (mergeIndex < mergeSize)
+            {
+                T2 intermediate = getter1.apply(merge.get(mergeIndex++));
+                if (intermediate == null)
+                    continue;
+
+                RangeDeps deps = getter2.apply(intermediate);
+                if (deps == null || deps.isEmpty())
+                    continue;
+
+                linearMerger.update(deps, deps.ranges, deps.txnIds, deps.rangesToTxnIds);
+            }
+
+            return linearMerger.get(RangeDeps::new, NONE);
+        }
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds)
+    {
+        this(ranges, txnIds, rangesToTxnIds, null);
+    }
+
+    private RangeDeps(Range[] ranges, TxnId[] txnIds, int[] rangesToTxnIds, int[] txnIdsToRanges)
+    {
+        Invariants.checkArgument(rangesToTxnIds.length >= ranges.length);
+        Invariants.checkArgument(ranges.length > 0 || rangesToTxnIds.length == 0);
+        Invariants.paranoid(SortedArrays.isSorted(ranges, Range::compare));
+        this.ranges = ranges;
+        this.txnIds = txnIds;
+        this.rangesToTxnIds = rangesToTxnIds;
+        this.txnIdsToRanges = txnIdsToRanges;
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Key key, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(key, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Key key, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(key, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    @Inline
+    public <P1, P2, P3> int forEach(Range range, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3, int minIndex)
+    {
+        return ensureSearchable().forEach(range, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private <P1, P2, P3> void forEach(Ranges ranges, IndexedTriConsumer<P1, P2, P3> forEachScanOrCheckpoint, IndexedRangeTriConsumer<P1, P2, P3> forEachRange, P1 p1, P2 p2, P3 p3)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEachScanOrCheckpoint, forEachRange, p1, p2, p3, minIndex);
+    }
+
+    private int forEach(Range range, Consumer<TxnId> forEach, int minIndex, @Nullable BitSet visited)
+    {
+        return forEach(range, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex,
+                this, forEach, visited, minIndex);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int rangeIndex)
+    {
+        for (int i = startOffset(ranges, rangesToTxnIds, rangeIndex), end = endOffset(rangesToTxnIds, rangeIndex) ; i < end ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    private void visitTxnIdsForRangeIndex(Consumer<TxnId> forEach, @Nullable BitSet visited, int start, int end)
+    {
+        if (end == 0)
+            return;
+        for (int i = startOffset(ranges, rangesToTxnIds, start) ; i < endOffset(rangesToTxnIds, end - 1) ; ++i)
+            visitTxnIdx(rangesToTxnIds[i], forEach, visited);
+    }
+
+    // TODO (low priority, efficiency): ideally we would accept something like a BitHashSet or IntegerTrie
+    //   as O(N) space needed for BitSet here (but with a very low constant multiplier)
+    private void visitTxnIdx(int txnIdx, Consumer<TxnId> forEach, @Nullable BitSet visited)
+    {
+        if (visited == null || !visited.get(txnIdx))
+        {
+            if (visited != null)
+                visited.set(txnIdx);
+            forEach.accept(txnIds[txnIdx]);
+        }
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Key key, Consumer<TxnId> forEach)
+    {
+        forEach(key, forEach, 0, new BitSet());
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, null);
+    }
+
+    /**
+     * The same TxnId may be provided as a parameter multiple times
+     */
+    public void forEach(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, null);
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     */
+    public void forEachUniqueTxnId(Range range, Consumer<TxnId> forEach)
+    {
+        forEach(range, forEach, 0, new BitSet());
+    }
+
+    /**
+     * Each matching TxnId will be provided precisely once
+     *
+     * @param ranges to match on
+     * @param forEach function to call on each unique {@link TxnId}
+     */
+    public void forEachUniqueTxnId(Ranges ranges, Consumer<TxnId> forEach)
+    {
+        int minIndex = 0;
+        for (int i = 0; i < ranges.size() ; ++i)
+            minIndex = forEach(ranges.get(i), forEach, minIndex, new BitSet());
+    }
+
+    // return true iff we map any ranges to any txnId
+    // if the mapping is empty we return false, whether or not we have any ranges or txnId by themselves
+    public boolean isEmpty()
+    {
+        return RelationMultiMap.isEmpty(ranges, rangesToTxnIds);
+    }
+
+    public Unseekables<Range, ?> someUnseekables(TxnId txnId)
+    {
+        return toUnseekables(txnId, array -> {
+            if (array.length == 0)
+                throw new IllegalStateException("Cannot create a RouteFragment without any keys");
+            return new Ranges(array);
+        });
+    }
+
+    private <R> R toUnseekables(TxnId txnId, Function<Range[], R> constructor)
+    {
+        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
+        if (txnIdIndex < 0)
+            constructor.apply(NO_RANGES);
+
+        ensureTxnIdToRange();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToRanges[txnIdIndex - 1];
+        int end = txnIdsToRanges[txnIdIndex];
+        Range[] result = new Range[end - start];
+        if (start == end)
+            constructor.apply(NO_RANGES);
+
+        result[0] = ranges[txnIdsToRanges[start]].toUnseekable();
+        int resultCount = 1;
+        for (int i = start + 1 ; i < end ; ++i)
+        {
+            Range next = ranges[txnIdsToRanges[i]];
+            if (!next.equals(result[resultCount - 1]))
+                result[resultCount++] = next;
+        }
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+        return constructor.apply(result);
+    }
+
+    void ensureTxnIdToRange()
+    {
+        if (txnIdsToRanges != null)
+            return;
+
+        txnIdsToRanges = invert(rangesToTxnIds, rangesToTxnIds.length, ranges.length, txnIds.length);
+    }
+
+    public RangeDeps slice(Ranges select)
+    {
+        if (isEmpty())
+            return new RangeDeps(NO_RANGES, txnIds, NO_INTS);
+
+        try (RangeAndMapCollector collector = new RangeAndMapCollector(ensureSearchable().maxScanAndCheckpointMatches))
+        {
+            forEach(select, collector, collector, ranges, rangesToTxnIds, null);
+
+            if (collector.rangesCount == 0)
+                return new RangeDeps(NO_RANGES, NO_TXNIDS, NO_INTS);
+
+            if (collector.rangesCount == this.ranges.length)
+                return this;
+
+            Range[] ranges = collector.getRanges();
+            int[] rangesToTxnIds = collector.getRangesToTxnIds();
+            TxnId[] txnIds = trimUnusedValues(ranges, this.txnIds, rangesToTxnIds, TxnId[]::new);
+            return new RangeDeps(ranges, txnIds, rangesToTxnIds);
+        }
+    }
+
+    public RangeDeps with(RangeDeps that)
+    {
+        if (isEmpty() || that.isEmpty())
+            return isEmpty() ? that : this;
+
+        return linearUnion(
+                this.ranges, this.ranges.length, this.txnIds, this.txnIds.length, this.rangesToTxnIds, this.rangesToTxnIds.length,
+                that.ranges, that.ranges.length, that.txnIds, that.txnIds.length, that.rangesToTxnIds, that.rangesToTxnIds.length,
+                rangeComparator(), TxnId::compareTo,
+                cachedRanges(), cachedTxnIds(), cachedInts(),
+                (ranges, rangesLength, txnIds, txnIdsLength, out, outLength) ->
+                        new RangeDeps(cachedRanges().complete(ranges, rangesLength),
+                                cachedTxnIds().complete(txnIds, txnIdsLength),
+                                cachedInts().complete(out, outLength))
+        );
+    }
+
+    public RangeDeps without(Predicate<TxnId> remove)
+    {
+        return remove(this, ranges, txnIds, rangesToTxnIds, remove,
+                NONE, TxnId[]::new, ranges, RangeDeps::new);
+    }
+
+    public boolean contains(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId) >= 0;
+    }
+
+    public boolean isCoveredBy(Ranges covering)
+    {
+        // check that every entry intersects with some entry in covering
+        int prev = 0;
+        for (Range range : covering)
+        {
+            int start = SortedArrays.binarySearch(ranges, 0, ranges.length, range.start(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (start < 0) start = -1 - start;
+            int end = SortedArrays.binarySearch(ranges, 0, ranges.length, range.end(), (a, b) -> a.compareTo(b.start()), CEIL);
+            if (end < 0) end = -1 - end;
+            for (int i = prev; i < start ; ++i)
+            {
+                if (range.compareIntersecting(ranges[i]) != 0)
+                    return false;
+            }
+            prev = end;
+        }
+        return prev == ranges.length;
+    }
+
+    public List<TxnId> txnIds(Key key)
+    {
+        List<TxnId> result = new ArrayList<>();
+        forEachUniqueTxnId(key, result::add);
+        result.sort(TxnId::compareTo);
+        return result;
+    }
+
+    public List<TxnId> txnIds(Range key)
+    {
+        List<TxnId> result = new ArrayList<>();
+        forEachUniqueTxnId(key, result::add);
+        result.sort(TxnId::compareTo);
+        return result;
+    }
+
+    public TxnId txnId(int i)
+    {
+        return txnIds[i];
+    }
+
+    public int txnIdCount()
+    {
+        return txnIds.length;
+    }
+
+    public Range range(int i)
+    {
+        return ranges[i];
+    }
+
+    public int rangeCount()
+    {
+        return ranges.length;
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return this == that || (that instanceof RangeDeps && equals((RangeDeps)that));
+    }
+
+    public boolean equals(RangeDeps that)
+    {
+        return testEquality(this.ranges, this.txnIds, this.rangesToTxnIds, that.ranges, that.txnIds, that.rangesToTxnIds);
+    }
+
+    @Override
+    public String toString()
+    {
+        return RelationMultiMap.toSimpleString(ranges, txnIds, rangesToTxnIds);
+    }
+
+    @Nonnull
+    @Override
+    public Iterator<Map.Entry<Range, TxnId>> iterator()
+    {
+        return newIterator(ranges, txnIds, rangesToTxnIds);
+    }
+
+    private SearchableRangeList ensureSearchable()
+    {
+        if (searchable == null)
+            buildSearchable();
+        return searchable;
+    }
+
+    @DontInline
+    private void buildSearchable()
+    {
+        searchable = SearchableRangeList.build(ranges);
+    }
+
+    public boolean isSearchable()
+    {
+        return searchable != null;
+    }
+
+    public void buildSearchable(int maxScanDistance)
+    {
+        searchable = SearchableRangeList.build(ranges, maxScanDistance, ACCURATE, LINKS);
+    }
+
+    static class RangeCollector implements
+            IndexedRangeTriConsumer<Range[], int[], Object>,
+            IndexedTriConsumer<Range[], int[], Object>,
+            AutoCloseable
+    {
+        int[] oooBuffer;
+        Range[] rangesOut;
+        int oooCount, rangesCount;
+
+        RangeCollector(int maxScanAndCheckpointCount)
+        {
+            oooBuffer = cachedInts().getInts(maxScanAndCheckpointCount);
+            rangesOut = cachedRanges().get(32);
+        }
+
+        @Override
+        public void accept(Range[] o, int[] o2, Object o3, int index)
+        {
+            oooBuffer[oooCount++] = index;
+        }
+
+        @Override
+        public void accept(Range[] ranges, int[] rangesToTxnIds, Object o3, int fromIndex, int toIndex)
+        {
+            if (oooCount > 0)
+            {
+                Arrays.sort(oooBuffer, 0, oooCount);
+                int oooCount = Arrays.binarySearch(oooBuffer, 0, this.oooCount, fromIndex);
+                if (oooCount < 0) oooCount = -1 - oooCount;
+                copy(ranges, rangesToTxnIds, oooCount, fromIndex, toIndex);
+                this.oooCount = 0;
+            }
+            else if (fromIndex < toIndex)
+            {
+                copy(ranges, rangesToTxnIds, 0, fromIndex, toIndex);
+            }
+        }
+
+        protected void copy(Range[] ranges, int[] rangesToTxnIds, int oooCount, int start, int end)
+        {
+            int count = oooCount + (end - start);
+            if (rangesCount + count >= rangesOut.length)
+                rangesOut = cachedRanges().resize(rangesOut, rangesCount, rangesCount + count + (rangesCount /2));
+            for (int i = 0 ; i < oooCount ; ++i)
+                rangesOut[rangesCount++] = ranges[oooBuffer[i]];
+            for (int i = start ; i < end ; ++i)
+                rangesOut[rangesCount++] = ranges[i];
+        }
+
+        Range[] getRanges()
+        {
+            Range[] result = cachedRanges().completeAndDiscard(rangesOut, rangesCount);
+            rangesOut = null;
+            return result;
+        }
+
+        @Override
+        public void close()
+        {
+            if (oooBuffer != null)
+            {
+                cachedInts().forceDiscard(oooBuffer);
+                oooBuffer = null;
+            }
+            if (rangesOut != null)
+            {
+                cachedRanges().forceDiscard(rangesOut, rangesCount);
+                rangesOut = null;
+            }
+        }
+    }
+
+    static class RangeAndMapCollector extends RangeCollector
+    {
+        int[] headers;
+        int[] lists;
+        int headerCount, listOffset;
+
+        RangeAndMapCollector(int maxScanAndCheckpointCount)
+        {
+            super(maxScanAndCheckpointCount);
+            headers = cachedInts().getInts(32);
+            lists = cachedInts().getInts(32);
+        }
+
+        @Override
+        protected void copy(Range[] ranges, int[] rangesToTxnIds, int oooCount, int start, int end)
+        {
+            super.copy(ranges, rangesToTxnIds, oooCount, start, end);
+            int count = oooCount + (end - start);
+            if (headerCount + count >= headers.length)
+                headers = cachedInts().resize(headers, headerCount, headerCount + count + (headerCount /2));
+            for (int i = 0 ; i < oooCount ; ++i)
+            {
+                int ri = oooBuffer[i];
+                copyToDynamic(rangesToTxnIds, startOffset(ranges, rangesToTxnIds, ri), endOffset(rangesToTxnIds, ri));
+                headers[headerCount++] = listOffset;
+            }
+            int startOffset = startOffset(ranges, rangesToTxnIds, start);
+            for (int i = start ; i < end ; ++i)
+                headers[this.headerCount++] = listOffset + rangesToTxnIds[i] - startOffset;
+            copyToDynamic(rangesToTxnIds, startOffset, startOffset(ranges, rangesToTxnIds, end));
+        }
+
+        protected void copyToDynamic(int[] rangesToTxnIds, int start, int end)
+        {
+            int count = end - start;
+            if (count + listOffset >= lists.length)
+                lists = cachedInts().resize(lists, listOffset, listOffset + (listOffset /2) + count);
+            System.arraycopy(rangesToTxnIds, start, lists, listOffset, count);
+            listOffset += count;
+        }
+
+        public int[] getRangesToTxnIds()
+        {
+            int[] out = new int[headerCount + listOffset];
+            for (int i = 0; i < headerCount; ++i)
+                out[i] = headers[i] + headerCount;
+            System.arraycopy(lists, 0, out, headerCount, listOffset);
+            return out;
+        }
+    }
+
+    public static RangeDeps of(Map<TxnId, Ranges> txnIdRanges)
+    {
+        if (txnIdRanges.isEmpty())
+            return NONE;
+
+        try (BuilderByTxnId builder = new BuilderByTxnId(true))
+        {
+            for (Map.Entry<TxnId, Ranges> e : txnIdRanges.entrySet())
+            {
+                builder.nextKey(e.getKey());
+                Ranges ranges = e.getValue();
+                for (int i = 0 ; i < ranges.size() ; ++i)
+                    builder.add(ranges.get(i));
+            }
+            return builder.build();
+        }
+    }
+
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static class Builder extends AbstractBuilder<Range, TxnId, RangeDeps>
+    {
+        public Builder()
+        {
+            super(ADAPTER);
+        }
+
+        @Override
+        protected RangeDeps none()
+        {
+            return RangeDeps.NONE;
+        }
+
+        @Override
+        protected RangeDeps build(Range[] ranges, TxnId[] txnIds, int[] keyToValue)
+        {
+            return new RangeDeps(ranges, txnIds, keyToValue);
+        }
+    }
+
+    public static BuilderByTxnId byTxnIdBuilder(boolean hasOrderedRanges)
+    {
+        return new BuilderByTxnId(hasOrderedRanges);
+    }
+
+    public static class BuilderByTxnId extends AbstractBuilder<TxnId, Range, RangeDeps>
+    {
+        public BuilderByTxnId(boolean hasOrderedTxnId)

Review Comment:
   `hasOrderedRanges` - vestige of some older code?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] belliottsmith commented on pull request #21: Introduce RangeDeps

Posted by "belliottsmith (via GitHub)" <gi...@apache.org>.
belliottsmith commented on PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21#issuecomment-1404223969

   Thanks, agreed that it would be nice to expand test cases a little, but I think it is adequately covered for now, between the RangeDeps and burn tests. I believe I have some TODO to address improving coverage in the future, and if not I will add them.
   
   >  I hope that we'll be able to reuse it in other places
   
   This is obviously a long term idea, but the same logic can be applied very straightforwardly to sstables, bringing is efficient range queries, at least within a node if the primary key supports it. For system tables for range transaction state management at least this is a possibility I’ve considered. Maybe the structure will come in handy elsewhere too.
   
   > I'll keep it all in my head for a few more days in case I can come up with some improvement ideas
   
   thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] iamaleksey commented on a diff in pull request #21: Introduce RangeDeps

Posted by "iamaleksey (via GitHub)" <gi...@apache.org>.
iamaleksey commented on code in PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21#discussion_r1084101886


##########
accord-core/src/main/java/accord/primitives/KeyDeps.java:
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.Key;
+import accord.api.RoutingKey;
+import accord.utils.ArrayBuffers;
+import accord.utils.SymmetricComparator;
+
+import java.util.*;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.SortedArrays.Search.FAST;
+
+/**
+ * A collection of dependencies for a transaction, organised by the key the dependency is adopted via.
+ * An inverse map from TxnId to Key may also be constructed and stored in this collection.
+ */
+// TODO (desired, consider): switch to RoutingKey? Would mean adopting execution dependencies less precisely, but saving ser/deser of large keys
+public class KeyDeps implements Iterable<Map.Entry<Key, TxnId>>
+{
+    public static final KeyDeps NONE = new KeyDeps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+
+    public static class SerializerSupport
+    {
+        private SerializerSupport() {}
+
+        public static int keysToTxnIdsCount(KeyDeps deps)
+        {
+            return deps.keysToTxnIds.length;
+        }
+
+        public static int keysToTxnIds(KeyDeps deps, int idx)
+        {
+            return deps.keysToTxnIds[idx];
+        }
+
+        public static KeyDeps create(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
+        {
+            return new KeyDeps(keys, txnIds, keyToTxnId);
+        }
+    }
+
+    public static KeyDeps none(Keys keys)
+    {
+        int[] keysToTxnId = new int[keys.size()];
+        Arrays.fill(keysToTxnId, keys.size());
+        return new KeyDeps(keys, NO_TXNIDS, keysToTxnId);
+    }
+
+    /**
+     * Expects Command to be provided in TxnId order
+     */
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static class Builder extends AbstractBuilder<Key, TxnId, KeyDeps>
+    {
+        public Builder()
+        {
+            super(ADAPTER);
+        }
+
+        @Override
+        protected KeyDeps none()
+        {
+            return KeyDeps.NONE;
+        }
+
+        @Override
+        protected KeyDeps build(Key[] keys, TxnId[] txnIds, int[] keysToTxnIds)
+        {
+            return new KeyDeps(Keys.ofSorted(keys), txnIds, keysToTxnIds);
+        }
+    }
+
+    public static <T1, T2> KeyDeps merge(List<T1> merge, Function<T1, T2> getter1, Function<T2, KeyDeps> getter2)
+    {
+        try (LinearMerger<Key, TxnId, KeyDeps> linearMerger = new LinearMerger<>(ADAPTER))
+        {
+            int mergeIndex = 0, mergeSize = merge.size();
+            while (mergeIndex < mergeSize)
+            {
+                T2 intermediate = getter1.apply(merge.get(mergeIndex++));
+                if (intermediate == null)
+                    continue;
+
+                KeyDeps deps = getter2.apply(intermediate);
+                if (deps == null || deps.isEmpty())
+                    continue;
+
+                linearMerger.update(deps, deps.keys.keys, deps.txnIds, deps.keysToTxnIds);
+            }
+
+            return linearMerger.get(KeyDeps::new, NONE);
+        }
+    }
+
+    final Keys keys; // unique Keys
+    final TxnId[] txnIds; // unique TxnId
+
+    /**
+     * This represents a map of {@code Key -> [TxnId] } where each TxnId is actually a pointer into the txnIds array.
+     * The beginning of the array (the first keys.size() entries) are offsets into this array.
+     * <p/>
+     * Example:
+     * <p/>
+     * {@code
+     *   int keyIdx = keys.indexOf(key);
+     *   int startOfTxnOffset = keyIdx == 0 ? keys.size() : keyToTxnId[keyIdx - 1];
+     *   int endOfTxnOffset = keyToTxnId[keyIdx];
+     *   for (int i = startOfTxnOffset; i < endOfTxnOffset; i++)
+     *   {
+     *       TxnId id = txnIds[keyToTxnId[i]]
+     *       ...
+     *   }
+     * }
+     */
+    final int[] keysToTxnIds; // Key -> [TxnId]
+    // Lazy loaded in ensureTxnIdToKey()
+    int[] txnIdsToKeys; // TxnId -> [Key] TODO (low priority, efficiency): this could be a BTree?
+
+    KeyDeps(Key[] keys, TxnId[] txnIds, int[] keysToTxnIds)
+    {
+        this(Keys.ofSorted(keys), txnIds, keysToTxnIds);
+    }
+
+    KeyDeps(Keys keys, TxnId[] txnIds, int[] keysToTxnIds)
+    {
+        this.keys = keys;
+        this.txnIds = txnIds;
+        this.keysToTxnIds = keysToTxnIds;
+        if (!(keys.isEmpty() || keysToTxnIds[keys.size() - 1] == keysToTxnIds.length))
+            throw new IllegalArgumentException(String.format("Last key (%s) in keyToTxnId does not point (%d) to the end of the array (%d);\nkeyToTxnId=%s", keys.get(keys.size() - 1), keysToTxnIds[keys.size() - 1], keysToTxnIds.length, Arrays.toString(keysToTxnIds)));
+        checkValid(keys.keys, txnIds, keysToTxnIds);
+    }
+
+    public KeyDeps slice(Ranges ranges)
+    {
+        if (isEmpty())
+            return new KeyDeps(keys, txnIds, keysToTxnIds);
+
+        // TODO (low priority, efficiency): can slice in parallel with selecting keyToTxnId contents to avoid duplicate merging
+        Keys select = keys.slice(ranges);
+
+        if (select.isEmpty())
+            return new KeyDeps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+
+        if (select.size() == keys.size())
+            return new KeyDeps(keys, txnIds, keysToTxnIds);
+
+        int i = 0;
+        int offset = select.size();
+        for (int j = 0 ; j < select.size() ; ++j)
+        {
+            int findi = keys.findNext(i, select.get(j), FAST);
+            if (findi < 0)
+                continue;
+
+            i = findi;
+            offset += keysToTxnIds[i] - (i == 0 ? keys.size() : keysToTxnIds[i - 1]);
+        }
+
+        int[] src = keysToTxnIds;
+        int[] trg = new int[offset];
+
+        i = 0;
+        offset = select.size();
+        for (int j = 0 ; j < select.size() ; ++j)
+        {
+            int findi = keys.findNext(i, select.get(j), FAST);
+            if (findi >= 0)
+            {
+                i = findi;
+                int start = i == 0 ? keys.size() : src[i - 1];
+                int count = src[i] - start;
+                System.arraycopy(src, start, trg, offset, count);
+                offset += count;
+            }
+            trg[j] = offset;
+        }
+
+        TxnId[] txnIds = trimUnusedValues(select.keys, this.txnIds, trg, TxnId[]::new);
+        return new KeyDeps(select, txnIds, trg);
+    }
+
+    public KeyDeps with(KeyDeps that)
+    {
+        if (isEmpty() || that.isEmpty())
+            return isEmpty() ? that : this;
+
+        return linearUnion(
+                this.keys.keys, this.keys.keys.length, this.txnIds, this.txnIds.length, this.keysToTxnIds, this.keysToTxnIds.length,
+                that.keys.keys, that.keys.keys.length, that.txnIds, that.txnIds.length, that.keysToTxnIds, that.keysToTxnIds.length,
+                Key::compareTo, TxnId::compareTo,
+                cachedKeys(), cachedTxnIds(), cachedInts(),
+                (keys, keysLength, txnIds, txnIdsLength, out, outLength) ->
+                        new KeyDeps(Keys.ofSortedUnchecked(cachedKeys().complete(keys, keysLength)),
+                                cachedTxnIds().complete(txnIds, txnIdsLength),
+                                cachedInts().complete(out, outLength))
+                );
+    }
+
+    public KeyDeps without(Predicate<TxnId> remove)
+    {
+        return remove(this, keys.keys, txnIds, keysToTxnIds, remove,
+                NONE, TxnId[]::new, keys, KeyDeps::new);
+    }
+
+    public boolean contains(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId) >= 0;
+    }
+
+    // return true iff we map any keys to any txnId
+    // if the mapping is empty we return false, whether or not we have any keys or txnId by themselves
+    public boolean isEmpty()
+    {
+        return keysToTxnIds.length == keys.size();
+    }
+
+    public Keys someKeys(TxnId txnId)
+    {
+        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
+        if (txnIdIndex < 0)
+            return Keys.EMPTY;
+
+        ensureTxnIdToKey();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToKeys[txnIdIndex - 1];
+        int end = txnIdsToKeys[txnIdIndex];
+        if (start == end)
+            return Keys.EMPTY;
+
+        Key[] result = new Key[end - start];
+        for (int i = start ; i < end ; ++i)
+            result[i - start] = keys.get(txnIdsToKeys[i]);
+        return Keys.of(result);
+    }
+
+    public Unseekables<RoutingKey, ?> someUnseekables(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)
+            constructor.apply(RoutingKeys.EMPTY.keys);
+
+        ensureTxnIdToKey();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToKeys[txnIdIndex - 1];
+        int end = txnIdsToKeys[txnIdIndex];
+        RoutingKey[] result = new RoutingKey[end - start];
+        if (start == end)
+            constructor.apply(RoutingKeys.EMPTY.keys);
+
+        result[0] = keys.get(txnIdsToKeys[start]).toUnseekable();
+        int resultCount = 1;
+        for (int i = start + 1 ; i < end ; ++i)
+        {
+            RoutingKey next = keys.get(txnIdsToKeys[i]).toUnseekable();
+            if (!next.equals(result[resultCount - 1]))
+                result[resultCount++] = next;
+        }
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+        return constructor.apply(result);
+    }
+
+    void ensureTxnIdToKey()
+    {
+        if (txnIdsToKeys != null)
+            return;
+
+        txnIdsToKeys = invert(keysToTxnIds, keysToTxnIds.length, keys.size(), txnIds.length);
+    }
+
+    public void forEach(Ranges ranges, BiConsumer<Key, TxnId> forEach)
+    {
+        Routables.foldl(keys, ranges, (key, value, index) -> {
+            for (int t = startOffset(index), end = endOffset(index); t < end ; ++t)
+            {
+                TxnId txnId = txnIds[keysToTxnIds[t]];
+                forEach.accept(key, txnId);
+            }
+            return null;
+        }, null);
+    }
+
+    /**
+     * 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 forEach function to call on each unique {@link TxnId}
+     */
+    public void forEachUniqueTxnId(Ranges ranges, 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
+        // the property that forEach is called in TxnId order.
+        //TODO (expected, efficiency): reconsider this, probably not worth trying to save allocations at cost of multiple loop
+        //                             use BitSet, or perhaps extend so we can have no nested allocations when few bits
+        for (int offset = 0 ; offset < txnIds.length ; offset += 64)
+        {
+            long bitset = Routables.foldl(keys, ranges, (key, off, value, keyIndex) -> {
+                int index = startOffset(keyIndex);
+                int end = endOffset(keyIndex);
+                if (off > 0)
+                {
+                    // TODO (low priority, efficiency): interpolation search probably great here
+                    index = Arrays.binarySearch(keysToTxnIds, index, end, (int)off);
+                    if (index < 0)
+                        index = -1 - index;
+                }
+
+                while (index < end)
+                {
+                    long next = keysToTxnIds[index++] - off;
+                    if (next >= 64)
+                        break;
+                    value |= 1L << next;
+                }
+
+                return value;
+            }, offset, 0, -1L);
+
+            while (bitset != 0)
+            {
+                int i = Long.numberOfTrailingZeros(bitset);
+                TxnId txnId = txnIds[offset + i];
+                forEach.accept(txnId);
+                bitset ^= Long.lowestOneBit(bitset);
+            }
+        }
+    }
+
+    public void forEach(Key key, Consumer<TxnId> forEach)
+    {
+        int keyIndex = keys.indexOf(key);
+        if (keyIndex < 0)
+            return;
+
+        int index = startOffset(keyIndex);
+        int end = endOffset(keyIndex);
+        while (index < end)
+            forEach.accept(txnIds[keysToTxnIds[index++]]);
+    }
+
+    public Keys keys()
+    {
+        return keys;
+    }
+
+    public int txnIdCount()
+    {
+        return txnIds.length;
+    }
+
+    public int totalCount()
+    {
+        return keysToTxnIds.length - keys.size();
+    }
+
+    public TxnId txnId(int i)
+    {
+        return txnIds[i];
+    }
+
+    public Collection<TxnId> txnIds()
+    {
+        return Arrays.asList(txnIds);
+    }
+
+    public List<TxnId> txnIds(Key key)
+    {
+        int keyIndex = keys.indexOf(key);
+        if (keyIndex < 0)
+            return Collections.emptyList();
+
+        int start = startOffset(keyIndex);
+        int end = endOffset(keyIndex);
+        int size = end - start;
+        return txnIds(keysToTxnIds, start, size);
+    }
+
+    public List<TxnId> txnIds(Range range)
+    {
+        int startIndex = keys.indexOf(range.start());
+        if (startIndex < 0) startIndex = -1 - startIndex;
+        else if (!range.startInclusive()) ++startIndex;
+        int endIndex = keys.indexOf(range.end());
+        if (endIndex < 0) endIndex = -1 - endIndex;
+        else if (range.endInclusive()) ++endIndex;
+
+        if (startIndex == endIndex)
+            return Collections.emptyList();
+
+        int maxLength = Math.min(txnIds.length, startOffset(endIndex) - startOffset(startIndex));
+        int[] scratch = cachedInts().getInts(maxLength);
+        int count = 0;
+        for (int i = startIndex ; i < endIndex ; ++i)
+        {
+            int ri = startOffset(i), re = endOffset(i);
+            if (ri == re) continue;
+            if (count == 0)
+            {
+                count = re - ri;
+                System.arraycopy(keysToTxnIds, ri, scratch, 0, count);
+            }
+            else
+            {
+                if (count == maxLength)
+                    break;
+
+                System.arraycopy(scratch, 0, scratch, maxLength - count, count);
+                int li = maxLength - count, le = maxLength;
+                count = 0;
+                while (li < le && ri < re)
+                {
+                    int c = keysToTxnIds[ri] - scratch[li];
+                    if (c <= 0)
+                    {
+                        scratch[count++] = scratch[li++];
+                        ri += c == 0 ? 1 : 0;
+                    }
+                    else
+                    {
+                        scratch[count++] = keysToTxnIds[ri++];
+                    }
+                }
+                while (li < le)
+                    scratch[count++] = scratch[li++];
+                while (ri < re)
+                    scratch[count++] = keysToTxnIds[ri++];
+            }
+        }
+
+        int[] ids = cachedInts().completeAndDiscard(scratch, count);
+        return txnIds(ids, 0, count);
+    }
+
+    private List<TxnId> txnIds(int[] ids, int start, int size)
+    {
+        return new AbstractList<TxnId>()
+        {
+            @Override
+            public TxnId get(int index)
+            {
+                if (index > size)
+                    throw new IndexOutOfBoundsException();
+                return txnIds[ids[start + index]];
+            }
+
+            @Override
+            public int size()
+            {
+                return size;
+            }
+        };
+    }
+
+    private int startOffset(int keyIndex)
+    {
+        return keyIndex == 0 ? keys.size() : keysToTxnIds[keyIndex - 1];
+    }
+
+    private int endOffset(int keyIndex)
+    {
+        return keysToTxnIds[keyIndex];
+    }
+
+    public boolean equals(Object that)
+    {
+        return this == that || (that instanceof KeyDeps && equals((KeyDeps)that));
+    }
+
+    public boolean equals(KeyDeps that)
+    {
+        return testEquality(this.keys.keys, this.txnIds, this.keysToTxnIds, that.keys.keys, that.txnIds, that.keysToTxnIds);
+    }
+
+    public Iterator<Map.Entry<Key, TxnId>> iterator()

Review Comment:
   And here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra-accord] iamaleksey commented on a diff in pull request #21: Introduce RangeDeps

Posted by "iamaleksey (via GitHub)" <gi...@apache.org>.
iamaleksey commented on code in PR #21:
URL: https://github.com/apache/cassandra-accord/pull/21#discussion_r1084100412


##########
accord-core/src/main/java/accord/primitives/KeyDeps.java:
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.Key;
+import accord.api.RoutingKey;
+import accord.utils.ArrayBuffers;
+import accord.utils.SymmetricComparator;
+
+import java.util.*;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static accord.utils.ArrayBuffers.*;
+import static accord.utils.RelationMultiMap.*;
+import static accord.utils.SortedArrays.Search.FAST;
+
+/**
+ * A collection of dependencies for a transaction, organised by the key the dependency is adopted via.
+ * An inverse map from TxnId to Key may also be constructed and stored in this collection.
+ */
+// TODO (desired, consider): switch to RoutingKey? Would mean adopting execution dependencies less precisely, but saving ser/deser of large keys
+public class KeyDeps implements Iterable<Map.Entry<Key, TxnId>>
+{
+    public static final KeyDeps NONE = new KeyDeps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+
+    public static class SerializerSupport
+    {
+        private SerializerSupport() {}
+
+        public static int keysToTxnIdsCount(KeyDeps deps)
+        {
+            return deps.keysToTxnIds.length;
+        }
+
+        public static int keysToTxnIds(KeyDeps deps, int idx)
+        {
+            return deps.keysToTxnIds[idx];
+        }
+
+        public static KeyDeps create(Keys keys, TxnId[] txnIds, int[] keyToTxnId)
+        {
+            return new KeyDeps(keys, txnIds, keyToTxnId);
+        }
+    }
+
+    public static KeyDeps none(Keys keys)
+    {
+        int[] keysToTxnId = new int[keys.size()];
+        Arrays.fill(keysToTxnId, keys.size());
+        return new KeyDeps(keys, NO_TXNIDS, keysToTxnId);
+    }
+
+    /**
+     * Expects Command to be provided in TxnId order
+     */
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static class Builder extends AbstractBuilder<Key, TxnId, KeyDeps>
+    {
+        public Builder()
+        {
+            super(ADAPTER);
+        }
+
+        @Override
+        protected KeyDeps none()
+        {
+            return KeyDeps.NONE;
+        }
+
+        @Override
+        protected KeyDeps build(Key[] keys, TxnId[] txnIds, int[] keysToTxnIds)
+        {
+            return new KeyDeps(Keys.ofSorted(keys), txnIds, keysToTxnIds);
+        }
+    }
+
+    public static <T1, T2> KeyDeps merge(List<T1> merge, Function<T1, T2> getter1, Function<T2, KeyDeps> getter2)
+    {
+        try (LinearMerger<Key, TxnId, KeyDeps> linearMerger = new LinearMerger<>(ADAPTER))
+        {
+            int mergeIndex = 0, mergeSize = merge.size();
+            while (mergeIndex < mergeSize)
+            {
+                T2 intermediate = getter1.apply(merge.get(mergeIndex++));
+                if (intermediate == null)
+                    continue;
+
+                KeyDeps deps = getter2.apply(intermediate);
+                if (deps == null || deps.isEmpty())
+                    continue;
+
+                linearMerger.update(deps, deps.keys.keys, deps.txnIds, deps.keysToTxnIds);
+            }
+
+            return linearMerger.get(KeyDeps::new, NONE);
+        }
+    }
+
+    final Keys keys; // unique Keys
+    final TxnId[] txnIds; // unique TxnId
+
+    /**
+     * This represents a map of {@code Key -> [TxnId] } where each TxnId is actually a pointer into the txnIds array.
+     * The beginning of the array (the first keys.size() entries) are offsets into this array.
+     * <p/>
+     * Example:
+     * <p/>
+     * {@code
+     *   int keyIdx = keys.indexOf(key);
+     *   int startOfTxnOffset = keyIdx == 0 ? keys.size() : keyToTxnId[keyIdx - 1];
+     *   int endOfTxnOffset = keyToTxnId[keyIdx];
+     *   for (int i = startOfTxnOffset; i < endOfTxnOffset; i++)
+     *   {
+     *       TxnId id = txnIds[keyToTxnId[i]]
+     *       ...
+     *   }
+     * }
+     */
+    final int[] keysToTxnIds; // Key -> [TxnId]
+    // Lazy loaded in ensureTxnIdToKey()
+    int[] txnIdsToKeys; // TxnId -> [Key] TODO (low priority, efficiency): this could be a BTree?
+
+    KeyDeps(Key[] keys, TxnId[] txnIds, int[] keysToTxnIds)
+    {
+        this(Keys.ofSorted(keys), txnIds, keysToTxnIds);
+    }
+
+    KeyDeps(Keys keys, TxnId[] txnIds, int[] keysToTxnIds)
+    {
+        this.keys = keys;
+        this.txnIds = txnIds;
+        this.keysToTxnIds = keysToTxnIds;
+        if (!(keys.isEmpty() || keysToTxnIds[keys.size() - 1] == keysToTxnIds.length))
+            throw new IllegalArgumentException(String.format("Last key (%s) in keyToTxnId does not point (%d) to the end of the array (%d);\nkeyToTxnId=%s", keys.get(keys.size() - 1), keysToTxnIds[keys.size() - 1], keysToTxnIds.length, Arrays.toString(keysToTxnIds)));
+        checkValid(keys.keys, txnIds, keysToTxnIds);
+    }
+
+    public KeyDeps slice(Ranges ranges)
+    {
+        if (isEmpty())
+            return new KeyDeps(keys, txnIds, keysToTxnIds);
+
+        // TODO (low priority, efficiency): can slice in parallel with selecting keyToTxnId contents to avoid duplicate merging
+        Keys select = keys.slice(ranges);
+
+        if (select.isEmpty())
+            return new KeyDeps(Keys.EMPTY, NO_TXNIDS, NO_INTS);
+
+        if (select.size() == keys.size())
+            return new KeyDeps(keys, txnIds, keysToTxnIds);
+
+        int i = 0;
+        int offset = select.size();
+        for (int j = 0 ; j < select.size() ; ++j)
+        {
+            int findi = keys.findNext(i, select.get(j), FAST);
+            if (findi < 0)
+                continue;
+
+            i = findi;
+            offset += keysToTxnIds[i] - (i == 0 ? keys.size() : keysToTxnIds[i - 1]);
+        }
+
+        int[] src = keysToTxnIds;
+        int[] trg = new int[offset];
+
+        i = 0;
+        offset = select.size();
+        for (int j = 0 ; j < select.size() ; ++j)
+        {
+            int findi = keys.findNext(i, select.get(j), FAST);
+            if (findi >= 0)
+            {
+                i = findi;
+                int start = i == 0 ? keys.size() : src[i - 1];
+                int count = src[i] - start;
+                System.arraycopy(src, start, trg, offset, count);
+                offset += count;
+            }
+            trg[j] = offset;
+        }
+
+        TxnId[] txnIds = trimUnusedValues(select.keys, this.txnIds, trg, TxnId[]::new);
+        return new KeyDeps(select, txnIds, trg);
+    }
+
+    public KeyDeps with(KeyDeps that)
+    {
+        if (isEmpty() || that.isEmpty())
+            return isEmpty() ? that : this;
+
+        return linearUnion(
+                this.keys.keys, this.keys.keys.length, this.txnIds, this.txnIds.length, this.keysToTxnIds, this.keysToTxnIds.length,
+                that.keys.keys, that.keys.keys.length, that.txnIds, that.txnIds.length, that.keysToTxnIds, that.keysToTxnIds.length,
+                Key::compareTo, TxnId::compareTo,
+                cachedKeys(), cachedTxnIds(), cachedInts(),
+                (keys, keysLength, txnIds, txnIdsLength, out, outLength) ->
+                        new KeyDeps(Keys.ofSortedUnchecked(cachedKeys().complete(keys, keysLength)),
+                                cachedTxnIds().complete(txnIds, txnIdsLength),
+                                cachedInts().complete(out, outLength))
+                );
+    }
+
+    public KeyDeps without(Predicate<TxnId> remove)
+    {
+        return remove(this, keys.keys, txnIds, keysToTxnIds, remove,
+                NONE, TxnId[]::new, keys, KeyDeps::new);
+    }
+
+    public boolean contains(TxnId txnId)
+    {
+        return Arrays.binarySearch(txnIds, txnId) >= 0;
+    }
+
+    // return true iff we map any keys to any txnId
+    // if the mapping is empty we return false, whether or not we have any keys or txnId by themselves
+    public boolean isEmpty()
+    {
+        return keysToTxnIds.length == keys.size();
+    }
+
+    public Keys someKeys(TxnId txnId)
+    {
+        int txnIdIndex = Arrays.binarySearch(txnIds, txnId);
+        if (txnIdIndex < 0)
+            return Keys.EMPTY;
+
+        ensureTxnIdToKey();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToKeys[txnIdIndex - 1];
+        int end = txnIdsToKeys[txnIdIndex];
+        if (start == end)
+            return Keys.EMPTY;
+
+        Key[] result = new Key[end - start];
+        for (int i = start ; i < end ; ++i)
+            result[i - start] = keys.get(txnIdsToKeys[i]);
+        return Keys.of(result);
+    }
+
+    public Unseekables<RoutingKey, ?> someUnseekables(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)
+            constructor.apply(RoutingKeys.EMPTY.keys);
+
+        ensureTxnIdToKey();
+
+        int start = txnIdIndex == 0 ? txnIds.length : txnIdsToKeys[txnIdIndex - 1];
+        int end = txnIdsToKeys[txnIdIndex];
+        RoutingKey[] result = new RoutingKey[end - start];
+        if (start == end)
+            constructor.apply(RoutingKeys.EMPTY.keys);
+
+        result[0] = keys.get(txnIdsToKeys[start]).toUnseekable();
+        int resultCount = 1;
+        for (int i = start + 1 ; i < end ; ++i)
+        {
+            RoutingKey next = keys.get(txnIdsToKeys[i]).toUnseekable();
+            if (!next.equals(result[resultCount - 1]))
+                result[resultCount++] = next;
+        }
+
+        if (resultCount < result.length)
+            result = Arrays.copyOf(result, resultCount);
+        return constructor.apply(result);
+    }
+
+    void ensureTxnIdToKey()
+    {
+        if (txnIdsToKeys != null)
+            return;
+
+        txnIdsToKeys = invert(keysToTxnIds, keysToTxnIds.length, keys.size(), txnIds.length);
+    }
+
+    public void forEach(Ranges ranges, BiConsumer<Key, TxnId> forEach)
+    {
+        Routables.foldl(keys, ranges, (key, value, index) -> {
+            for (int t = startOffset(index), end = endOffset(index); t < end ; ++t)
+            {
+                TxnId txnId = txnIds[keysToTxnIds[t]];
+                forEach.accept(key, txnId);
+            }
+            return null;
+        }, null);
+    }
+
+    /**
+     * 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 forEach function to call on each unique {@link TxnId}
+     */
+    public void forEachUniqueTxnId(Ranges ranges, 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
+        // the property that forEach is called in TxnId order.
+        //TODO (expected, efficiency): reconsider this, probably not worth trying to save allocations at cost of multiple loop
+        //                             use BitSet, or perhaps extend so we can have no nested allocations when few bits
+        for (int offset = 0 ; offset < txnIds.length ; offset += 64)
+        {
+            long bitset = Routables.foldl(keys, ranges, (key, off, value, keyIndex) -> {
+                int index = startOffset(keyIndex);
+                int end = endOffset(keyIndex);
+                if (off > 0)
+                {
+                    // TODO (low priority, efficiency): interpolation search probably great here
+                    index = Arrays.binarySearch(keysToTxnIds, index, end, (int)off);
+                    if (index < 0)
+                        index = -1 - index;
+                }
+
+                while (index < end)
+                {
+                    long next = keysToTxnIds[index++] - off;
+                    if (next >= 64)
+                        break;
+                    value |= 1L << next;
+                }
+
+                return value;
+            }, offset, 0, -1L);
+
+            while (bitset != 0)
+            {
+                int i = Long.numberOfTrailingZeros(bitset);
+                TxnId txnId = txnIds[offset + i];
+                forEach.accept(txnId);
+                bitset ^= Long.lowestOneBit(bitset);
+            }
+        }
+    }
+
+    public void forEach(Key key, Consumer<TxnId> forEach)
+    {
+        int keyIndex = keys.indexOf(key);
+        if (keyIndex < 0)
+            return;
+
+        int index = startOffset(keyIndex);
+        int end = endOffset(keyIndex);
+        while (index < end)
+            forEach.accept(txnIds[keysToTxnIds[index++]]);
+    }
+
+    public Keys keys()
+    {
+        return keys;
+    }
+
+    public int txnIdCount()
+    {
+        return txnIds.length;
+    }
+
+    public int totalCount()
+    {
+        return keysToTxnIds.length - keys.size();
+    }
+
+    public TxnId txnId(int i)
+    {
+        return txnIds[i];
+    }
+
+    public Collection<TxnId> txnIds()
+    {
+        return Arrays.asList(txnIds);
+    }
+
+    public List<TxnId> txnIds(Key key)
+    {
+        int keyIndex = keys.indexOf(key);
+        if (keyIndex < 0)
+            return Collections.emptyList();
+
+        int start = startOffset(keyIndex);
+        int end = endOffset(keyIndex);
+        int size = end - start;
+        return txnIds(keysToTxnIds, start, size);
+    }
+
+    public List<TxnId> txnIds(Range range)
+    {
+        int startIndex = keys.indexOf(range.start());
+        if (startIndex < 0) startIndex = -1 - startIndex;
+        else if (!range.startInclusive()) ++startIndex;
+        int endIndex = keys.indexOf(range.end());
+        if (endIndex < 0) endIndex = -1 - endIndex;
+        else if (range.endInclusive()) ++endIndex;
+
+        if (startIndex == endIndex)
+            return Collections.emptyList();
+
+        int maxLength = Math.min(txnIds.length, startOffset(endIndex) - startOffset(startIndex));
+        int[] scratch = cachedInts().getInts(maxLength);
+        int count = 0;
+        for (int i = startIndex ; i < endIndex ; ++i)
+        {
+            int ri = startOffset(i), re = endOffset(i);
+            if (ri == re) continue;
+            if (count == 0)
+            {
+                count = re - ri;
+                System.arraycopy(keysToTxnIds, ri, scratch, 0, count);
+            }
+            else
+            {
+                if (count == maxLength)
+                    break;
+
+                System.arraycopy(scratch, 0, scratch, maxLength - count, count);
+                int li = maxLength - count, le = maxLength;
+                count = 0;
+                while (li < le && ri < re)
+                {
+                    int c = keysToTxnIds[ri] - scratch[li];
+                    if (c <= 0)
+                    {
+                        scratch[count++] = scratch[li++];
+                        ri += c == 0 ? 1 : 0;
+                    }
+                    else
+                    {
+                        scratch[count++] = keysToTxnIds[ri++];
+                    }
+                }
+                while (li < le)
+                    scratch[count++] = scratch[li++];
+                while (ri < re)
+                    scratch[count++] = keysToTxnIds[ri++];
+            }
+        }
+
+        int[] ids = cachedInts().completeAndDiscard(scratch, count);
+        return txnIds(ids, 0, count);
+    }
+
+    private List<TxnId> txnIds(int[] ids, int start, int size)
+    {
+        return new AbstractList<TxnId>()
+        {
+            @Override
+            public TxnId get(int index)
+            {
+                if (index > size)
+                    throw new IndexOutOfBoundsException();
+                return txnIds[ids[start + index]];
+            }
+
+            @Override
+            public int size()
+            {
+                return size;
+            }
+        };
+    }
+
+    private int startOffset(int keyIndex)
+    {
+        return keyIndex == 0 ? keys.size() : keysToTxnIds[keyIndex - 1];
+    }
+
+    private int endOffset(int keyIndex)
+    {
+        return keysToTxnIds[keyIndex];
+    }
+
+    public boolean equals(Object that)

Review Comment:
   Plus a missing `@Override` here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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