diff --git a/accord-core/src/main/java/accord/api/Key.java b/accord-core/src/main/java/accord/api/Key.java index 1504b406c..82dfcdc94 100644 --- a/accord-core/src/main/java/accord/api/Key.java +++ b/accord-core/src/main/java/accord/api/Key.java @@ -18,6 +18,8 @@ package accord.api; +import javax.annotation.Nonnull; + import accord.primitives.Range; import accord.primitives.RoutableKey; import accord.primitives.Seekable; @@ -35,4 +37,6 @@ public interface Key extends Seekable, RoutableKey @Override default Range asRange() { throw new UnsupportedOperationException(); } + + default int compareAsRoutingKey(@Nonnull RoutingKey that) { return toUnseekable().compareTo(that); } } diff --git a/accord-core/src/main/java/accord/api/Read.java b/accord-core/src/main/java/accord/api/Read.java index fe0d94034..89ba6b027 100644 --- a/accord-core/src/main/java/accord/api/Read.java +++ b/accord-core/src/main/java/accord/api/Read.java @@ -19,6 +19,7 @@ package accord.api; import accord.local.SafeCommandStore; +import accord.primitives.Participants; import accord.primitives.Ranges; import accord.primitives.Seekable; import accord.primitives.Seekables; @@ -33,5 +34,7 @@ public interface Read Seekables keys(); AsyncChain read(Seekable key, SafeCommandStore commandStore, Timestamp executeAt, DataStore store); Read slice(Ranges ranges); + Read intersecting(Participants participants); Read merge(Read other); + } diff --git a/accord-core/src/main/java/accord/api/Update.java b/accord-core/src/main/java/accord/api/Update.java index 8a56efcc3..e9b6f8784 100644 --- a/accord-core/src/main/java/accord/api/Update.java +++ b/accord-core/src/main/java/accord/api/Update.java @@ -18,6 +18,7 @@ package accord.api; +import accord.primitives.Participants; import accord.primitives.Ranges; import accord.primitives.Seekables; import accord.primitives.Timestamp; @@ -35,5 +36,7 @@ public interface Update // null is provided only if nothing was read Write apply(Timestamp executeAt, @Nullable Data data); Update slice(Ranges ranges); + Update intersecting(Participants participants); Update merge(Update other); + } diff --git a/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java b/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java index b5c5b49d8..fb74f4f7d 100644 --- a/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java +++ b/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java @@ -18,9 +18,9 @@ package accord.coordinate; -import java.util.LinkedHashMap; +import java.util.Collection; import java.util.Map; -import java.util.Set; +import java.util.TreeMap; import java.util.function.BiConsumer; import javax.annotation.Nullable; @@ -103,7 +103,7 @@ else if (tracker.recordSuccess(from) == Success) private Topologies topologies; private boolean initialRoundIsDone; private ExtraEpochs extraEpochs; - private Map debug = Invariants.debug() ? new LinkedHashMap<>() : null; + private Map debug = Invariants.debug() ? new TreeMap<>() : null; AbstractCoordinatePreAccept(Node node, FullRoute route, TxnId txnId) { @@ -124,7 +124,7 @@ final void start() } abstract Seekables keysOrRanges(); - abstract void contact(Set nodes, Topologies topologies, Callback callback); + abstract void contact(Collection nodes, Topologies topologies, Callback callback); abstract void onSuccessInternal(Id from, R reply); /** * The tracker for the extra rounds only is provided by the AbstractCoordinatePreAccept, so we expect a boolean back diff --git a/accord-core/src/main/java/accord/coordinate/Barrier.java b/accord-core/src/main/java/accord/coordinate/Barrier.java index a9bce0bfa..fbc9bcbd1 100644 --- a/accord-core/src/main/java/accord/coordinate/Barrier.java +++ b/accord-core/src/main/java/accord/coordinate/Barrier.java @@ -259,7 +259,7 @@ class ExistingTransactionCheck extends AsyncResults.AbstractResult i @Override public BarrierTxn apply(SafeCommandStore safeStore) { - // TODO (required): consider these semantics + // TODO (required, consider): consider these semantics BarrierTxn found = safeStore.mapReduceFull( seekables, safeStore.ranges().allAfter(minEpoch), diff --git a/accord-core/src/main/java/accord/coordinate/CollectDeps.java b/accord-core/src/main/java/accord/coordinate/CollectDeps.java index 2a59716e9..31e2308bb 100644 --- a/accord-core/src/main/java/accord/coordinate/CollectDeps.java +++ b/accord-core/src/main/java/accord/coordinate/CollectDeps.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.function.BiConsumer; +import accord.api.RoutingKey; import accord.coordinate.tracking.QuorumTracker; import accord.local.CommandStore; import accord.local.Node; @@ -39,7 +40,7 @@ public class CollectDeps implements Callback { final Node node; final TxnId txnId; - final Route route; + final RoutingKey homeKey; final Timestamp executeAt; @@ -48,25 +49,25 @@ public class CollectDeps implements Callback private final BiConsumer callback; private boolean isDone; - CollectDeps(Node node, Topologies topologies, TxnId txnId, FullRoute route, Timestamp executeAt, BiConsumer callback) + CollectDeps(Node node, Topologies topologies, TxnId txnId, RoutingKey homeKey, Timestamp executeAt, BiConsumer callback) { this.node = node; this.txnId = txnId; - this.route = route; + this.homeKey = homeKey; this.executeAt = executeAt; this.callback = callback; this.oks = new ArrayList<>(); this.tracker = new QuorumTracker(topologies); } - public static void withDeps(Node node, TxnId txnId, FullRoute route, Seekables keysOrRanges, Timestamp executeAt, BiConsumer callback) + public static void withDeps(Node node, TxnId txnId, FullRoute fullRoute, Unseekables sendTo, Seekables keysOrRanges, Timestamp executeAt, BiConsumer callback) { - Topologies topologies = node.topology().withUnsyncedEpochs(route, txnId, executeAt); - CollectDeps collect = new CollectDeps(node, topologies, txnId, route, executeAt, callback); + Topologies topologies = node.topology().withUnsyncedEpochs(sendTo, txnId, executeAt); + CollectDeps collect = new CollectDeps(node, topologies, txnId, fullRoute.homeKey(), executeAt, callback); CommandStore store = CommandStore.maybeCurrent(); if (store == null) - store = node.commandStores().select(route); - node.send(collect.tracker.nodes(), to -> new GetDeps(to, topologies, route, txnId, keysOrRanges, executeAt), + store = node.commandStores().select(fullRoute); + node.send(collect.tracker.nodes(), to -> new GetDeps(to, topologies, fullRoute, txnId, keysOrRanges, executeAt), store, collect); } @@ -90,7 +91,7 @@ public void onFailure(Id from, Throwable failure) if (tracker.recordFailure(from) == Failed) { isDone = true; - callback.accept(null, new Timeout(txnId, route.homeKey())); + callback.accept(null, new Timeout(txnId, homeKey)); } } diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java b/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java index fa94afbbc..60e3ff347 100644 --- a/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java +++ b/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java @@ -19,8 +19,8 @@ package accord.coordinate; import java.util.ArrayList; +import java.util.Collection; import java.util.List; -import java.util.Set; import accord.api.Result; import accord.coordinate.tracking.QuorumTracker; @@ -90,7 +90,7 @@ public static AsyncResult coordinate(Node node, FullRoute route, TxnI } @Override - void contact(Set nodes, Topologies topologies, Callback callback) + void contact(Collection nodes, Topologies topologies, Callback callback) { CommandStore commandStore = CommandStore.maybeCurrent(); if (commandStore == null) commandStore = node.commandStores().select(route.homeKey()); diff --git a/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java b/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java index 1c19fc606..a5cd3807a 100644 --- a/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java +++ b/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java @@ -19,8 +19,8 @@ package accord.coordinate; import java.util.ArrayList; +import java.util.Collection; import java.util.List; -import java.util.Set; import accord.coordinate.tracking.FastPathTracker; import accord.local.CommandStore; @@ -61,13 +61,13 @@ abstract class CoordinatePreAccept extends AbstractCoordinatePreAccept route, Topologies topologies) { - super(node, route, txnId); + super(node, route, txnId, topologies); this.tracker = new FastPathTracker(topologies); this.oks = new ArrayList<>(topologies.estimateUniqueNodes()); this.txn = txn; } - void contact(Set nodes, Topologies topologies, Callback callback) + void contact(Collection nodes, Topologies topologies, Callback callback) { // TODO (desired, efficiency): consider sending only to electorate of most recent topology (as only these PreAccept votes matter) // note that we must send to all replicas of old topology, as electorate may not be reachable @@ -141,7 +141,7 @@ void onNewEpochTopologyMismatch(TopologyMismatch mismatch) * We cannot execute the transaction because the execution epoch's topology no longer contains all of the * participating keys/ranges, so we propose that the transaction is invalidated in its coordination epoch */ - Propose.Invalidate.proposeInvalidate(node, new Ballot(node.uniqueNow()), txnId, route.someParticipatingKey(), (outcome, failure) -> { + Propose.Invalidate.proposeInvalidate(node, new Ballot(node.uniqueNow()), txnId, route.homeKey(), (outcome, failure) -> { if (failure != null) mismatch.addSuppressed(failure); accept(null, mismatch); diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java b/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java index 61b2e1758..4f2a4d23a 100644 --- a/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java +++ b/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java @@ -18,6 +18,7 @@ package accord.coordinate; +import accord.coordinate.ExecuteSyncPoint.ExecuteExclusiveSyncPoint; import accord.coordinate.tracking.AppliedTracker; import accord.local.Node; import accord.messages.Callback; @@ -28,11 +29,18 @@ import accord.primitives.SyncPoint; import accord.utils.async.AsyncResult; -public class CoordinateShardDurable extends ExecuteSyncPoint implements Callback +public class CoordinateShardDurable extends ExecuteExclusiveSyncPoint implements Callback { private CoordinateShardDurable(Node node, SyncPoint exclusiveSyncPoint) { - super(node, new AppliedTracker(node.topology().forEpoch(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.sourceEpoch())), exclusiveSyncPoint); + super(node, exclusiveSyncPoint, AppliedTracker::new); + addCallback((success, fail) -> { + if (fail == null) + { + node.configService().reportEpochRedundant(syncPoint.keysOrRanges, syncPoint.syncId.epoch()); + node.send(tracker.nodes(), new SetShardDurable(syncPoint)); + } + }); } public static AsyncResult> coordinate(Node node, SyncPoint exclusiveSyncPoint) @@ -46,12 +54,4 @@ protected void start() { node.send(tracker.nodes(), to -> new WaitUntilApplied(to, tracker.topologies(), syncPoint.syncId, syncPoint.keysOrRanges, syncPoint.syncId.epoch()), this); } - - @Override - protected void onSuccess() - { - node.configService().reportEpochRedundant(syncPoint.keysOrRanges, syncPoint.syncId.epoch()); - node.send(tracker.nodes(), new SetShardDurable(syncPoint)); - super.onSuccess(); - } } diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java b/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java index cf500e893..11ddca5c2 100644 --- a/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java +++ b/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java @@ -24,6 +24,7 @@ import org.slf4j.LoggerFactory; import accord.coordinate.CoordinationAdapter.Adapters; +import accord.coordinate.CoordinationAdapter.Adapters.SyncPointAdapter; import accord.local.Node; import accord.messages.Apply; import accord.messages.PreAccept.PreAcceptOk; @@ -38,10 +39,7 @@ import accord.primitives.TxnId; import accord.topology.Topologies; import accord.utils.async.AsyncResult; -import accord.utils.async.AsyncResults; -import static accord.coordinate.CoordinationAdapter.Invoke.execute; -import static accord.coordinate.CoordinationAdapter.Invoke.propose; import static accord.coordinate.ExecutePath.FAST; import static accord.coordinate.Propose.Invalidate.proposeAndCommitInvalidate; import static accord.primitives.Timestamp.mergeMax; @@ -62,9 +60,9 @@ public class CoordinateSyncPoint> extends CoordinatePr final CoordinationAdapter> adapter; - private CoordinateSyncPoint(Node node, TxnId txnId, Txn txn, FullRoute route, CoordinationAdapter> adapter) + private CoordinateSyncPoint(Node node, TxnId txnId, Topologies topologies, Txn txn, FullRoute route, CoordinationAdapter> adapter) { - super(node, txnId, txn, route, node.topology().withOpenEpochs(route, txnId, txnId)); + super(node, txnId, txn, route, topologies); this.adapter = adapter; } @@ -88,21 +86,18 @@ private CoordinateSyncPoint(Node node, TxnId txnId, Txn txn, FullRoute route, return coordinate(node, Kind.SyncPoint, keysOrRanges, Adapters.inclusiveSyncPointBlocking()); } - public static > AsyncResult> coordinate(Node node, Kind kind, S keysOrRanges, CoordinationAdapter> adapter) + public static > AsyncResult> coordinate(Node node, Kind kind, S keysOrRanges, SyncPointAdapter adapter) { checkArgument(kind == Kind.SyncPoint || kind == ExclusiveSyncPoint); TxnId txnId = node.nextTxnId(kind, keysOrRanges.domain()); return node.withEpoch(txnId.epoch(), () -> coordinate(node, txnId, keysOrRanges, adapter)).beginAsResult(); } - private static > AsyncResult> coordinate(Node node, TxnId txnId, S keysOrRanges, CoordinationAdapter> adapter) + private static > AsyncResult> coordinate(Node node, TxnId txnId, S keysOrRanges, SyncPointAdapter adapter) { checkArgument(txnId.kind() == Kind.SyncPoint || txnId.kind() == ExclusiveSyncPoint); FullRoute route = node.computeRoute(txnId, keysOrRanges); - TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(txnId.epoch()), txnId, route.homeKey(), keysOrRanges); - if (mismatch != null) - return AsyncResults.failure(mismatch); - CoordinateSyncPoint coordinate = new CoordinateSyncPoint<>(node, txnId, node.agent().emptyTxn(txnId.kind(), keysOrRanges), route, adapter); + CoordinateSyncPoint coordinate = new CoordinateSyncPoint<>(node, txnId, adapter.forDecision(node, route, txnId), node.agent().emptyTxn(txnId.kind(), keysOrRanges), route, adapter); coordinate.start(); return coordinate; } @@ -124,13 +119,10 @@ void onPreAccepted(Topologies topologies, Timestamp executeAt, List } else { - // we don't need to fetch deps from Accept replies, so we don't need to contact unsynced epochs - topologies = node.topology().forEpoch(route, txnId.epoch()); - // TODO (required): consider the required semantics of a SyncPoint if (tracker.hasFastPathAccepted() && txnId.kind() == Kind.SyncPoint) - execute(adapter, node, topologies, route, FAST, txnId, txn, txnId, deps, this); + adapter.execute(node, topologies, route, FAST, txnId, txn, txnId, deps, this); else - propose(adapter, node, topologies, route, Ballot.ZERO, txnId, txn, executeAt, deps, this); + adapter.propose(node, topologies, route, Ballot.ZERO, txnId, txn, executeAt, deps, this); } } diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java b/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java index 0b05657de..3dd28c2e5 100644 --- a/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java +++ b/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java @@ -36,8 +36,6 @@ import accord.utils.async.AsyncResults; import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue; -import static accord.coordinate.CoordinationAdapter.Invoke.execute; -import static accord.coordinate.CoordinationAdapter.Invoke.propose; import static accord.coordinate.ExecutePath.FAST; import static accord.coordinate.Propose.Invalidate.proposeAndCommitInvalidate; @@ -73,7 +71,7 @@ void onPreAccepted(Topologies topologies, Timestamp executeAt, List if (tracker.hasFastPathAccepted()) { Deps deps = Deps.merge(oks, ok -> ok.witnessedAt.equals(txnId) ? ok.deps : null); - execute(executeAdapter(), node, topologies, route, FAST, txnId, txn, txnId, deps, settingCallback()); + executeAdapter().execute(node, topologies, route, FAST, txnId, txn, txnId, deps, settingCallback()); node.agent().metricsEventsListener().onFastPathTaken(txnId, deps); } else @@ -93,7 +91,7 @@ void onPreAccepted(Topologies topologies, Timestamp executeAt, List if (PreAccept.rejectExecuteAt(txnId, topologies)) proposeAndCommitInvalidate(node, Ballot.ZERO, txnId, route.homeKey(), route, executeAt, this); else - propose(proposeAdapter(), node, topologies, route, Ballot.ZERO, txnId, txn, executeAt, deps, this); + proposeAdapter().propose(node, topologies, route, Ballot.ZERO, txnId, txn, executeAt, deps, this); } node.agent().metricsEventsListener().onSlowPathTaken(txnId, deps); diff --git a/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java b/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java index 0aced3c67..4a4e82a47 100644 --- a/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java +++ b/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java @@ -53,10 +53,14 @@ enum Step { Continue, InitiateRecovery } CoordinationAdapter get(TxnId txnId, Step step); } - void propose(Node node, Topologies withUnsynced, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback); - void stabilise(Node node, Topologies coordinates, Topologies all, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback); - void execute(Node node, Topologies all, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback); - void persist(Node node, Topologies all, FullRoute route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer callback); + void propose(Node node, @Nullable Topologies preaccept, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback); + void stabilise(Node node, @Nullable Topologies any, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback); + void execute(Node node, @Nullable Topologies any, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback); + void persist(Node node, @Nullable Topologies any, FullRoute route, Participants sendTo, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer callback); + default void persist(Node node, @Nullable Topologies any, FullRoute route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer callback) + { + persist(node, any, route, route, txnId, txn, executeAt, deps, writes, result, callback); + } class DefaultFactory implements Factory { @@ -72,63 +76,6 @@ public CoordinationAdapter get(TxnId txnId, Step step) } } - /** - * Utility methods for correctly invoking the next phase of the state machine via a CoordinationAdapter. - * Simply ensures the topologies are correct before being passed to the instance method. - */ - class Invoke - { - public static void propose(CoordinationAdapter adapter, Node node, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) - { - propose(adapter, node, node.topology().withUnsyncedEpochs(route, txnId, executeAt), route, ballot, txnId, txn, executeAt, deps, callback); - } - - public static void propose(CoordinationAdapter adapter, Node node, Topologies withUnsynced, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) - { - adapter.propose(node, withUnsynced, route, ballot, txnId, txn, executeAt, deps, callback); - } - - public static void stabilise(CoordinationAdapter adapter, Node node, Topologies any, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) - { - if (!node.topology().hasEpoch(executeAt.epoch())) - { - node.withEpoch(executeAt.epoch(), () -> stabilise(adapter, node, any, route, ballot, txnId, txn, executeAt, deps, callback)); - return; - } - Topologies coordinates = any.forEpochs(txnId.epoch(), txnId.epoch()); - Topologies all; - if (txnId.epoch() == executeAt.epoch()) all = coordinates; - else if (any.currentEpoch() >= executeAt.epoch() && any.oldestEpoch() <= txnId.epoch()) all = any.forEpochs(txnId.epoch(), executeAt.epoch()); - else all = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch()); - - adapter.stabilise(node, coordinates, all, route, ballot, txnId, txn, executeAt, deps, callback); - } - - public static void execute(CoordinationAdapter adapter, Node node, Topologies any, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) - { - if (any.oldestEpoch() <= txnId.epoch() && any.currentEpoch() >= executeAt.epoch()) any = any.forEpochs(txnId.epoch(), executeAt.epoch()); - else any = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch()); - adapter.execute(node, any, route, path, txnId, txn, executeAt, deps, callback); - } - - public static void persist(CoordinationAdapter adapter, Node node, Topologies any, FullRoute route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, @Nullable BiConsumer callback) - { - if (any.oldestEpoch() <= txnId.epoch() && any.currentEpoch() >= executeAt.epoch()) any = any.forEpochs(txnId.epoch(), executeAt.epoch()); - else any = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch()); - Topologies executes = any.forEpochs(executeAt.epoch(), executeAt.epoch()); - - adapter.persist(node, any, route, txnId, txn, executeAt, deps, writes, result, callback); - } - - public static void persist(CoordinationAdapter adapter, Node node, FullRoute route, Participants sendTo, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, @Nullable BiConsumer callback) - { - Topologies all = node.topology().preciseEpochs(sendTo, txnId.epoch(), executeAt.epoch()); - Topologies executes = all.forEpochs(executeAt.epoch(), executeAt.epoch()); - - adapter.persist(node, all, route, txnId, txn, executeAt, deps, writes, result, callback); - } - } - class Adapters { public static CoordinationAdapter standard() @@ -143,17 +90,17 @@ public static CoordinationAdapter recovery() return RecoveryTxnAdapter.INSTANCE; } - public static > CoordinationAdapter> inclusiveSyncPoint() + public static > SyncPointAdapter inclusiveSyncPoint() { return AsyncInclusiveSyncPointAdapter.INSTANCE; } - public static > CoordinationAdapter> inclusiveSyncPointBlocking() + public static > SyncPointAdapter inclusiveSyncPointBlocking() { return InclusiveSyncPointBlockingAdapter.INSTANCE; } - public static > CoordinationAdapter> exclusiveSyncPoint() + public static > SyncPointAdapter exclusiveSyncPoint() { return ExclusiveSyncPointAdapter.INSTANCE; } @@ -161,32 +108,58 @@ public static CoordinationAdapter recovery() public static abstract class AbstractTxnAdapter implements CoordinationAdapter { @Override - public void propose(Node node, Topologies withUnsynced, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) + public void propose(Node node, @Nullable Topologies preaccept, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) { - new ProposeTxn(node, withUnsynced, route, ballot, txnId, txn, executeAt, deps, callback).start(); + Topologies all = preaccept == null ? node.topology().withUnsyncedEpochs(route, txnId, executeAt) : preaccept; + new ProposeTxn(node, all, route, ballot, txnId, txn, executeAt, deps, callback).start(); } @Override - public void stabilise(Node node, Topologies coordinates, Topologies all, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) + public void stabilise(Node node, Topologies any, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) { + if (!node.topology().hasEpoch(executeAt.epoch())) + { + node.withEpoch(executeAt.epoch(), () -> stabilise(node, any, route, ballot, txnId, txn, executeAt, deps, callback)); + return; + } + + Topologies coordinates = any.forEpochs(txnId.epoch(), txnId.epoch()); + Topologies all; + if (txnId.epoch() == executeAt.epoch()) all = coordinates; + else if (any.currentEpoch() >= executeAt.epoch() && any.oldestEpoch() <= txnId.epoch()) all = any.forEpochs(txnId.epoch(), executeAt.epoch()); + else all = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch()); + if (Faults.TRANSACTION_INSTABILITY) execute(node, all, route, SLOW, txnId, txn, executeAt, deps, callback); else new StabiliseTxn(node, coordinates, all, route, ballot, txnId, txn, executeAt, deps, callback).start(); } @Override - public void execute(Node node, Topologies all, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) + public void execute(Node node, Topologies any, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer callback) { - if (txn.read().keys().isEmpty()) Invoke.persist(this, node, all, route, txnId, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null), callback); + Topologies all = execution(node, any, route, txnId, executeAt); + + if (txn.read().keys().isEmpty()) persist(node, all, route, txnId, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null), callback); else new ExecuteTxn(node, all, route, path, txnId, txn, txn.read().keys().toParticipants(), executeAt, deps, callback).start(); } + + Topologies execution(Node node, Topologies any, Participants participants, TxnId txnId, Timestamp executeAt) + { + if (any != null && any.oldestEpoch() <= txnId.epoch() && any.currentEpoch() >= executeAt.epoch()) + return any.forEpochs(txnId.epoch(), executeAt.epoch()); + else + return node.topology().preciseEpochs(participants, txnId.epoch(), executeAt.epoch()); + } } public static class StandardTxnAdapter extends AbstractTxnAdapter { public static final StandardTxnAdapter INSTANCE = new StandardTxnAdapter(); + @Override - public void persist(Node node, Topologies all, FullRoute route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer callback) + public void persist(Node node, Topologies any, FullRoute route, Participants participants, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer callback) { + Topologies all = execution(node, any, participants, txnId, executeAt); + if (callback != null) callback.accept(result, null); new PersistTxn(node, all, txnId, route, txn, executeAt, deps, writes, result) .start(Apply.FACTORY, Minimal, all, writes, result); @@ -197,16 +170,21 @@ public static class RecoveryTxnAdapter extends AbstractTxnAdapter { public static final RecoveryTxnAdapter INSTANCE = new RecoveryTxnAdapter(); @Override - public void persist(Node node, Topologies all, FullRoute route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer callback) + public void persist(Node node, Topologies any, FullRoute route, Participants participants, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer callback) { + Topologies all = execution(node, any, participants, txnId, executeAt); + if (callback != null) callback.accept(result, null); new PersistTxn(node, all, txnId, route, txn, executeAt, deps, writes, result) .start(Apply.FACTORY, Maximal, all, writes, result); } } - public static abstract class AbstractSyncPointAdapter> implements CoordinationAdapter> + public static abstract class SyncPointAdapter> implements CoordinationAdapter> { + abstract Topologies forDecision(Node node, FullRoute route, TxnId txnId); + abstract Topologies forExecution(Node node, FullRoute route, TxnId txnId, Timestamp executeAt, Deps deps); + void invokeSuccess(Node node, FullRoute route, TxnId txnId, Txn txn, Deps deps, BiConsumer, Throwable> callback) { if (txn.keys().domain() == Range) @@ -215,62 +193,88 @@ void invokeSuccess(Node node, FullRoute route, TxnId txnId, Txn txn, Deps dep } @Override - public void propose(Node node, Topologies withUnsynced, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) + public void propose(Node node, Topologies any, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) { - new ProposeSyncPoint<>(this, node, withUnsynced, route, ballot, txnId, txn, executeAt, deps, callback).start(); + Topologies all = forDecision(node, route, txnId); + new ProposeSyncPoint<>(this, node, all, route, ballot, txnId, txn, executeAt, deps, callback).start(); } @Override - public void stabilise(Node node, Topologies coordinates, Topologies all, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) + public void stabilise(Node node, Topologies any, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) { + Topologies all = forExecution(node, route, txnId, executeAt, deps); + Topologies coordinates = all.forEpochs(txnId.epoch(), txnId.epoch()); new StabiliseSyncPoint<>(this, node, coordinates, all, route, ballot, txnId, txn, executeAt, deps, callback).start(); } @Override - public void execute(Node node, Topologies all, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) + public void execute(Node node, Topologies any, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) { - Invoke.persist(this, node, all, route, txnId, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null), callback); + Topologies all = forExecution(node, route, txnId, executeAt, deps); + persist(node, all, route, txnId, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null), callback); } @Override - public void persist(Node node, Topologies all, FullRoute route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer, Throwable> callback) + public void persist(Node node, Topologies any, FullRoute route, Participants participants, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer, Throwable> callback) { + Topologies all = forExecution(node, route, txnId, executeAt, deps); + invokeSuccess(node, route, txnId, txn, deps, callback); new PersistTxn(node, all, txnId, route, txn, executeAt, deps, writes, result) - .start(Apply.FACTORY, Maximal, all, writes, result); + .start(Apply.FACTORY, Maximal, any, writes, result); } } - public static class ExclusiveSyncPointAdapter> extends AbstractSyncPointAdapter + public static class ExclusiveSyncPointAdapter> extends SyncPointAdapter { private static final ExclusiveSyncPointAdapter INSTANCE = new ExclusiveSyncPointAdapter(); @Override - public void execute(Node node, Topologies all, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) + Topologies forDecision(Node node, FullRoute route, TxnId txnId) { - // TODO (required): remember and document why we don't use fast path for exclusive sync points - if (path == FAST) - { - Invoke.stabilise(this, node, all, route, Ballot.ZERO, txnId, txn, executeAt, deps, callback); - } - else - { - super.execute(node, all, route, path, txnId, txn, executeAt, deps, callback); - } + return node.topology().withOpenEpochs(route, null, txnId); + } + + @Override + Topologies forExecution(Node node, FullRoute route, TxnId txnId, Timestamp executeAt, Deps deps) + { + TxnId minId = TxnId.nonNullOrMin(txnId, deps.minTxnId()); + return node.topology().withUncompletedEpochs(route, minId, txnId); + } + + @Override + public void execute(Node node, Topologies any, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) + { + // TODO (required, consider): remember and document why we don't use fast path for exclusive sync points + if (path == FAST) stabilise(node, any, route, Ballot.ZERO, txnId, txn, executeAt, deps, callback); + else super.execute(node, any, route, path, txnId, txn, executeAt, deps, callback); } } - private static abstract class AbstractInclusiveSyncPointAdapter> extends AbstractSyncPointAdapter + private static abstract class AbstractInclusiveSyncPointAdapter> extends SyncPointAdapter { - protected AbstractInclusiveSyncPointAdapter() { super(); } @Override - public void execute(Node node, Topologies all, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) + Topologies forDecision(Node node, FullRoute route, TxnId txnId) + { + return node.topology().withUnsyncedEpochs(route, txnId, txnId); + } + + @Override + Topologies forExecution(Node node, FullRoute route, TxnId txnId, Timestamp executeAt, Deps deps) + { + return node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch()); + } + + @Override + public void execute(Node node, Topologies any, FullRoute route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer, Throwable> callback) { + Topologies all = forExecution(node, route, txnId, executeAt, deps); + ExecuteBlocking execute = ExecuteBlocking.atQuorum(node, all, new SyncPoint<>(txnId, deps, (S)txn.keys(), route), executeAt); execute.start(); addOrExecuteCallback(execute, callback); @@ -317,7 +321,7 @@ protected void addOrExecuteCallback(ExecuteBlocking execute, BiConsumer route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer, Throwable> callback) + public void persist(Node node, Topologies any, FullRoute route, Participants participants, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer, Throwable> callback) { throw new UnsupportedOperationException(); } diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java b/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java index 81f7f6ccb..bec5483e0 100644 --- a/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java +++ b/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java @@ -38,10 +38,12 @@ import accord.primitives.Txn; import accord.primitives.TxnId; import accord.topology.Topologies; +import accord.utils.Invariants; import static accord.coordinate.ReadCoordinator.Action.Aborted; import static accord.coordinate.ReadCoordinator.Action.Approve; import static accord.coordinate.ReadCoordinator.Action.ApprovePartial; +import static accord.primitives.Txn.Kind.EphemeralRead; import static accord.utils.Invariants.illegalState; public class ExecuteEphemeralRead extends ReadCoordinator @@ -60,7 +62,8 @@ public class ExecuteEphemeralRead extends ReadCoordinator { // we need to send Stable to the origin epoch as well as the execution epoch // TODO (desired): permit slicing Topologies by key (though unnecessary if we eliminate the concept of non-participating home keys) - super(node, route.isParticipatingHomeKey() ? topologies : node.topology().preciseEpochs(route, txnId.epoch(), executionEpoch), txnId); + super(node, topologies, txnId); + Invariants.checkArgument(txnId.kind() == EphemeralRead); this.txn = txn; this.route = route; this.allTopologies = topologies; diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java b/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java index 2bb8f800c..2c9db4d63 100644 --- a/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java +++ b/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java @@ -18,7 +18,10 @@ package accord.coordinate; +import java.util.function.Function; + import accord.api.Result; +import accord.coordinate.CoordinationAdapter.Adapters; import accord.coordinate.tracking.AbstractSimpleTracker; import accord.coordinate.tracking.QuorumTracker; import accord.coordinate.tracking.RequestStatus; @@ -27,15 +30,20 @@ import accord.messages.Callback; import accord.messages.ReadData; import accord.messages.ReadData.ReadReply; +import accord.messages.WaitUntilApplied; import accord.primitives.Participants; +import accord.primitives.Ranges; import accord.primitives.Seekables; import accord.primitives.SyncPoint; import accord.primitives.Timestamp; import accord.primitives.Txn; import accord.primitives.Writes; import accord.topology.Topologies; +import accord.utils.Invariants; import accord.utils.async.AsyncResults.SettableResult; +import static accord.primitives.Txn.Kind.ExclusiveSyncPoint; + public abstract class ExecuteSyncPoint> extends SettableResult> implements Callback { public static class SyncPointErased extends Throwable {} @@ -43,15 +51,16 @@ public static class SyncPointErased extends Throwable {} public static class ExecuteBlocking> extends ExecuteSyncPoint { private final Timestamp executeAt; - public ExecuteBlocking(Node node, AbstractSimpleTracker tracker, SyncPoint syncPoint, Timestamp executeAt) + public ExecuteBlocking(Node node, SyncPoint syncPoint, AbstractSimpleTracker tracker, Timestamp executeAt) { - super(node, tracker, syncPoint); + super(node, syncPoint, tracker); + Invariants.checkArgument(!syncPoint.syncId.kind().awaitsOnlyDeps()); this.executeAt = executeAt; } public static > ExecuteBlocking atQuorum(Node node, Topologies topologies, SyncPoint syncPoint, Timestamp executeAt) { - return new ExecuteBlocking<>(node, new QuorumTracker(topologies), syncPoint, executeAt); + return new ExecuteBlocking<>(node, syncPoint, new QuorumTracker(topologies), executeAt); } @Override @@ -68,17 +77,81 @@ public void start() } } + public static class ExecuteExclusiveSyncPoint extends ExecuteSyncPoint + { + private long retryInFutureEpoch; + public ExecuteExclusiveSyncPoint(Node node, SyncPoint syncPoint, Function> trackerSupplier) + { + super(node, syncPoint, Adapters.exclusiveSyncPoint().forExecution(node, syncPoint.route(), syncPoint.syncId, syncPoint.syncId, syncPoint.waitFor), trackerSupplier); + Invariants.checkArgument(syncPoint.syncId.kind() == ExclusiveSyncPoint); + } + + public ExecuteExclusiveSyncPoint(Node node, SyncPoint syncPoint, Function> trackerSupplier, AbstractSimpleTracker tracker) + { + super(node, syncPoint, trackerSupplier, tracker); + Invariants.checkArgument(syncPoint.syncId.kind() == ExclusiveSyncPoint); + } + + @Override + protected void start() + { + node.send(tracker.nodes(), to -> new WaitUntilApplied(to, tracker.topologies(), syncPoint.syncId, syncPoint.keysOrRanges, syncPoint.syncId.epoch()), this); + } + + @Override + public synchronized void onSuccess(Node.Id from, ReadReply reply) + { + if (reply instanceof ReadData.ReadOkWithFutureEpoch) + retryInFutureEpoch = Math.max(retryInFutureEpoch, ((ReadData.ReadOkWithFutureEpoch) reply).futureEpoch); + + super.onSuccess(from, reply); + } + + @Override + protected void onSuccess() + { + if (retryInFutureEpoch > tracker.topologies().currentEpoch()) + { + ExecuteExclusiveSyncPoint continuation = new ExecuteExclusiveSyncPoint(node, syncPoint, trackerSupplier, trackerSupplier.apply(node.topology().preciseEpochs(syncPoint.route(), tracker.topologies().currentEpoch(), retryInFutureEpoch))); + continuation.addCallback((success, failure) -> { + if (failure == null) trySuccess(success); + else tryFailure(failure); + }); + continuation.start(); + } + else + { + super.onSuccess(); + } + } + } + final Node node; - final AbstractSimpleTracker tracker; final SyncPoint syncPoint; + + final Function> trackerSupplier; + final AbstractSimpleTracker tracker; private Throwable failures = null; - ExecuteSyncPoint(Node node, AbstractSimpleTracker tracker, SyncPoint syncPoint) + ExecuteSyncPoint(Node node, SyncPoint syncPoint, AbstractSimpleTracker tracker) { - // TODO (required): this isn't correct, we need to potentially perform a second round if a dependency executes in a future epoch and we have lost ownership of that epoch this.node = node; + this.syncPoint = syncPoint; + this.trackerSupplier = null; this.tracker = tracker; + } + + ExecuteSyncPoint(Node node, SyncPoint syncPoint, Topologies topologies, Function> trackerSupplier) + { + this(node, syncPoint, trackerSupplier, trackerSupplier.apply(topologies)); + } + + ExecuteSyncPoint(Node node, SyncPoint syncPoint, Function> trackerSupplier, AbstractSimpleTracker tracker) + { + this.node = node; this.syncPoint = syncPoint; + this.trackerSupplier = trackerSupplier; + this.tracker = tracker; } protected abstract void start(); @@ -109,7 +182,7 @@ public synchronized void onSuccess(Node.Id from, ReadReply reply) } else { - // TODO (required): we also need to handle ranges not being safe to read + // TODO (required, consider): do we need to handle ranges not being safe to read if (tracker.recordSuccess(from) == RequestStatus.Success) onSuccess(); } diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java b/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java index be2623d74..2be1080fb 100644 --- a/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java +++ b/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java @@ -69,7 +69,7 @@ public class ExecuteTxn extends ReadCoordinator { // we need to send Stable to the origin epoch as well as the execution epoch // TODO (desired): permit slicing Topologies by key (though unnecessary if we eliminate the concept of non-participating home keys) - super(node, route.isParticipatingHomeKey() ? topologies : node.topology().preciseEpochs(readScope, txnId.epoch(), executeAt.epoch()), txnId); + super(node, topologies, txnId); this.path = path; this.txn = txn; this.route = route; @@ -147,7 +147,7 @@ protected void onDone(Success success, Throwable failure) if (failure == null) { Result result = txn.result(txnId, executeAt, data); - CoordinationAdapter.Invoke.persist(adapter(), node, allTopologies, route, txnId, txn, executeAt, stableDeps, txn.execute(txnId, executeAt, data), result, callback); + adapter().persist(node, allTopologies, route, txnId, txn, executeAt, stableDeps, txn.execute(txnId, executeAt, data), result, callback); } else { diff --git a/accord-core/src/main/java/accord/coordinate/FetchCoordinator.java b/accord-core/src/main/java/accord/coordinate/FetchCoordinator.java index b33192aa1..2d3c19fac 100644 --- a/accord-core/src/main/java/accord/coordinate/FetchCoordinator.java +++ b/accord-core/src/main/java/accord/coordinate/FetchCoordinator.java @@ -151,6 +151,7 @@ protected FetchCoordinator(Node node, Ranges ranges, SyncPoint syncPoint, FetchR this.fetchRanges = fetchRanges; // TODO (expected): prioritise nodes that were members in the "prior" epoch also // (by prior, we mean the prior epoch affecting ownership of this shard, not the prior numerical epoch) + // TODO (required): the sourceEpoch may be wrong, as a transaction may execute in a future epoch Topology topology = node.topology().forEpoch(ranges, syncPoint.sourceEpoch()).get(0); for (Node.Id id : topology.nodes()) { diff --git a/accord-core/src/main/java/accord/coordinate/FetchData.java b/accord-core/src/main/java/accord/coordinate/FetchData.java index b1689bbaf..15901bbb8 100644 --- a/accord-core/src/main/java/accord/coordinate/FetchData.java +++ b/accord-core/src/main/java/accord/coordinate/FetchData.java @@ -32,6 +32,7 @@ import static accord.coordinate.Infer.InvalidateAndCallback.locallyInvalidateAndCallback; import static accord.primitives.Route.castToRoute; +import static accord.primitives.Route.isFullRoute; import static accord.primitives.Route.isRoute; /** @@ -61,13 +62,13 @@ public static Object fetch(Known fetch, Node node, TxnId txnId, Route route, Invariants.checkArgument(node.topology().hasEpoch(srcEpoch), "Unknown epoch %d, latest known is %d", srcEpoch, node.epoch()); long toEpoch = Math.max(srcEpoch, forLocalEpoch == null ? 0 : forLocalEpoch.epoch()); Ranges ranges = node.topology().localRangesForEpochs(txnId.epoch(), toEpoch); - if (!route.covers(ranges)) + if (!Route.isFullRoute(route)) { return fetchWithIncompleteRoute(fetch, node, txnId, route, forLocalEpoch, executeAt, callback); } else { - return fetchInternal(ranges, fetch, node, txnId, route.sliceStrict(ranges), executeAt, toEpoch, callback); + return fetchInternal(ranges, fetch, node, txnId, route.slice(ranges), executeAt, toEpoch, callback); } } @@ -80,6 +81,10 @@ else if (foundRoute.route == null) { reportRouteNotFound(node, txnId, executeAt, forLocalEpoch, someUnseekables, foundRoute.known, callback); } + else if (isFullRoute(foundRoute.route)) + { + fetch(fetch, node, txnId, Route.castToFullRoute(foundRoute.route), forLocalEpoch, executeAt, callback); + } else if (isRoute(someUnseekables) && someUnseekables.containsAll(foundRoute.route)) { // this is essentially a reentrancy check; we can only reach this point if we have already tried once to fetchSomeRoute @@ -119,7 +124,7 @@ private static void reportRouteNotFound(Node node, TxnId txnId, @Nullable Timest case Erased: case WasApply: case Apply: - // TODO (required): we may now be stale + // TODO (expected): we may be stale callback.accept(found, null); } } @@ -140,7 +145,7 @@ public static Object fetch(Known fetch, Node node, TxnId txnId, FullRoute rou return node.awaitEpoch(executeAt).map(ignore -> { long toEpoch = Math.max(fetch.fetchEpoch(txnId, executeAt), forLocalEpoch == null ? 0 : forLocalEpoch.epoch()); Ranges ranges = node.topology().localRangesForEpochs(txnId.epoch(), toEpoch); - return fetchInternal(ranges, fetch, node, txnId, route.sliceStrict(ranges), executeAt, toEpoch, callback); + return fetchInternal(ranges, fetch, node, txnId, route.slice(ranges), executeAt, toEpoch, callback); }).beginAsResult(); } @@ -148,7 +153,7 @@ private static Object fetchInternal(Ranges ranges, Known target, Node node, TxnI { long srcEpoch = target.fetchEpoch(txnId, executeAt); Invariants.checkArgument(node.topology().hasEpoch(srcEpoch), "Unknown epoch %d, latest known is %d", srcEpoch, node.epoch()); - PartialRoute fetch = route.sliceStrict(ranges); + PartialRoute fetch = route.slice(ranges); return fetchData(target, node, txnId, fetch, srcEpoch, forLocalEpoch, (sufficientFor, fail) -> { if (fail != null) callback.accept(null, fail); else callback.accept(sufficientFor, null); diff --git a/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java b/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java index 5963ebd9a..677c40fc9 100644 --- a/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java +++ b/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java @@ -18,7 +18,7 @@ package accord.coordinate; -import java.util.Set; +import java.util.Collection; import accord.coordinate.tracking.QuorumTracker; import accord.local.Node; @@ -83,7 +83,7 @@ public static AsyncResult fetchMaxConflict(Node node, Seekables } @Override - void contact(Set nodes, Topologies topologies, Callback callback) + void contact(Collection nodes, Topologies topologies, Callback callback) { node.send(nodes, to -> new GetMaxConflict(to, topologies, route, keysOrRanges, executionEpoch), callback); } diff --git a/accord-core/src/main/java/accord/coordinate/Infer.java b/accord-core/src/main/java/accord/coordinate/Infer.java index eecae95ab..4b41408d9 100644 --- a/accord-core/src/main/java/accord/coordinate/Infer.java +++ b/accord-core/src/main/java/accord/coordinate/Infer.java @@ -31,8 +31,10 @@ import accord.local.SafeCommandStore; import accord.local.Status; import accord.local.Status.Known; +import accord.messages.CheckStatus.FoundKnown; +import accord.messages.CheckStatus.FoundKnownMap; import accord.primitives.EpochSupplier; -import accord.primitives.Participants; +import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.Route; import accord.primitives.Timestamp; @@ -43,6 +45,7 @@ import static accord.coordinate.Infer.InvalidIf.IfPreempted; import static accord.coordinate.Infer.InvalidIf.IfQuorum; +import static accord.coordinate.Infer.InvalidIf.NotInvalid; import static accord.coordinate.Infer.InvalidIf.NotKnown; import static accord.coordinate.Infer.InvalidIfNot.IfUndecided; import static accord.coordinate.Infer.InvalidIfNot.IfUnknown; @@ -52,7 +55,9 @@ import static accord.local.Status.PreCommitted; import static accord.primitives.Route.castToRoute; import static accord.primitives.Route.isRoute; +import static accord.utils.Invariants.illegalState; +// TODO (required): dedicated randomised testing of all inferences public class Infer { public enum InvalidIfNot @@ -91,18 +96,26 @@ public enum InvalidIfNot /** * If the command has not had its execution timestamp agreed on any shard */ - IfUndecided(IfQuorum, IfQuorum); + IfUndecided(IfQuorum, IfQuorum), + + /** + * This command is known to be decided, so it is a logic bug if it is inferred elsewhere to be invalid. + */ + IsNotInvalid(NotInvalid, NotInvalid); final InvalidIf unknown, undecided; private static final InvalidIfNot[] LOOKUP; - private static final int invalidIfs = InvalidIf.values().length; + private static final int invalidIfs = InvalidIf.values().length - 1; static { LOOKUP = new InvalidIfNot[invalidIfs * invalidIfs]; InvalidIfNot[] invalidIfNot = InvalidIfNot.values(); for (InvalidIfNot ifNot : invalidIfNot) - LOOKUP[ifNot.unknown.ordinal() * invalidIfs + ifNot.undecided.ordinal()] = ifNot; + { + if (ifNot != IsNotInvalid) + LOOKUP[ifNot.unknown.ordinal() * invalidIfs + ifNot.undecided.ordinal()] = ifNot; + } } InvalidIfNot(InvalidIf unknown, InvalidIf undecided) @@ -113,7 +126,7 @@ public enum InvalidIfNot public static boolean isMax(InvalidIfNot that) { - return that == IfUndecided; + return that == IsNotInvalid; } public InvalidIfNot atLeast(InvalidIfNot that) @@ -128,11 +141,14 @@ public InvalidIfNot reduce(InvalidIfNot that) private InvalidIfNot lookup(InvalidIf unknown, InvalidIf undecided) { + if (unknown == NotInvalid) + return IsNotInvalid; return LOOKUP[unknown.ordinal() * invalidIfs + undecided.ordinal()]; } private static InvalidIf atLeast(InvalidIf a, InvalidIf b) { + if (a == NotInvalid || b == NotInvalid) return NotInvalid; if (a == b) return a; return IfPreempted; } @@ -156,8 +172,11 @@ private static boolean inferInvalidWithQuorum(InvalidIf invalidIf, IsPreempted i switch (invalidIf) { default: throw new AssertionError("Unhandled InvalidIf: " + invalidIf); - case NotKnown: break; - case IfQuorum: return true; + case NotInvalid: + case NotKnown: + break; + case IfQuorum: + return true; case IfPreempted: if (isPreempted == IsPreempted.Preempted) return true; @@ -179,7 +198,12 @@ enum InvalidIf /** * If we obtain a quorum of responses with the associated lower bound, we can infer the command is invalidated if it has not been witnessed at the lower bound */ - IfQuorum + IfQuorum, + + /** + * Definitely not invalid + */ + NotInvalid } // only valid with a quorum of responses @@ -266,7 +290,7 @@ Void apply(SafeCommandStore safeStore, SafeCommand safeCommand) { // we're applying an invalidation, so the record will not be cleaned up until the whole range is truncated Command command = safeCommand.current(); - // TODO (required): consider the !command.hasBeen(PreCommitted) condition + // TODO (required, consider): consider the !command.hasBeen(PreCommitted) condition Invariants.checkState(!command.hasBeen(PreCommitted) || command.hasBeen(Status.Truncated), "Unexpected status for %s", command); Commands.commitInvalidate(safeStore, safeCommand, someUnseekables); return null; @@ -293,45 +317,74 @@ public static void safeEraseAndCallback(Node node, TxnId txnId, EpochSupplie Void apply(SafeCommandStore safeStore, SafeCommand safeCommand) { Command command = safeCommand.current(); - // TODO (required): introduce a special form of Erased where we do not imply the phase is "Cleanup" + // TODO (required, consider): introduce a special form of Erased where we do not imply the phase is "Cleanup" if (!command.hasBeen(PreApplied) && safeToCleanup(safeStore, command, Route.castToRoute(someUnseekables), null)) Commands.setErased(safeStore, safeCommand); return null; } } - public static InvalidIfNot invalidIfNot(SafeCommandStore safeStore, TxnId txnId, Unseekables query) + public static FoundKnownMap withInvalidIfNot(SafeCommandStore safeStore, TxnId txnId, Unseekables localKeys, Unseekables maxKeys, FoundKnown known) { + if (isRoute(maxKeys)) + maxKeys = castToRoute(maxKeys).withHomeKey(); + if (safeStore.commandStore().globalDurability(txnId).compareTo(Majority) >= 0) { - Unseekables preacceptsWith = isRoute(query) ? castToRoute(query).withHomeKey() : query; - return safeStore.commandStore().isRejectedIfNotPreAccepted(txnId, preacceptsWith) ? IfUnknown : IfUndecided; + InvalidIfNot invalidIfNot = safeStore.commandStore().isRejectedIfNotPreAccepted(txnId, maxKeys) ? IfUnknown : IfUndecided; + return FoundKnownMap.create(maxKeys, known.withAtLeast(invalidIfNot)); } - // TODO (expected, consider): should we force this to be a Route or a Participants? - if (isRoute(query)) + // TODO (required): document the cleanup semantics that make this safe and tie the locations together preferably by compiler + // (in essence iirc we permit + + Ranges coordinateRanges = safeStore.ranges().allAt(txnId.epoch()); + FoundKnownMap map = FoundKnownMap.create(localKeys, known); + if (Cleanup.isSafeToCleanup(safeStore.commandStore().durableBefore(), txnId, coordinateRanges)) { - Participants participants = castToRoute(query).participants(); - // TODO (desired): limit to local participants to avoid O(n2) work across cluster - if (safeStore.commandStore().durableBefore().isSomeShardDurable(txnId, participants, Majority)) - return IfUndecided; + // it is safe to cleanup for all keys we own, as they are all known to have UniversalOrInvalidated durability past the TxnId in question + map = FoundKnownMap.merge(map, FoundKnownMap.create(localKeys, known.withAtLeast(IfUndecided))); } - if (Cleanup.isSafeToCleanup(safeStore.commandStore().durableBefore(), txnId, safeStore.ranges().allAt(txnId.epoch()))) - return IfUndecided; + // TODO (desired): limit to local participants to avoid O(n2) work across cluster + class Builder extends FoundKnownMap.Builder + { + int minIndex; + public Builder(boolean inclusiveEnds, int capacity) + { + super(inclusiveEnds, capacity); + } + } + Builder builder = new Builder(maxKeys.get(0).asRange().endInclusive(), 2 * maxKeys.size()); + safeStore.commandStore().durableBefore().foldl(maxKeys, (e, b, q, id, i, j, k) -> { + if (e.majorityBefore.compareTo(id) > 0) + { + i = Math.max(i, b.minIndex); + while (i < j) + { + Range range = q.get(i++).asRange(); + b.append(range.start(), FoundKnown.Nothing.withAtLeast(IfUndecided), (i1, i2) -> { throw illegalState(); }); + b.append(range.end(), null, (i1, i2) -> { throw illegalState(); }); + } + b.minIndex = i; + } + return b; + }, builder, maxKeys, txnId, i -> false); + + if (!builder.isEmpty()) + map = FoundKnownMap.merge(map, builder.build()); - return InvalidIfNot.NotKnownToBeInvalid; + return map; } public static boolean safeToCleanup(SafeCommandStore safeStore, Command command, Route fetchedWith, @Nullable Timestamp executeAt) { Invariants.checkArgument(fetchedWith != null || command.route() != null); TxnId txnId = command.txnId(); - if (command.route() == null || !fetchedWith.covers(safeStore.ranges().allAt(txnId.epoch()))) - return false; + Route route = Route.merge(command.route(), (Route)fetchedWith); - Route route = command.route(); - if (route == null) route = fetchedWith; + if (!Route.isFullRoute(route)) + return false; // TODO (required): is it safe to cleanup without an executeAt? We don't know for sure which ranges it might participate in. // We can infer the upper bound of execution by the "execution" of any ExclusiveSyncPoint used to infer the invalidation. diff --git a/accord-core/src/main/java/accord/coordinate/Invalidate.java b/accord-core/src/main/java/accord/coordinate/Invalidate.java index 3f6112605..b4cefb4bf 100644 --- a/accord-core/src/main/java/accord/coordinate/Invalidate.java +++ b/accord-core/src/main/java/accord/coordinate/Invalidate.java @@ -18,10 +18,6 @@ package accord.coordinate; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import java.util.function.BiConsumer; import accord.coordinate.tracking.InvalidationTracker; @@ -31,12 +27,14 @@ import accord.messages.Commit; import accord.local.*; import accord.primitives.*; +import accord.topology.Shard; import accord.topology.Topologies; import accord.api.RoutingKey; import accord.messages.BeginInvalidation; import accord.messages.BeginInvalidation.InvalidateReply; import accord.messages.Callback; +import accord.topology.Topology; import accord.utils.Invariants; import javax.annotation.Nullable; @@ -60,10 +58,10 @@ public class Invalidate implements Callback private boolean isDone; private boolean isPrepareDone; private final boolean transitivelyInvokedByPriorInvalidation; - private final List replies = new ArrayList<>(); + private final InvalidateReply[] replies; + private final Topology topology; private final InvalidationTracker tracker; private Throwable failure; - private final Map debug = Invariants.debug() ? new HashMap<>() : null; private Invalidate(Node node, Ballot ballot, TxnId txnId, Unseekables invalidateWith, boolean transitivelyInvokedByPriorInvalidation, BiConsumer callback) { @@ -74,7 +72,10 @@ private Invalidate(Node node, Ballot ballot, TxnId txnId, Unseekables invalid this.transitivelyInvokedByPriorInvalidation = transitivelyInvokedByPriorInvalidation; this.invalidateWith = invalidateWith; Topologies topologies = node.topology().forEpoch(invalidateWith, txnId.epoch()); + Invariants.checkState(topologies.size() == 1); this.tracker = new InvalidationTracker(topologies); + this.topology = topologies.current(); + this.replies = new InvalidateReply[topology.nodes().size()]; } public static Invalidate invalidate(Node node, TxnId txnId, Unseekables invalidateWith, BiConsumer callback) @@ -101,8 +102,7 @@ public synchronized void onSuccess(Id from, InvalidateReply reply) if (isDone || isPrepareDone) return; - if (debug != null) debug.put(from, reply); - replies.add(reply); + replies[topology.nodes().find(from)] = reply; handle(tracker.recordSuccess(from, reply.isPromised(), reply.hasDecision(), reply.acceptedFastPath)); } @@ -148,19 +148,20 @@ private void invalidate() Invariants.checkState(!isPrepareDone); isPrepareDone = true; - // first look to see if it has already been - { - FullRoute fullRoute = InvalidateReply.findRoute(replies); - Route someRoute = InvalidateReply.mergeRoutes(replies); - InvalidateReply maxReply = InvalidateReply.max(replies); + FullRoute fullRoute = InvalidateReply.findRoute(replies); + Route someRoute = InvalidateReply.mergeRoutes(replies); + + // first look to see if it has already been decided/invalidated + // check each shard independently - if we find any that can be invalidated, do so + InvalidateReply max = InvalidateReply.max(replies); + InvalidateReply maxNotTruncated = max.status != Status.Truncated ? max : InvalidateReply.maxNotTruncated(replies); - switch (maxReply.status) + if (maxNotTruncated != null) + { + switch (maxNotTruncated.status) { - default: throw illegalState(); - case Truncated: - isDone = true; - callback.accept(TRUNCATED, null); - return; + default: throw new AssertionError("Unhandled status: " + maxNotTruncated.status); + case Truncated: throw illegalState(); case AcceptedInvalidate: // latest accept also invalidating, so we're on the same page and should finish our invalidation @@ -176,7 +177,7 @@ private void invalidate() case Stable: case Committed: case PreCommitted: - Invariants.checkState(maxReply.status == PreAccepted || !invalidateWith.contains(someRoute.homeKey()) || fullRoute != null); + Invariants.checkState(maxNotTruncated.status == PreAccepted || !invalidateWith.contains(someRoute.homeKey()) || fullRoute != null); case Accepted: // TODO (desired, efficiency): if we see Committed or above, go straight to Execute if we have assembled enough information @@ -193,7 +194,7 @@ private void invalidate() // Note that there's lots of scope for variations in behaviour here, but lots of care is needed. - Status witnessedByInvalidation = maxReply.status; + Status witnessedByInvalidation = maxNotTruncated.status; if (!witnessedByInvalidation.hasBeen(Accepted)) { Invariants.checkState(tracker.all(InvalidationShardTracker::isPromised)); @@ -211,6 +212,22 @@ private void invalidate() } } + if (max != maxNotTruncated) + { + boolean allShardsTruncated = true; + for (Shard shard : topology.shards()) + { + InvalidateReply maxReply = InvalidateReply.max(replies, shard, topology.nodes()); + allShardsTruncated &= maxReply.status == Status.Truncated; + } + if (allShardsTruncated) + { + isDone = true; + callback.accept(TRUNCATED, null); + return; + } + } + // if we have witnessed the transaction, but are able to invalidate, do we want to proceed? // Probably simplest to do so, but perhaps better for user if we don't. Ranges ranges = Ranges.of(tracker.promisedShard().range); diff --git a/accord-core/src/main/java/accord/coordinate/MaybeRecover.java b/accord-core/src/main/java/accord/coordinate/MaybeRecover.java index 96ccd6e29..10aa0994b 100644 --- a/accord-core/src/main/java/accord/coordinate/MaybeRecover.java +++ b/accord-core/src/main/java/accord/coordinate/MaybeRecover.java @@ -21,6 +21,7 @@ import java.util.function.BiConsumer; import accord.local.Status.Known; +import accord.messages.InformDurable; import accord.primitives.*; import accord.utils.Invariants; @@ -104,6 +105,8 @@ protected void onDone(Success success, Throwable fail) // we have included the home key, and one that witnessed the definition has responded, so it should also know the full route if (hasMadeProgress(full)) { + if (full.durability.isDurable()) + node.send(topologies.forEpoch(txnId.epoch()).forKey(route.homeKey()).nodes, to -> new InformDurable(to, topologies, route, txnId, full.executeAtIfKnown(), full.durability)); callback.accept(full.toProgressToken(), null); } else diff --git a/accord-core/src/main/java/accord/coordinate/Propose.java b/accord-core/src/main/java/accord/coordinate/Propose.java index cb5783bc0..aa3534240 100644 --- a/accord-core/src/main/java/accord/coordinate/Propose.java +++ b/accord-core/src/main/java/accord/coordinate/Propose.java @@ -19,9 +19,9 @@ package accord.coordinate; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.function.BiConsumer; import accord.api.RoutingKey; @@ -60,7 +60,7 @@ abstract class Propose implements Callback final Deps deps; final List acceptOks; - private final Map debug = debug() ? new HashMap<>() : null; + private final Map debug = debug() ? new TreeMap<>() : null; final Timestamp executeAt; final QuorumTracker acceptTracker; final BiConsumer callback; @@ -144,7 +144,7 @@ static class Invalidate implements Callback final TxnId txnId; final RoutingKey someParticipant; final BiConsumer callback; - final Map debug = debug() ? new HashMap<>() : null; + final Map debug = debug() ? new TreeMap<>() : null; private final QuorumShardTracker acceptTracker; private boolean isDone; diff --git a/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java b/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java index 6d3aab266..bd58373df 100644 --- a/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java +++ b/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java @@ -35,8 +35,6 @@ import accord.topology.Topologies; import accord.utils.Faults; -import static accord.coordinate.CoordinationAdapter.Invoke.stabilise; - public class ProposeSyncPoint> extends Propose> { @SuppressWarnings("unused") @@ -53,6 +51,6 @@ public class ProposeSyncPoint> extends Propose ok.deps)); - stabilise(adapter, node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback); + adapter.stabilise(node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback); } } diff --git a/accord-core/src/main/java/accord/coordinate/ProposeTxn.java b/accord-core/src/main/java/accord/coordinate/ProposeTxn.java index 075839d7e..5914f5b03 100644 --- a/accord-core/src/main/java/accord/coordinate/ProposeTxn.java +++ b/accord-core/src/main/java/accord/coordinate/ProposeTxn.java @@ -32,7 +32,6 @@ import accord.utils.Faults; import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue; -import static accord.coordinate.CoordinationAdapter.Invoke.stabilise; class ProposeTxn extends Propose { @@ -48,7 +47,7 @@ void onAccepted() if (!Faults.TRANSACTION_UNMERGED_DEPS) deps = deps.with(Deps.merge(acceptOks, ok -> ok.deps)); - stabilise(adapter(), node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback); + adapter().stabilise(node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback); } protected CoordinationAdapter adapter() diff --git a/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java b/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java index 17b8ff13a..e6ddd3c3e 100644 --- a/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java +++ b/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java @@ -95,7 +95,7 @@ protected enum Success protected final TxnId txnId; private boolean isDone; private Throwable failure; - final Map debug = debug() ? new HashMap<>() : null; + final Map debug = debug() ? new TreeMap<>() : null; protected ReadCoordinator(Node node, Topologies topologies, TxnId txnId) { diff --git a/accord-core/src/main/java/accord/coordinate/Recover.java b/accord-core/src/main/java/accord/coordinate/Recover.java index ea2af7a73..4f1878e49 100644 --- a/accord-core/src/main/java/accord/coordinate/Recover.java +++ b/accord-core/src/main/java/accord/coordinate/Recover.java @@ -18,17 +18,17 @@ package accord.coordinate; -import java.util.ArrayList; -import java.util.HashMap; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.Consumer; +import java.util.stream.Stream; import accord.api.Result; -import accord.coordinate.CoordinationAdapter.Invoke; import accord.coordinate.tracking.QuorumTracker; import accord.coordinate.tracking.RecoveryTracker; import accord.local.Node; @@ -51,6 +51,7 @@ import accord.primitives.Timestamp; import accord.primitives.Txn; import accord.primitives.TxnId; +import accord.topology.Shard; import accord.topology.Topologies; import accord.topology.Topology; import accord.utils.Invariants; @@ -58,15 +59,14 @@ import accord.utils.async.AsyncResults; import static accord.coordinate.CoordinationAdapter.Factory.Step.InitiateRecovery; -import static accord.coordinate.CoordinationAdapter.Invoke.execute; -import static accord.coordinate.CoordinationAdapter.Invoke.persist; -import static accord.coordinate.CoordinationAdapter.Invoke.stabilise; import static accord.coordinate.ExecutePath.RECOVER; import static accord.coordinate.Infer.InvalidateAndCallback.locallyInvalidateAndCallback; import static accord.coordinate.Propose.Invalidate.proposeInvalidate; import static accord.coordinate.tracking.RequestStatus.Failed; import static accord.coordinate.tracking.RequestStatus.Success; -import static accord.messages.BeginRecovery.RecoverOk.maxAcceptedOrLater; +import static accord.messages.BeginRecovery.RecoverOk.maxAccepted; +import static accord.messages.BeginRecovery.RecoverOk.maxAcceptedNotTruncated; +import static accord.primitives.ProgressToken.TRUNCATED; import static accord.utils.Invariants.debug; import static accord.utils.Invariants.illegalState; @@ -142,10 +142,11 @@ AsyncResult awaitCommits(Node node, Deps waitOn) private final BiConsumer callback; private boolean isDone; - private final List recoverOks = new ArrayList<>(); + private final RecoverOk[] recoverOks; private final RecoveryTracker tracker; + private final Topology topology; private boolean isBallotPromised; - private final Map debug = debug() ? new HashMap<>() : null; + private final Map debug = debug() ? new TreeMap<>() : null; private Recover(Node node, Ballot ballot, TxnId txnId, Txn txn, FullRoute route, BiConsumer callback, Topologies topologies) { @@ -162,6 +163,8 @@ private Recover(Node node, Ballot ballot, TxnId txnId, Txn txn, FullRoute rou this.callback = callback; assert topologies.oldestEpoch() == topologies.currentEpoch() && topologies.currentEpoch() == txnId.epoch(); this.tracker = new RecoveryTracker(topologies); + this.topology = topologies.current(); + this.recoverOks = new RecoverOk[topology.nodes().size()]; } @Override @@ -210,7 +213,7 @@ private static Recover recover(Node node, Ballot ballot, TxnId txnId, Txn txn, F return recover; } - void start(Set nodes) + void start(Collection nodes) { node.send(nodes, to -> new BeginRecovery(to, tracker.topologies(), txnId, txn, route, ballot), this); } @@ -230,7 +233,7 @@ public synchronized void onSuccess(Id from, RecoverReply reply) } RecoverOk ok = (RecoverOk) reply; - recoverOks.add(ok); + recoverOks[topology.nodes().find(from)] = ok; boolean fastPath = ok.executeAt.compareTo(txnId) == 0; if (tracker.recordSuccess(from, fastPath) == Success) recover(); @@ -241,18 +244,18 @@ private void recover() Invariants.checkState(!isBallotPromised); isBallotPromised = true; - // first look for the most recent Accept (or later); if present, go straight to proposing it again - RecoverOk acceptOrCommit = maxAcceptedOrLater(recoverOks); - if (acceptOrCommit != null) + List recoverOkList = Arrays.asList(recoverOks); + RecoverOk acceptOrCommit = maxAccepted(recoverOkList); + RecoverOk acceptOrCommitNotTruncated = acceptOrCommit == null || acceptOrCommit.status != Status.Truncated + ? acceptOrCommit : maxAcceptedNotTruncated(recoverOkList); + + if (acceptOrCommitNotTruncated != null) { - Timestamp executeAt = acceptOrCommit.executeAt; - switch (acceptOrCommit.status) + Timestamp executeAt = acceptOrCommitNotTruncated.executeAt; + switch (acceptOrCommitNotTruncated.status) { - default: throw illegalState("Unknown status: " + acceptOrCommit.status); - case Truncated: - callback.accept(ProgressToken.TRUNCATED, null); - return; - + default: throw new AssertionError("Unhandled Status: " + acceptOrCommitNotTruncated.status); + case Truncated: throw illegalState("Truncate should be filtered"); case Invalidated: { commitInvalidate(); @@ -264,16 +267,16 @@ private void recover() { withCommittedDeps(executeAt, stableDeps -> { // TODO (future development correctness): when writes/result are partially replicated, need to confirm we have quorum of these - persist(adapter, node, tracker.topologies(), route, txnId, txn, executeAt, stableDeps, acceptOrCommit.writes, acceptOrCommit.result, null); + adapter.persist(node, tracker.topologies(), route, txnId, txn, executeAt, stableDeps, acceptOrCommitNotTruncated.writes, acceptOrCommitNotTruncated.result, null); }); - accept(acceptOrCommit.result, null); + accept(acceptOrCommitNotTruncated.result, null); return; } case Stable: { withCommittedDeps(executeAt, stableDeps -> { - execute(adapter, node, tracker.topologies(), route, RECOVER, txnId, txn, executeAt, stableDeps, this); + adapter.execute(node, tracker.topologies(), route, RECOVER, txnId, txn, executeAt, stableDeps, this); }); return; } @@ -282,7 +285,7 @@ private void recover() case Committed: { withCommittedDeps(executeAt, committedDeps -> { - stabilise(adapter, node, tracker.topologies(), route, ballot, txnId, txn, executeAt, committedDeps, this); + adapter.stabilise(node, tracker.topologies(), route, ballot, txnId, txn, executeAt, committedDeps, this); }); return; } @@ -292,8 +295,8 @@ private void recover() // TODO (desired, behaviour): if we didn't find Accepted in *every* shard, consider invalidating for consistency of behaviour // however, note that we may have taken the fast path and recovered, so we can only do this if acceptedOrCommitted=Ballot.ZERO // (otherwise recovery was attempted and did not invalidate, so it must have determined it needed to complete) - Deps proposeDeps = LatestDeps.mergeProposal(recoverOks, ok -> ok.deps); - propose(acceptOrCommit.executeAt, proposeDeps); + Deps proposeDeps = LatestDeps.mergeProposal(recoverOkList, ok -> ok == null ? null : ok.deps); + propose(acceptOrCommitNotTruncated.executeAt, proposeDeps); return; } @@ -309,16 +312,32 @@ private void recover() } } - if (tracker.rejectsFastPath() || recoverOks.stream().anyMatch(ok -> ok.rejectsFastPath)) + if (acceptOrCommit != acceptOrCommitNotTruncated) + { + boolean allShardsTruncated = true; + for (Shard shard : topology.shards()) + { + RecoverOk maxReply = maxAccepted(topology.nodes().select(recoverOks, shard.nodes)); + allShardsTruncated &= maxReply.status == Status.Truncated; + } + if (allShardsTruncated) + { + isDone = true; + callback.accept(TRUNCATED, null); + return; + } + } + + if (tracker.rejectsFastPath() || Stream.of(recoverOks).anyMatch(ok -> ok != null && ok.rejectsFastPath)) { invalidate(); return; } // should all be PreAccept - Deps proposeDeps = LatestDeps.mergeProposal(recoverOks, ok -> ok.deps); - Deps earlierAcceptedNoWitness = Deps.merge(recoverOks, ok -> ok.earlierAcceptedNoWitness); - Deps earlierCommittedWitness = Deps.merge(recoverOks, ok -> ok.earlierCommittedWitness); + Deps proposeDeps = LatestDeps.mergeProposal(recoverOkList, ok -> ok == null ? null : ok.deps); + Deps earlierAcceptedNoWitness = Deps.merge(recoverOkList, ok -> ok == null ? null : ok.earlierAcceptedNoWitness); + Deps earlierCommittedWitness = Deps.merge(recoverOkList, ok -> ok == null ? null : ok.earlierCommittedWitness); earlierAcceptedNoWitness = earlierAcceptedNoWitness.without(earlierCommittedWitness::contains); if (!earlierAcceptedNoWitness.isEmpty()) { @@ -341,7 +360,7 @@ private void recover() private void withCommittedDeps(Timestamp executeAt, Consumer withDeps) { - LatestDeps.MergedCommitResult merged = LatestDeps.mergeCommit(txnId, executeAt, recoverOks, ok -> ok.deps); + LatestDeps.MergedCommitResult merged = LatestDeps.mergeCommit(txnId, executeAt, Arrays.asList(recoverOks), ok -> ok == null ? null : ok.deps); node.withEpoch(executeAt.epoch(), () -> { Seekables missing = txn.keys().subtract(merged.sufficientFor); if (missing.isEmpty()) @@ -350,9 +369,16 @@ private void withCommittedDeps(Timestamp executeAt, Consumer withDeps) } else { - CollectDeps.withDeps(node, txnId, missing.toRoute(route.homeKey()), missing, executeAt, (extraDeps, fail) -> { - if (fail != null) node.agent().onHandledException(fail); - else withDeps.accept(merged.deps.with(extraDeps)); + CollectDeps.withDeps(node, txnId, route, missing.toParticipants(), missing, executeAt, (extraDeps, fail) -> { + if (fail != null) + { + isDone = true; + callback.accept(null, fail); + } + else + { + withDeps.accept(merged.deps.with(extraDeps)); + } }); } }); @@ -360,7 +386,7 @@ private void withCommittedDeps(Timestamp executeAt, Consumer withDeps) private void invalidate() { - proposeInvalidate(node, ballot, txnId, route.someParticipatingKey(), (success, fail) -> { + proposeInvalidate(node, ballot, txnId, route.homeKey(), (success, fail) -> { if (fail != null) accept(null, fail); else commitInvalidate(); }); @@ -370,7 +396,7 @@ private void commitInvalidate() { // If not accepted then the executeAt is not consistent cross the peers and likely different on every node. There is also an edge case // when ranges are removed from the topology, during this case the executeAt won't know the ranges and the invalidate commit will fail. - Timestamp invalidateUntil = recoverOks.stream().map(ok -> ok.status.hasBeen(Status.Accepted) ? ok.executeAt : ok.txnId).reduce(txnId, Timestamp::max); + Timestamp invalidateUntil = Stream.of(recoverOks).map(ok -> ok == null ? null : ok.status.hasBeen(Status.Accepted) ? ok.executeAt : ok.txnId).reduce(txnId, Timestamp::nonNullOrMax); node.withEpoch(invalidateUntil.epoch(), () -> Commit.Invalidate.commitInvalidate(node, txnId, route, invalidateUntil)); isDone = true; locallyInvalidateAndCallback(node, txnId, invalidateUntil, route, ProgressToken.INVALIDATED, callback); @@ -378,7 +404,7 @@ private void commitInvalidate() private void propose(Timestamp executeAt, Deps deps) { - node.withEpoch(executeAt.epoch(), () -> Invoke.propose(adapter, node, route, ballot, txnId, txn, executeAt, deps, this)); + node.withEpoch(executeAt.epoch(), () -> adapter.propose(node, null, route, ballot, txnId, txn, executeAt, deps, this)); } private void retry() diff --git a/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java b/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java index 698716396..2e53f7ebf 100644 --- a/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java +++ b/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java @@ -43,7 +43,6 @@ import javax.annotation.Nullable; import static accord.coordinate.CoordinationAdapter.Factory.Step.InitiateRecovery; -import static accord.coordinate.CoordinationAdapter.Invoke.persist; import static accord.local.Status.Durability.Majority; import static accord.local.Status.KnownDeps.DepsKnown; import static accord.local.Status.KnownExecuteAt.ExecuteAtKnown; @@ -186,9 +185,9 @@ else if (!known.definition.isOrWasKnown()) // so that we know to WasApply, but not if (known.executeAt == ExecuteAtKnown && known.deps == DepsKnown && known.outcome == Apply) { - Invariants.checkState(full.stableDeps.covering.containsAll(sendTo)); - Invariants.checkState(full.partialTxn.covering().containsAll(sendTo)); - persist(node.coordinationAdapter(txnId, InitiateRecovery), node, route, sendTo, txnId, full.partialTxn, full.executeAt, full.stableDeps, full.writes, full.result, null); + Invariants.checkState(full.stableDeps.covers(sendTo)); + Invariants.checkState(full.partialTxn.covers(sendTo)); + node.coordinationAdapter(txnId, InitiateRecovery).persist(node, null, route, sendTo, txnId, full.partialTxn, full.executeAt, full.stableDeps, full.writes, full.result, null); } propagate = full; } @@ -212,7 +211,7 @@ else if (!known.definition.isOrWasKnown()) { Deps deps = full.stableDeps.reconstitute(route()); node.withEpoch(full.executeAt.epoch(), () -> { - persist(node.coordinationAdapter(txnId, InitiateRecovery), node, topologies, route(), txnId, txn, full.executeAt, deps, full.writes, full.result, null); + node.coordinationAdapter(txnId, InitiateRecovery).persist(node, topologies, route(), txnId, txn, full.executeAt, deps, full.writes, full.result, null); }); callback.accept(APPLIED, null); } diff --git a/accord-core/src/main/java/accord/coordinate/Stabilise.java b/accord-core/src/main/java/accord/coordinate/Stabilise.java index d204fab77..56bf8b3e8 100644 --- a/accord-core/src/main/java/accord/coordinate/Stabilise.java +++ b/accord-core/src/main/java/accord/coordinate/Stabilise.java @@ -18,8 +18,8 @@ package accord.coordinate; -import java.util.HashMap; import java.util.Map; +import java.util.TreeMap; import java.util.function.BiConsumer; import accord.coordinate.tracking.QuorumTracker; @@ -38,7 +38,6 @@ import accord.primitives.TxnId; import accord.topology.Topologies; -import static accord.coordinate.CoordinationAdapter.Invoke.execute; import static accord.coordinate.ExecutePath.SLOW; import static accord.coordinate.tracking.RequestStatus.Failed; import static accord.messages.Commit.Kind.CommitWithTxn; @@ -54,7 +53,7 @@ public abstract class Stabilise implements Callback final Timestamp executeAt; final Deps stabiliseDeps; - private final Map debug = debug() ? new HashMap<>() : null; + private final Map debug = debug() ? new TreeMap<>() : null; final QuorumTracker stableTracker; final Topologies allTopologies; final BiConsumer callback; @@ -142,7 +141,7 @@ public void onCallbackFailure(Node.Id from, Throwable failure) protected void onStabilised() { - execute(adapter(), node, allTopologies, route, SLOW, txnId, txn, executeAt, stabiliseDeps, callback); + adapter().execute(node, allTopologies, route, SLOW, txnId, txn, executeAt, stabiliseDeps, callback); } protected abstract CoordinationAdapter adapter(); diff --git a/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java b/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java index e4936d29e..5c1408843 100644 --- a/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java +++ b/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java @@ -34,9 +34,9 @@ public class StabiliseSyncPoint> extends Stabilise> { final CoordinationAdapter> adapter; - StabiliseSyncPoint(CoordinationAdapter> adapter, Node node, Topologies coordinates, Topologies allTopologies, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer, Throwable> callback) + StabiliseSyncPoint(CoordinationAdapter> adapter, Node node, Topologies coordinates, Topologies all, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer, Throwable> callback) { - super(node, coordinates, allTopologies, route, txnId, ballot, txn, executeAt, unstableDeps, callback); + super(node, coordinates, all, route, txnId, ballot, txn, executeAt, unstableDeps, callback); this.adapter = adapter; } diff --git a/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java b/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java index da36f659f..ffd174dd7 100644 --- a/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java +++ b/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java @@ -34,9 +34,9 @@ public class StabiliseTxn extends Stabilise { - StabiliseTxn(Node node, Topologies coordinates, Topologies allTopologies, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer callback) + StabiliseTxn(Node node, Topologies coordinates, Topologies all, FullRoute route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer callback) { - super(node, coordinates, allTopologies, route, txnId, ballot, txn, executeAt, unstableDeps, callback); + super(node, coordinates, all, route, txnId, ballot, txn, executeAt, unstableDeps, callback); } @Override diff --git a/accord-core/src/main/java/accord/coordinate/tracking/AbstractTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/AbstractTracker.java index f74bcee42..11b311cbf 100644 --- a/accord-core/src/main/java/accord/coordinate/tracking/AbstractTracker.java +++ b/accord-core/src/main/java/accord/coordinate/tracking/AbstractTracker.java @@ -175,7 +175,7 @@ public boolean all(Predicate test) return true; } - public Set nodes() + public Collection nodes() { return topologies.nodes(); } diff --git a/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java b/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java index a88c5418d..6f16b6c1d 100644 --- a/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java +++ b/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java @@ -53,6 +53,7 @@ import javax.annotation.Nullable; import static accord.local.SaveStatus.Applied; +import static accord.local.SaveStatus.TruncatedApply; import static accord.messages.ReadData.CommitOrReadNack.Insufficient; import static accord.primitives.Routables.Slice.Minimal; @@ -137,7 +138,7 @@ public void contact(Node.Id to, Ranges ranges) inflight.put(key, starting(to, ranges)); Ranges ownedRanges = ownedRangesForNode(to); Invariants.checkArgument(ownedRanges.containsAll(ranges), "Got a reply from %s for ranges %s, but owned ranges %s does not contain all the ranges", to, ranges, ownedRanges); - PartialDeps partialDeps = syncPoint.waitFor.slice(ownedRanges, ranges); + PartialDeps partialDeps = syncPoint.waitFor.intersecting(ranges); node.send(to, newFetchRequest(syncPoint.sourceEpoch(), syncPoint.syncId, ranges, partialDeps, rangeReadTxn(ranges)), new Callback() { @Override @@ -236,7 +237,7 @@ void abort(Ranges abort) public static class FetchRequest extends ReadData { - private static final ExecuteOn EXECUTE_ON = new ExecuteOn(Applied, Applied); + private static final ExecuteOn EXECUTE_ON = new ExecuteOn(Applied, TruncatedApply); public final PartialTxn read; public final PartialDeps partialDeps; @@ -261,7 +262,8 @@ public ReadType kind() } @Override - protected AsyncChain beginRead(SafeCommandStore safeStore, Timestamp executeAt, PartialTxn txn, Ranges unavailable) { + protected AsyncChain beginRead(SafeCommandStore safeStore, Timestamp executeAt, PartialTxn txn, Ranges unavailable) + { return read.read(safeStore, executeAt, unavailable); } diff --git a/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java b/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java index df39788b8..1f62888aa 100644 --- a/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java +++ b/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java @@ -71,7 +71,8 @@ * The work for CoordinateShardDurable is further subdivided where each subrange a node operates on is divided a fixed * number of times and then processed one at a time with a fixed wait between them. * - * // TODO (expected): cap number of coordinations we can have in flight at once + * TODO (expected): cap number of coordinations we can have in flight at once + * TODO (expected): do not start new ExclusiveSyncPoint if we have more than X already agreed and not yet applied * Didn't go with recurring because it doesn't play well with async execution of these tasks */ public class CoordinateDurabilityScheduling @@ -214,11 +215,19 @@ private void run() private void startShardSync(Ranges ranges) { TxnId at = node.nextTxnId(ExclusiveSyncPoint, Domain.Range); + logger.trace("{}: Coordinating ExclusiveSyncPoint for local shard durability of {}", at, ranges); node.scheduler().once(() -> node.withEpoch(at.epoch(), () -> { CoordinateSyncPoint.exclusive(node, at, ranges) .addCallback((success, fail) -> { - if (fail != null) logger.trace("Exception coordinating exclusive sync point for local shard durability of {}", ranges, fail); - else coordinateShardDurableAfterExclusiveSyncPoint(node, success); + if (fail != null) + { + logger.trace("{}: Exception coordinating ExclusiveSyncPoint for local shard durability of {}", at, ranges, fail); + } + else + { + coordinateShardDurableAfterExclusiveSyncPoint(node, success); + logger.trace("{}: Successfully coordinated ExclusiveSyncPoint for local shard durability of {}", at, ranges); + } }); }), txnIdLagMicros, MICROSECONDS); } @@ -264,8 +273,8 @@ private List rangesToShardSync(long nowMicros) { Shard shard = e.getKey(); int index = e.getValue(); - long microsOffset = (index * shardCycleTimeMicros) / shard.rf(); int shardCycleTimeMicros = Math.max(this.shardCycleTimeMicros, Ints.saturatedCast(shard.rf() * 3L * frequencyMicros)); + long microsOffset = (index * (long)shardCycleTimeMicros) / shard.rf(); long prevSyncTimeMicros = Math.max(prevShardSyncTimeMicros, nowMicros - ((shardCycleTimeMicros / shard.rf()) / 2L)); int from = (int) ((prevSyncTimeMicros + microsOffset) % shardCycleTimeMicros); int to = (int) ((nowMicros + microsOffset) % shardCycleTimeMicros); diff --git a/accord-core/src/main/java/accord/impl/ErasedSafeCommand.java b/accord-core/src/main/java/accord/impl/ErasedSafeCommand.java index 675f62fa1..d2285f91d 100644 --- a/accord-core/src/main/java/accord/impl/ErasedSafeCommand.java +++ b/accord-core/src/main/java/accord/impl/ErasedSafeCommand.java @@ -28,7 +28,8 @@ import static accord.local.Listeners.Immutable.EMPTY; import static accord.local.SaveStatus.Erased; -import static accord.local.SaveStatus.ErasedOrInvalidated; +import static accord.local.SaveStatus.ErasedOrInvalidOrVestigial; +import static accord.local.Status.Durability.NotDurable; import static accord.local.Status.Durability.UniversalOrInvalidated; public class ErasedSafeCommand extends SafeCommand @@ -38,8 +39,8 @@ public class ErasedSafeCommand extends SafeCommand public ErasedSafeCommand(TxnId txnId, SaveStatus saveStatus) { super(txnId); - Invariants.checkArgument(saveStatus == Erased || saveStatus == ErasedOrInvalidated); - this.erased = new Command.Truncated(txnId, saveStatus, UniversalOrInvalidated, null, null, EMPTY, null, null); + Invariants.checkArgument(saveStatus == Erased || saveStatus == ErasedOrInvalidOrVestigial); + this.erased = new Command.Truncated(txnId, saveStatus, saveStatus == ErasedOrInvalidOrVestigial ? NotDurable : UniversalOrInvalidated, null, null, EMPTY, null, null); } @Override diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java index ab1b3e2b9..e6a2ac27a 100644 --- a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java +++ b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java @@ -79,6 +79,7 @@ import static accord.local.SafeCommandStore.TestStartedAt.STARTED_BEFORE; import static accord.local.SaveStatus.Applying; import static accord.local.SaveStatus.Erased; +import static accord.local.SaveStatus.ErasedOrInvalidOrVestigial; import static accord.local.SaveStatus.ReadyToExecute; import static accord.local.Status.Applied; import static accord.local.Status.Stable; @@ -121,6 +122,18 @@ public NavigableMap unsafeCommands() return commands; } + @VisibleForTesting + public NavigableMap unsafeCommandsByExecuteAt() + { + return commandsByExecuteAt; + } + + @VisibleForTesting + public NavigableMap unsafeCommandsForKey() + { + return commandsForKey; + } + @Override public Agent agent() { @@ -186,7 +199,7 @@ public void onChange(SafeCommandStore safeStore, SafeCommand safeCommand) break; Participants participantsOfPrev = prev.route().participants(ranges, Minimal); - Participants intersectingParticipants = participants.intersect(participantsOfPrev); + Participants intersectingParticipants = participants.intersecting(participantsOfPrev, Minimal); if (intersectingParticipants.isEmpty()) continue; @@ -518,7 +531,7 @@ public AsyncChain maxAppliedFor(Seekables keysOrRanges, Ranges @Override public String toString() { - return getClass().getSimpleName() + "{" + "id=" + id + ",node=" + time.id().id + '}'; + return getClass().getSimpleName() + "{id=" + id + ",node=" + time.id().id + '}'; } static class RangeCommand @@ -734,7 +747,7 @@ protected void update(Command prev, Command updated) return; // TODO (expected): consider removing if erased - if (updated.saveStatus() == Erased) + if (updated.saveStatus() == Erased || updated.saveStatus() == ErasedOrInvalidOrVestigial) return; Seekables keysOrRanges = updated.keysOrRanges(); diff --git a/accord-core/src/main/java/accord/impl/SimpleProgressLog.java b/accord-core/src/main/java/accord/impl/SimpleProgressLog.java index 0ab1962f6..334858ecd 100644 --- a/accord-core/src/main/java/accord/impl/SimpleProgressLog.java +++ b/accord-core/src/main/java/accord/impl/SimpleProgressLog.java @@ -69,6 +69,7 @@ import static accord.local.PreLoadContext.empty; import static accord.local.SaveStatus.LocalExecution.NotReady; import static accord.local.SaveStatus.LocalExecution.WaitingToApply; +import static accord.local.Status.KnownRoute.Full; import static accord.local.Status.PreApplied; import static accord.utils.Invariants.illegalState; @@ -292,7 +293,7 @@ void recordBlocking(LocalExecution blockedUntil, @Nullable Route route, @Null { Invariants.checkState(route != null || participants != null, "Route and participants are both undefined"); Invariants.checkState(participants == null || !participants.isEmpty(), "participants is empty"); - Invariants.checkState(route == null || route.hasParticipants(), "Route %s does not have participants", route); + Invariants.checkState(route == null || !route.isEmpty(), "Route %s is empty", route); this.route = Route.merge(this.route, (Route)route); this.participants = Participants.merge(this.participants, (Participants) participants); @@ -340,7 +341,7 @@ void run(SafeCommandStore safeStore, SafeCommand safeCommand) setProgress(Expected); // TODO (required): we might not be in the coordinating OR execution epochs if an accept round contacted us but recovery did not (quite hard to achieve) - Invariants.checkState(fail != null || !blockedUntil.isSatisfiedBy(success.propagates())); + Invariants.checkState(fail != null || !blockedUntil.isSatisfiedBy(success.propagates()) || success.route != Full); }).begin(commandStore.agent()); }; diff --git a/accord-core/src/main/java/accord/local/Bootstrap.java b/accord-core/src/main/java/accord/local/Bootstrap.java index f37af6d32..9d2d7f573 100644 --- a/accord-core/src/main/java/accord/local/Bootstrap.java +++ b/accord-core/src/main/java/accord/local/Bootstrap.java @@ -118,6 +118,12 @@ class Attempt implements FetchRanges, BiConsumer void start(SafeCommandStore safeStore0) { + if (valid.isEmpty()) + { + maybeComplete(); + return; + } + globalSyncId = node.nextTxnId(ExclusiveSyncPoint, Routable.Domain.Range); localSyncId = globalSyncId.as(LocalOnly).withEpoch(epoch); Invariants.checkArgument(epoch <= globalSyncId.epoch(), "Attempting to use local epoch %d which is larger than global epoch %d", epoch, globalSyncId.epoch()); @@ -134,14 +140,12 @@ void start(SafeCommandStore safeStore0) Ranges commitRanges = valid; store.markBootstrapping(safeStore0, globalSyncId, valid); CoordinateSyncPoint.exclusive(node, globalSyncId, commitRanges) - // TODO (required, correcness) : PreLoadContext only works with Seekables, which doesn't allow mixing Keys and Ranges... But Deps has both Keys AND Ranges! - // TODO (required): is localSyncId even being used anymore // ATM all known implementations store ranges in-memory, but this will not be true soon, so this will need to be addressed .flatMap(syncPoint -> node.withEpoch(epoch, () -> store.submit(contextFor(localSyncId, syncPoint.waitFor.keyDeps.keys(), KeyHistory.COMMANDS), safeStore1 -> { if (valid.isEmpty()) // we've lost ownership of the range return AsyncResults.success(Ranges.EMPTY); - Commands.createBootstrapCompleteMarkerTransaction(safeStore1, localSyncId, syncPoint, valid); + Commands.createBootstrapCompleteMarkerTransaction(safeStore1, localSyncId, valid); safeStore1.registerHistoricalTransactions(syncPoint.waitFor); return fetch = safeStore1.dataStore().fetch(node, safeStore1, valid, syncPoint, this); }))) diff --git a/accord-core/src/main/java/accord/local/Cleanup.java b/accord-core/src/main/java/accord/local/Cleanup.java index 5da56e3c6..36714b4ff 100644 --- a/accord-core/src/main/java/accord/local/Cleanup.java +++ b/accord-core/src/main/java/accord/local/Cleanup.java @@ -57,14 +57,6 @@ public enum Cleanup this.appliesIfNot = appliesIfNot; } - /** - * Durability has been achieved globally across all keys/ranges that make this txnId's metadata safe to purge - */ - public static boolean isSafeToCleanup(DurableBefore durableBefore, TxnId txnId) - { - return durableBefore.min(txnId) == UniversalOrInvalidated; - } - /** * Durability has been achieved for the specific keys associated with this txnId that makes its metadata safe to purge */ @@ -128,14 +120,12 @@ public static Cleanup shouldCleanup(TxnId txnId, Status status, Durability durab // - we can impose additional validations here IF we receive an epoch upper bound // - we should be more robust to the presence/absence of executeAt // - be cognisant of future epochs that participated only for PreAccept/Accept, but where txn was not committed to execute in the epoch (this is why we provide null toEpoch here) - if (route.isParticipatingHomeKey() || redundantBefore.get(txnId, NO_UPPER_BOUND, route.homeKey()) == NOT_OWNED) - illegalState("Command " + txnId + " that is being loaded is not owned by this shard on route " + route); + illegalState("Command " + txnId + " that is being loaded is not owned by this shard on route " + route); } } switch (redundant) { default: throw new AssertionError(); - case NOT_OWNED: case LIVE: case PARTIALLY_PRE_BOOTSTRAP_OR_STALE: case PRE_BOOTSTRAP_OR_STALE: diff --git a/accord-core/src/main/java/accord/local/Command.java b/accord-core/src/main/java/accord/local/Command.java index d082b6767..8c98882da 100644 --- a/accord-core/src/main/java/accord/local/Command.java +++ b/accord-core/src/main/java/accord/local/Command.java @@ -56,6 +56,7 @@ import static accord.local.Command.AbstractCommand.validate; import static accord.local.Listeners.Immutable.EMPTY; +import static accord.local.SaveStatus.ErasedOrInvalidOrVestigial; import static accord.local.SaveStatus.Uninitialised; import static accord.local.Status.Durability.Local; import static accord.local.Status.Durability.NotDurable; @@ -143,7 +144,7 @@ public PreLoadContext listenerPreLoadContext(TxnId caller) private static Durability durability(Durability durability, SaveStatus status) { - if (status.compareTo(SaveStatus.PreApplied) >= 0 && !status.hasBeen(Invalidated) && durability == NotDurable) + if (durability == NotDurable && status.compareTo(SaveStatus.PreApplied) >= 0 && status.compareTo(ErasedOrInvalidOrVestigial) < 0) return Local; // not necessary anywhere, but helps for logical consistency return durability; } @@ -234,7 +235,7 @@ private static SaveStatus validateCommandClass(TxnId txnId, SaveStatus status, C if (txnId.kind().awaitsOnlyDeps()) return validateCommandClass(status, TruncatedAwaitsOnlyDeps.class, klass); case Erased: - case ErasedOrInvalidated: + case ErasedOrInvalidOrVestigial: case Invalidated: return validateCommandClass(status, Truncated.class, klass); default: @@ -778,14 +779,19 @@ public static Truncated erased(Command command) return erased(command.txnId(), durability, command.route()); } - public static Truncated erasedOrInvalidated(TxnId txnId, Status.Durability durability, Route route) + public static Truncated erased(TxnId txnId, Status.Durability durability, Route route) { - return validate(new Truncated(txnId, SaveStatus.ErasedOrInvalidated, durability, route, null, EMPTY, null, null)); + return validate(new Truncated(txnId, SaveStatus.Erased, durability, route, null, EMPTY, null, null)); } - public static Truncated erased(TxnId txnId, Status.Durability durability, Route route) + public static Truncated erasedOrInvalidOrVestigial(Command command) { - return validate(new Truncated(txnId, SaveStatus.Erased, durability, route, null, EMPTY, null, null)); + return erasedOrInvalidOrVestigial(command.txnId(), command.durability(), command.route()); + } + + public static Truncated erasedOrInvalidOrVestigial(TxnId txnId, Status.Durability durability, Route route) + { + return validate(new Truncated(txnId, SaveStatus.ErasedOrInvalidOrVestigial, durability, route, null, EMPTY, null, null)); } public static Truncated truncatedApply(Command command) @@ -1298,6 +1304,13 @@ public boolean isWaitingOnKey() return waitingOn.lastSetBit() >= txnIdCount(); } + public Key lastWaitingOnKey() + { + int keyIndex = waitingOn.lastSetBit() - txnIdCount(); + if (keyIndex < 0) return null; + return keys.get(keyIndex); + } + public boolean isWaitingOnKey(int keyIndex) { Invariants.checkIndex(keyIndex, keys.size()); diff --git a/accord-core/src/main/java/accord/local/CommandStore.java b/accord-core/src/main/java/accord/local/CommandStore.java index cdb8a4679..fb97cf42b 100644 --- a/accord-core/src/main/java/accord/local/CommandStore.java +++ b/accord-core/src/main/java/accord/local/CommandStore.java @@ -465,7 +465,7 @@ private void fetchMajorityDeps(AsyncResults.SettableResult coordination, N Timestamp before = Timestamp.minForEpoch(epoch); FullRoute route = node.computeRoute(id, ranges); // TODO (required): we need to ensure anyone we receive a reply from proposes newer timestamps for anything we don't see - CollectDeps.withDeps(node, id, route, ranges, before, (deps, fail) -> { + CollectDeps.withDeps(node, id, route, route, ranges, before, (deps, fail) -> { if (fail != null) { fetchMajorityDeps(coordination, node, epoch, ranges); diff --git a/accord-core/src/main/java/accord/local/CommandStores.java b/accord-core/src/main/java/accord/local/CommandStores.java index 20f08a8e0..231f1b054 100644 --- a/accord-core/src/main/java/accord/local/CommandStores.java +++ b/accord-core/src/main/java/accord/local/CommandStores.java @@ -40,6 +40,7 @@ import accord.api.RoutingKey; import accord.local.CommandStore.EpochUpdateHolder; import accord.primitives.EpochSupplier; +import accord.primitives.Participants; import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.Routables; @@ -115,9 +116,9 @@ CommandStore create(int id, EpochUpdateHolder rangesForEpoch) } } - static class ShardHolder + protected static class ShardHolder { - final CommandStore store; + public final CommandStore store; RangesForEpoch ranges; ShardHolder(CommandStore store) @@ -335,9 +336,9 @@ public Ranges removed(long presentIn, long removedByInclusive) } } - static class Snapshot + protected static class Snapshot { - final ShardHolder[] shards; + public final ShardHolder[] shards; final Int2ObjectHashMap byId; final Topology local; final Topology global; @@ -665,6 +666,11 @@ public CommandStore select(Route route) return select(ranges -> ranges.intersects(route)); } + public CommandStore select(Participants participants) + { + return select(ranges -> ranges.intersects(participants)); + } + private CommandStore select(Predicate fn) { ShardHolder[] shards = current.shards; @@ -723,4 +729,9 @@ public CommandStore unsafeForKey(Key key) } throw new IllegalArgumentException(); } + + protected Snapshot current() + { + return current; + } } diff --git a/accord-core/src/main/java/accord/local/Commands.java b/accord-core/src/main/java/accord/local/Commands.java index ecb0d6da8..14dfcc9ca 100644 --- a/accord-core/src/main/java/accord/local/Commands.java +++ b/accord-core/src/main/java/accord/local/Commands.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.Collections; +import java.util.function.BiPredicate; import java.util.function.Consumer; import javax.annotation.Nullable; @@ -44,7 +45,6 @@ import accord.primitives.Ranges; import accord.primitives.Route; import accord.primitives.Seekables; -import accord.primitives.SyncPoint; import accord.primitives.Timestamp; import accord.primitives.Txn; import accord.primitives.TxnId; @@ -57,18 +57,11 @@ import static accord.api.ProgressLog.ProgressShard.Home; import static accord.api.ProgressLog.ProgressShard.Local; import static accord.api.ProgressLog.ProgressShard.No; -import static accord.api.ProgressLog.ProgressShard.UnmanagedHome; -import static accord.api.ProgressLog.ProgressShard.Unsure; import static accord.local.Cleanup.ERASE; import static accord.local.Cleanup.shouldCleanup; import static accord.local.Command.Truncated.erased; import static accord.local.Command.Truncated.truncatedApply; import static accord.local.Command.Truncated.truncatedApplyWithOutcome; -import static accord.local.Commands.EnsureAction.Add; -import static accord.local.Commands.EnsureAction.Ignore; -import static accord.local.Commands.EnsureAction.Set; -import static accord.local.Commands.EnsureAction.TryAdd; -import static accord.local.Commands.EnsureAction.TrySet; import static accord.local.KeyHistory.TIMESTAMPS; import static accord.local.PreLoadContext.contextFor; import static accord.local.RedundantStatus.PRE_BOOTSTRAP_OR_STALE; @@ -77,13 +70,12 @@ import static accord.local.SaveStatus.LocalExecution.ReadyToExclude; import static accord.local.SaveStatus.LocalExecution.WaitingToApply; import static accord.local.SaveStatus.TruncatedApply; -import static accord.local.Status.Accepted; -import static accord.local.Status.AcceptedInvalidate; import static accord.local.Status.Applied; import static accord.local.Status.Committed; import static accord.local.Status.Durability; import static accord.local.Status.Invalidated; import static accord.local.Status.KnownExecuteAt.ExecuteAtKnown; +import static accord.local.Status.KnownRoute.Full; import static accord.local.Status.NotDefined; import static accord.local.Status.PreAccepted; import static accord.local.Status.PreApplied; @@ -103,16 +95,6 @@ private Commands() { } - private static Ranges covers(@Nullable PartialTxn txn) - { - return txn == null ? null : txn.covering(); - } - - private static Ranges covers(@Nullable PartialDeps deps) - { - return deps == null ? null : deps.covering; - } - private static boolean hasQuery(PartialTxn txn) { return txn != null && txn.query() != null; @@ -166,13 +148,14 @@ private static AcceptOutcome preacceptOrRecover(SafeCommandStore safeStore, Safe return ballot.equals(Ballot.ZERO) ? AcceptOutcome.Redundant : AcceptOutcome.Success; } - Ranges preacceptRanges = preacceptRanges(safeStore, txnId, acceptEpoch); - Invariants.checkState(!preacceptRanges.isEmpty()); - ProgressShard shard = progressShard(route, progressKey, preacceptRanges); - Invariants.checkState(validate(command.status(), command, Ranges.EMPTY, preacceptRanges, shard, route, Set, partialTxn, Set, null, Ignore)); + Ranges coordinateRanges = coordinateRanges(safeStore, txnId); + Ranges acceptRanges = acceptRanges(safeStore, txnId, acceptEpoch, coordinateRanges); + Invariants.checkState(!acceptRanges.isEmpty()); + ProgressShard shard = progressShard(route, progressKey, coordinateRanges); + Invariants.checkState(validate(SaveStatus.PreAccepted, command, acceptRanges, route, partialTxn, null, null)); // FIXME: this should go into a consumer method - CommonAttributes attrs = set(command, Ranges.EMPTY, preacceptRanges, shard, route, partialTxn, Set, null, Ignore); + CommonAttributes attrs = set(SaveStatus.PreAccepted, command, command, acceptRanges, ballot, route, partialTxn, null); if (command.executeAt() == null) { // unlike in the Accord paper, we partition shards within a node, so that to ensure a total order we must either: @@ -249,12 +232,12 @@ public static AcceptOutcome accept(SafeCommandStore safeStore, TxnId txnId, Ball Invariants.checkState(!acceptRanges.isEmpty()); ProgressShard shard = progressShard(route, progressKey, coordinateRanges); - Invariants.checkState(validate(command.status(), command, coordinateRanges, acceptRanges, shard, route, Ignore, null, Ignore, partialDeps, Set)); + Invariants.checkState(validate(SaveStatus.Accepted, command, acceptRanges, route, null, partialDeps, null)); // TODO (desired, clarity/efficiency): we don't need to set the route here, and perhaps we don't even need to // distributed partialDeps at all, since all we gain is not waiting for these transactions to commit during // recovery. We probably don't want to directly persist a Route in any other circumstances, either, to ease persistence. - CommonAttributes attrs = set(command, coordinateRanges, acceptRanges, shard, route, null, Ignore, partialDeps, Set); + CommonAttributes attrs = set(SaveStatus.Accepted, command, command, acceptRanges, ballot, route, null, partialDeps); keysOrRanges = keysOrRanges.slice(acceptRanges); command = safeCommand.accept(safeStore, keysOrRanges, attrs, executeAt, ballot); @@ -339,10 +322,10 @@ public static CommitOutcome commit(SafeCommandStore safeStore, SafeCommand safeC Ranges acceptRanges = acceptRanges(safeStore, txnId, executeAt, coordinateRanges); ProgressShard shard = progressShard(route, progressKey, coordinateRanges); - if (!validate(command.status(), command, coordinateRanges, acceptRanges, shard, route, Add, partialTxn, Add, partialDeps, Set)) + if (!validate(newStatus, command, acceptRanges, route, partialTxn, partialDeps, null)) return CommitOutcome.Insufficient; - CommonAttributes attrs = set(command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, Set); + CommonAttributes attrs = set(newStatus, command, command, acceptRanges, ballot, route, partialTxn, partialDeps); logger.trace("{}: committed with executeAt: {}, deps: {}", txnId, executeAt, partialDeps); if (newStatus == SaveStatus.Stable) @@ -398,29 +381,20 @@ public static CommitOutcome precommit(SafeCommandStore safeStore, SafeCommand sa return CommitOutcome.Success; } - public static void createBootstrapCompleteMarkerTransaction(SafeCommandStore safeStore, TxnId localSyncId, SyncPoint syncPoint, Seekables keys) + public static void createBootstrapCompleteMarkerTransaction(SafeCommandStore safeStore, TxnId localSyncId, Seekables keys) { SafeCommand safeCommand = safeStore.get(localSyncId); Command command = safeCommand.current(); Invariants.checkState(!command.hasBeen(Committed)); - createBootstrapCompleteMarkerTransaction(safeStore, localSyncId, keys, syncPoint.route()); - } - - private static void createBootstrapCompleteMarkerTransaction(SafeCommandStore safeStore, TxnId localSyncId, Seekables keys, Route route) - { - SafeCommand safeCommand = safeStore.get(localSyncId); - Command command = safeCommand.current(); - if (command.hasBeen(Stable)) - return; + FullRoute route = keys.toRoute(keys.get(0).someIntersectingRoutingKey(null)); Ranges coordinateRanges = coordinateRanges(safeStore, localSyncId); // TODO (desired, consider): in the case of sync points, the coordinator is unlikely to be a home shard, do we mind this? should document at least Txn emptyTxn = safeStore.agent().emptyTxn(localSyncId.kind(), keys); - ProgressShard progressShard = coordinateRanges.contains(route.homeKey()) ? UnmanagedHome : No; - PartialDeps none = Deps.NONE.slice(coordinateRanges); + PartialDeps none = Deps.NONE.intersecting(route); PartialTxn partialTxn = emptyTxn.slice(coordinateRanges, true); - Invariants.checkState(validate(command.status(), command, Ranges.EMPTY, coordinateRanges, progressShard, route, Set, partialTxn, Set, none, Set)); - CommonAttributes newAttributes = set(command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, none, Set); + Invariants.checkState(validate(SaveStatus.Stable, command, coordinateRanges, route, partialTxn, none, null)); + CommonAttributes newAttributes = set(SaveStatus.Stable, command, command, coordinateRanges, Ballot.ZERO, route, partialTxn, none); safeCommand.stable(safeStore, newAttributes, Ballot.ZERO, localSyncId, WaitingOn.EMPTY); safeStore.notifyListeners(safeCommand); } @@ -432,15 +406,12 @@ public static void ephemeralRead(SafeCommandStore safeStore, SafeCommand safeCom if (command.hasBeen(Stable)) return; - // TODO (required): by creating synthetic TxnId in future epochs we may not be evictable - // but for ephemeral reads we want parallel eviction - or preferably no durability - anyway - txnId = txnId.withEpoch(executeAtEpoch); - + // BREAKING CHANGE NOTE: if in future we support a CommandStore adopting additional ranges (rather than only shedding them) + // then we need to revisit how we execute transactions that awaitsOnlyDeps, as they may need additional + // information to execute in the eventual execution epoch (that they didn't know they needed when they were made stable) Ranges coordinateRanges = coordinateRanges(safeStore, txnId); - // TODO (desired, consider): in the case of sync points, the coordinator is unlikely to be a home shard, do we mind this? should document at least - ProgressShard progressShard = No; - Invariants.checkState(validate(command.status(), command, Ranges.EMPTY, coordinateRanges, progressShard, route, Set, partialTxn, Set, partialDeps, Set)); - CommonAttributes attrs = set(command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, partialDeps, Set); + Invariants.checkState(validate(SaveStatus.Stable, command, coordinateRanges, route, partialTxn, partialDeps, null)); + CommonAttributes attrs = set(SaveStatus.Stable, command, command, coordinateRanges, Ballot.ZERO, route, partialTxn, partialDeps); safeCommand.stable(safeStore, attrs, Ballot.ZERO, txnId, initialiseWaitingOn(safeStore, txnId, attrs, txnId, route)); maybeExecute(safeStore, safeCommand, false, true); } @@ -504,13 +475,12 @@ else if (command.hasBeen(PreCommitted) && !executeAt.equals(command.executeAt()) Ranges coordinateRanges = coordinateRanges(safeStore, txnId); Ranges acceptRanges = acceptRanges(safeStore, txnId, executeAt, coordinateRanges); - ProgressShard shard = progressShard(route, progressKey, coordinateRanges); - if (!validate(command.status(), command, coordinateRanges, acceptRanges, shard, route, TryAdd, partialTxn, Add, partialDeps, command.hasBeen(Committed) ? TryAdd : TrySet, safeStore)) + if (!validate(SaveStatus.PreApplied, command, acceptRanges, route, partialTxn, partialDeps, safeStore)) return ApplyOutcome.Insufficient; // TODO (expected, consider): this should probably be an assertion failure if !TrySet - CommonAttributes attrs = set(command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, command.hasBeen(Committed) ? TryAdd : TrySet); + CommonAttributes attrs = set(SaveStatus.PreApplied, command, command, acceptRanges, null, route, partialTxn, partialDeps); WaitingOn waitingOn = !command.hasBeen(Stable) ? initialiseWaitingOn(safeStore, txnId, attrs, executeAt, attrs.route()) : command.asCommitted().waitingOn(); safeCommand.preapplied(safeStore, attrs, executeAt, waitingOn, writes, result); @@ -532,7 +502,7 @@ public static void listenerUpdate(SafeCommandStore safeStore, SafeCommand safeLi // This listener must be a stale vestige // TODO (desired): would be nice to ensure these are deregistered explicitly, but would be costly Invariants.checkState(listener.saveStatus().isUninitialised() || listener.is(Truncated), "Listener status expected to be Uninitialised or Truncated, but was %s", listener.saveStatus()); - Invariants.checkState(updated.is(NotDefined) || updated.is(Truncated) || !updated.asCommitted().waitingOn().isWaitingOn(listener.txnId()), "Updated status expected to be Applied or NotDefined, but was %s", updated); + Invariants.checkState(updated.is(NotDefined) || updated.hasBeen(Truncated) || !updated.asCommitted().waitingOn().isWaitingOn(listener.txnId()), "Updated status expected to be Applied or NotDefined, but was %s", updated); safeUpdated.removeListener(listener.asListener()); return; } @@ -577,12 +547,9 @@ private static Ranges coordinateRanges(SafeCommandStore safeStore, TxnId txnId) return safeStore.ranges().coordinates(txnId); } - /** - * The ranges for which we participate in the consensus decision of when a transaction executes - */ - private static Ranges preacceptRanges(SafeCommandStore safeStore, TxnId txnId, long untilEpoch) + private static Ranges acceptRanges(SafeCommandStore safeStore, TxnId txnId, long untilEpoch, Ranges coordinateRanges) { - return safeStore.ranges().allBetween(txnId.epoch(), untilEpoch); + return safeStore.ranges().extend(coordinateRanges, txnId.epoch(), untilEpoch); } private static Ranges acceptRanges(SafeCommandStore safeStore, TxnId txnId, Timestamp executeAt, Ranges coordinateRanges) @@ -767,8 +734,8 @@ private static boolean updateWaitingOn(SafeCommandStore safeStore, CommonAttribu case TruncatedApply: case TruncatedApplyWithOutcome: case TruncatedApplyWithDeps: - Invariants.checkState(dependency.executeAt().compareTo(waitingExecuteAt) < 0 || waitingId.kind().awaitsOnlyDeps()); - case ErasedOrInvalidated: + Invariants.checkState(dependency.executeAt().compareTo(waitingExecuteAt) < 0 || waitingId.kind().awaitsOnlyDeps() || !dependency.txnId().kind().witnesses(waitingId)); + case ErasedOrInvalidOrVestigial: case Erased: logger.trace("{}: {} is truncated. Stop listening and removing from waiting on commit set.", waitingId, dependencyId); break; @@ -853,12 +820,12 @@ public static void removeWaitingOnKeyAndMaybeExecute(SafeCommandStore safeStore, } // TODO (now): document and justify all calls - public static void setTruncatedApply(SafeCommandStore safeStore, SafeCommand safeCommand) + public static void setTruncatedApplyOrErasedVestigial(SafeCommandStore safeStore, SafeCommand safeCommand) { - setTruncatedApply(safeStore, safeCommand, null, null); + setTruncatedApplyOrErasedVestigial(safeStore, safeCommand, null, null); } - public static void setTruncatedApply(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable Timestamp executeAt, Route maybeFullRoute) + public static void setTruncatedApplyOrErasedVestigial(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable Timestamp executeAt, Route maybeFullRoute) { Command command = safeCommand.current(); if (command.saveStatus().compareTo(TruncatedApply) >= 0) return; @@ -867,7 +834,9 @@ public static void setTruncatedApply(SafeCommandStore safeStore, SafeCommand saf if (executeAt == null) executeAt = command.executeAtIfKnown(); if (route == null || executeAt == null) { - safeCommand.update(safeStore, erased(command)); + safeCommand.update(safeStore, Command.Truncated.erasedOrInvalidOrVestigial(command)); + if (route != null && !safeStore.ranges().allAt(command.txnId()).contains(route.homeKey())) + safeStore.progressLog().clear(command.txnId()); } else { @@ -882,6 +851,7 @@ else if (safeCommand.current().saveStatus().hasBeen(Applied)) if (executesAtLeast == null) safeCommand.update(safeStore, erased(command)); else safeCommand.update(safeStore, truncatedApply(attributes, TruncatedApply, executeAt, null, null, executesAtLeast)); } + safeStore.progressLog().clear(command.txnId()); } } @@ -924,8 +894,9 @@ public static Command purge(SafeCommandStore safeStore, SafeCommand safeCommand, case TRUNCATE: // TODO (expected): consider passing through any information we have about the reason for loading, so we can infer APPLIED if !PreCommitted Invariants.checkState(command.saveStatus().compareTo(TruncatedApply) < 0); - if (!command.hasBeen(PreCommitted)) result = erased(command); + if (!command.hasBeen(PreCommitted)) result = Command.Truncated.erasedOrInvalidOrVestigial(command); else result = truncatedApply(command, Route.tryCastToFullRoute(maybeFullRoute)); + safeStore.progressLog().clear(command.txnId()); break; case ERASE: @@ -1227,193 +1198,93 @@ private static ProgressShard progressShard(Route route, @Nullable RoutingKey return progressKey.equals(route.homeKey()) ? Home : Local; } - enum EnsureAction - { - /** Don't check */ - Ignore, - /** Add, but return false if insufficient for any reason */ - TryAdd, - /** Supplement existing information, asserting that the existing and additional information are independently sufficient, - * returning false only if the existing information is absent AND the new information is insufficient. */ - Add, - /** Set, but only return false if insufficient */ - TrySet, - /** Overwrite existing information if sufficient; fail otherwise */ - Set - } - @SuppressWarnings({"unchecked", "rawtypes"}) - private static CommonAttributes set(CommonAttributes attrs, - Ranges existingRanges, Ranges additionalRanges, - ProgressShard shard, Route route, - @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn, - @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps) + private static CommonAttributes set(SaveStatus newStatus, Command cur, CommonAttributes upd, + Ranges acceptRanges, @Nullable Ballot newAcceptedOrCommittedBallot, + Route route, @Nullable PartialTxn partialTxn, @Nullable PartialDeps partialDeps) { - Invariants.checkState(shard != Unsure); - Ranges allRanges = existingRanges.with(additionalRanges); - attrs = attrs.mutable().route(Route.merge(attrs.route(), (Route)route)); + Status.Known haveKnown = cur.saveStatus().known; + Status.Known expectKnown = newStatus.known; - // TODO (soon): stop round-robin hashing; partition only on ranges - switch (ensurePartialTxn) - { - case Add: - if (partialTxn == null) - break; - - if (attrs.partialTxn() != null) - { - partialTxn = partialTxn.slice(allRanges, shard.isHome()); - attrs = attrs.mutable().partialTxn(attrs.partialTxn().with(partialTxn)); - break; - } + if (Route.isFullRoute(upd.route())) route = upd.route(); + else upd = upd.mutable().route(route = Route.merge(upd.route(), (Route)route)); - case Set: - case TrySet: - // TODO (desired): only includeQuery if shard.isHome(); this affects state eviction and is low priority given size in C* - attrs = attrs.mutable().partialTxn(partialTxn.slice(allRanges, true)); - break; + Route scope; + if (partialTxn != null && expectKnown.definition.isKnown()) + { + scope = route.slice(acceptRanges); + partialTxn = partialTxn.intersecting(scope, true); + if (haveKnown.definition.isKnown()) + upd = upd.mutable().partialTxn(upd.partialTxn().with(partialTxn)); + else + upd = upd.mutable().partialTxn(partialTxn); } - switch (ensurePartialDeps) + if (partialDeps != null && expectKnown.deps.hasProposedOrDecidedDeps() && (haveKnown.deps != expectKnown.deps || (newAcceptedOrCommittedBallot != null && !newAcceptedOrCommittedBallot.equals(cur.acceptedOrCommitted())))) { - case Add: - if (partialDeps == null) - break; - - if (attrs.partialDeps() != null) - { - attrs = attrs.mutable().partialDeps(attrs.partialDeps().with(partialDeps.slice(allRanges))); - break; - } - - case Set: - case TrySet: - attrs = attrs.mutable().partialDeps(partialDeps.slice(allRanges)); - break; + scope = route.slice(acceptRanges); + upd = upd.mutable().partialDeps(partialDeps.intersecting(scope)); } - return attrs; - } - /** - * Validate we have sufficient information for the route, partialTxn and partialDeps fields, and if so update them; - * otherwise return false (or throw an exception if an illegal state is encountered) - */ - private static boolean validate(Status status, CommonAttributes attrs, - Ranges existingRanges, Ranges additionalRanges, ProgressShard shard, - Route route, EnsureAction ensureRoute, - @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn, - @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps) - { - return validate(status, attrs, existingRanges, additionalRanges, shard, route, ensureRoute, partialTxn, ensurePartialTxn, partialDeps, ensurePartialDeps, null); + return upd; } - private static boolean validate(Status status, CommonAttributes attrs, - Ranges existingRanges, Ranges additionalRanges, ProgressShard shard, - Route route, EnsureAction ensureRoute, - @Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn, - @Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps, + private static boolean validate(SaveStatus newStatus, Command cur, + Ranges acceptRanges, + Route addRoute, + @Nullable PartialTxn addPartialTxn, + @Nullable PartialDeps newPartialDeps, @Nullable SafeCommandStore permitStaleMissing) { - if (shard == Unsure) - return false; + Status.Known haveKnown = cur.saveStatus().known; + Status.Known expectKnown = newStatus.known; + + FullRoute fullRoute; + if (isFullRoute(cur.route())) fullRoute = Route.castToFullRoute(cur.route()); + else if (isFullRoute(addRoute)) fullRoute = Route.castToFullRoute(addRoute); + else return expectKnown.route != Full; // we cannot validate further without a full route, so in cases we don't expect a full route we just assume we have received the right information - // first validate route - switch (ensureRoute) + Participants acceptScope = null; + if (expectKnown.definition.isKnown()) { - default: throw new AssertionError("Unexpected action: " + ensureRoute); - case TryAdd: - case Add: - if (!isFullRoute(attrs.route()) && !isFullRoute(route)) + acceptScope = fullRoute.slice(acceptRanges); + if (haveKnown.definition.isKnown()) + { + // TODO (desired): avoid converting to participants before subtracting + Participants extraScope = acceptScope.subtract(cur.partialTxn().keys().toParticipants()); + if (!containsAll(addPartialTxn, PartialTxn::covers, extraScope, permitStaleMissing)) return false; - case Ignore: - break; - case Set: - if (!isFullRoute(route)) - throw new IllegalArgumentException("Incomplete route (" + route + ") sent to home shard"); - break; - case TrySet: - if (!isFullRoute(route)) + } + else + { + if (!containsAll(addPartialTxn, PartialTxn::covers, acceptScope, permitStaleMissing)) return false; + } } - // invalid to Add deps to Accepted or AcceptedInvalidate statuses, as Committed deps are not equivalent - // and we may erroneously believe we have covered a wider range than we have infact covered - if (ensurePartialDeps == Add) - Invariants.checkState(status != Accepted && status != AcceptedInvalidate); - - // validate new partial txn - if (!validate(ensurePartialTxn, existingRanges, additionalRanges, covers(attrs.partialTxn()), covers(partialTxn), permitStaleMissing, "txn", partialTxn)) - return false; - - Invariants.checkState(partialTxn == null || attrs.txnId().kind().equals(partialTxn.kind()), "Transaction has different kind to its TxnId"); - Invariants.checkState(partialTxn == null || !shard.isHome() || ensurePartialTxn == Ignore || hasQuery(attrs.partialTxn()) || hasQuery(partialTxn), "Home transaction should include query"); - - return validate(ensurePartialDeps, existingRanges, additionalRanges, covers(attrs.partialDeps()), covers(partialDeps), permitStaleMissing, "deps", partialDeps); - } - - // FIXME (immutable-state): has this been removed? - private static boolean validate(EnsureAction action, Ranges existingRanges, Ranges requiredRanges, - Ranges existing, Ranges adding, @Nullable SafeCommandStore permitStaleMissing, - String kind, Object obj) - { - switch (action) + if (haveKnown.deps != expectKnown.deps && expectKnown.deps.hasProposedOrDecidedDeps()) { - default: throw illegalState("Unexpected action: " + action); - case Ignore: - return true; - - case TrySet: - case Set: - if (containsAll(adding, requiredRanges, permitStaleMissing)) - return true; - - if (action == Set) - illegalState("Incomplete " + kind + " (" + obj + ") provided; does not cover " + requiredRanges.subtract(adding)); - - return false; - - case TryAdd: - case Add: - if (existing == null) - { - if (adding == null) - return false; // we don't want to permit a null value for txn/deps, even if we are stale for all participating ranges, as it breaks assumptions elsewhere - - if (!adding.containsAll(existingRanges)) - return false; - - return validate(action == TryAdd ? TrySet : Set, existingRanges, requiredRanges, existing, adding, permitStaleMissing, kind, obj); - } - - Invariants.checkState(existing.containsAll(existingRanges), "Existing ranges insufficient"); - if (requiredRanges == existingRanges) - return true; - - if (adding == null) - return permitStaleMissing != null && containsAll(Ranges.EMPTY, requiredRanges.subtract(existing), permitStaleMissing); - - requiredRanges = requiredRanges.subtract(existing); - if (containsAll(adding, requiredRanges, permitStaleMissing)) - return true; - - if (action == Add) - illegalState("Incomplete " + kind + " (" + obj + ") provided; does not cover " + requiredRanges.subtract(adding)); + if (acceptScope == null) + acceptScope = fullRoute.slice(acceptRanges); - return false; + return containsAll(newPartialDeps, PartialDeps::covers, acceptScope, permitStaleMissing); } + + return true; } - private static boolean containsAll(Ranges adding, Ranges requiredRanges, @Nullable SafeCommandStore permitStaleMissing) + private static boolean containsAll(V adding, BiPredicate> covers, Participants required, @Nullable SafeCommandStore permitStaleMissing) { - if (adding.containsAll(requiredRanges)) + if (adding == null ? required.isEmpty() : covers.test(adding, required)) return true; if (permitStaleMissing != null) { + // TODO (required, later): in the event we are depending on a stale key for an insert into a non-stale key, we cannot proceed and must mark the new key stale + // I think today this is unsupported in practice, but must be addressed before we improve efficiency of result handling Ranges staleRanges = permitStaleMissing.commandStore().redundantBefore().staleRanges(); - requiredRanges = requiredRanges.subtract(staleRanges); - if (adding.containsAll(requiredRanges)) - return true; + required = required.subtract(staleRanges); + return adding == null ? required.isEmpty() : covers.test(adding, required); } return false; diff --git a/accord-core/src/main/java/accord/local/DurableBefore.java b/accord-core/src/main/java/accord/local/DurableBefore.java index 9da07232c..6e485f381 100644 --- a/accord-core/src/main/java/accord/local/DurableBefore.java +++ b/accord-core/src/main/java/accord/local/DurableBefore.java @@ -24,8 +24,8 @@ import accord.api.RoutingKey; import accord.local.Status.Durability; +import accord.primitives.AbstractRanges; import accord.primitives.Participants; -import accord.primitives.Ranges; import accord.primitives.TxnId; import accord.primitives.Unseekables; import accord.utils.Invariants; @@ -147,7 +147,7 @@ private DurableBefore() } } - public static DurableBefore create(Ranges ranges, @Nonnull TxnId majority, @Nonnull TxnId universal) + public static DurableBefore create(AbstractRanges ranges, @Nonnull TxnId majority, @Nonnull TxnId universal) { if (ranges.isEmpty()) return DurableBefore.EMPTY; diff --git a/accord-core/src/main/java/accord/local/MaxConflicts.java b/accord-core/src/main/java/accord/local/MaxConflicts.java index 3ccf3c3f1..c31a1bfad 100644 --- a/accord-core/src/main/java/accord/local/MaxConflicts.java +++ b/accord-core/src/main/java/accord/local/MaxConflicts.java @@ -21,7 +21,7 @@ import javax.annotation.Nonnull; import accord.api.RoutingKey; -import accord.primitives.Ranges; +import accord.primitives.AbstractRanges; import accord.primitives.Routables; import accord.primitives.Seekables; import accord.primitives.Timestamp; @@ -58,7 +58,7 @@ MaxConflicts update(Seekables keysOrRanges, Timestamp maxConflict) return merge(this, create(keysOrRanges, maxConflict)); } - public static MaxConflicts create(Ranges ranges, @Nonnull Timestamp maxConflict) + public static MaxConflicts create(AbstractRanges ranges, @Nonnull Timestamp maxConflict) { if (ranges.isEmpty()) return MaxConflicts.EMPTY; diff --git a/accord-core/src/main/java/accord/local/Node.java b/accord-core/src/main/java/accord/local/Node.java index 74446cee9..80f205f3e 100644 --- a/accord-core/src/main/java/accord/local/Node.java +++ b/accord-core/src/main/java/accord/local/Node.java @@ -71,7 +71,6 @@ import accord.primitives.EpochSupplier; import accord.primitives.FullRoute; import accord.primitives.ProgressToken; -import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.Routable.Domain; import accord.primitives.Routables; @@ -145,6 +144,7 @@ public String toString() public boolean isCoordinating(TxnId txnId, Ballot promised) { + // TODO (required): on a prod system expire coordination ownership by time for safety return promised.node.equals(id) && coordinating.containsKey(txnId); } @@ -622,18 +622,18 @@ public FullRoute computeRoute(TxnId txnId, Seekables keysOrRanges) public FullRoute computeRoute(long epoch, Seekables keysOrRanges) { Invariants.checkArgument(!keysOrRanges.isEmpty(), "Attempted to compute a route from empty keys or ranges"); - RoutingKey homeKey = trySelectHomeKey(epoch, keysOrRanges); - if (homeKey == null) - homeKey = selectRandomHomeKey(epoch); - + RoutingKey homeKey = selectHomeKey(epoch, keysOrRanges); return keysOrRanges.toRoute(homeKey); } - private @Nullable RoutingKey trySelectHomeKey(long epoch, Seekables keysOrRanges) + private RoutingKey selectHomeKey(long epoch, Seekables keysOrRanges) { Ranges owned = topology().localForEpoch(epoch).ranges(); int i = (int)keysOrRanges.findNextIntersection(0, owned, 0); - return i >= 0 ? keysOrRanges.get(i).someIntersectingRoutingKey(owned) : null; + if (i >= 0) + return keysOrRanges.get(i).someIntersectingRoutingKey(owned); + + return keysOrRanges.get(random.nextInt(keysOrRanges.size())).someIntersectingRoutingKey(null); } public RoutingKey selectProgressKey(TxnId txnId, Route route, RoutingKey homeKey) @@ -675,23 +675,6 @@ private static RoutingKey trySelectProgressKey(Topology topology, Route route return route.get(i).someIntersectingRoutingKey(topology.ranges()); } - public RoutingKey selectRandomHomeKey(TxnId txnId) - { - return selectRandomHomeKey(txnId.epoch()); - } - - public RoutingKey selectRandomHomeKey(long epoch) - { - Ranges ranges = topology().localForEpoch(epoch).ranges(); - // TODO (expected): should we try to pick keys in the same Keyspace in C*? Might want to adapt this to an Agent behaviour - if (ranges.isEmpty()) // should not really happen, but pick some other replica to serve as home key - ranges = topology().globalForEpoch(epoch).ranges(); - if (ranges.isEmpty()) - throw illegalState("Unable to select a HomeKey as the topology does not have any ranges for epoch " + epoch); - Range range = ranges.get(random.nextInt(ranges.size())); - return range.someIntersectingRoutingKey(null); - } - static class RecoverFuture extends AsyncResults.SettableResult implements BiConsumer { @Override diff --git a/accord-core/src/main/java/accord/local/SafeCommandStore.java b/accord-core/src/main/java/accord/local/SafeCommandStore.java index 0cea6eb3e..6d06959c7 100644 --- a/accord-core/src/main/java/accord/local/SafeCommandStore.java +++ b/accord-core/src/main/java/accord/local/SafeCommandStore.java @@ -46,7 +46,7 @@ import static accord.local.KeyHistory.COMMANDS; import static accord.local.RedundantBefore.PreBootstrapOrStale.FULLY; import static accord.local.SaveStatus.Erased; -import static accord.local.SaveStatus.ErasedOrInvalidated; +import static accord.local.SaveStatus.ErasedOrInvalidOrVestigial; import static accord.primitives.Route.isFullRoute; /** @@ -94,7 +94,7 @@ public SafeCommand get(TxnId txnId, RoutingKey unseekable) if (command.saveStatus().isUninitialised()) { if (commandStore().durableBefore().isUniversal(txnId, unseekable)) - return new ErasedSafeCommand(txnId, ErasedOrInvalidated); + return new ErasedSafeCommand(txnId, ErasedOrInvalidOrVestigial); } return maybeTruncate(safeCommand, command, txnId, null); } @@ -126,7 +126,7 @@ public SafeCommand get(TxnId txnId, EpochSupplier toEpoch, Unseekables unseek if (command.saveStatus().isUninitialised()) { if (Cleanup.isSafeToCleanup(commandStore().durableBefore(), txnId, unseekables)) - return new ErasedSafeCommand(txnId, isFullRoute(unseekables) ? Erased : ErasedOrInvalidated); + return new ErasedSafeCommand(txnId, isFullRoute(unseekables) ? Erased : ErasedOrInvalidOrVestigial); } return maybeTruncate(safeCommand, command, toEpoch, unseekables); } diff --git a/accord-core/src/main/java/accord/local/SaveStatus.java b/accord-core/src/main/java/accord/local/SaveStatus.java index f0b959405..5dfa13fb2 100644 --- a/accord-core/src/main/java/accord/local/SaveStatus.java +++ b/accord-core/src/main/java/accord/local/SaveStatus.java @@ -79,10 +79,10 @@ public enum SaveStatus TruncatedApplyWithDeps (Status.Truncated, Full, DefinitionErased, ExecuteAtKnown, DepsKnown, Outcome.Apply, CleaningUp), TruncatedApplyWithOutcome (Status.Truncated, Full, DefinitionErased, ExecuteAtKnown, DepsErased, Outcome.Apply, CleaningUp), TruncatedApply (Status.Truncated, Full, DefinitionErased, ExecuteAtKnown, DepsErased, Outcome.WasApply, CleaningUp), - // ErasedOrInvalidated means the command is redundant for the shard and data being queried, but no FullRoute is known, so it is not known to be globally Erased - ErasedOrInvalidated (Status.Truncated, Maybe, DefinitionUnknown, ExecuteAtUnknown, DepsUnknown, Unknown, CleaningUp), - // NOTE: Erased should ONLY be adopted on a replica that knows EVERY shard has successfully applied the transaction at all healthy replicas. + // NOTE: Erased should ONLY be adopted on a replica that knows EVERY shard has successfully applied the transaction at all healthy replicas (or else it is durably invalidated) Erased (Status.Truncated, Maybe, DefinitionErased, ExecuteAtErased, DepsErased, Outcome.Erased, CleaningUp), + // ErasedOrInvalidOrVestigial means the command cannot be completed and is either pre-bootstrap, did not commit, or did not participate in this shard's epoch + ErasedOrInvalidOrVestigial (Status.Truncated, Maybe, DefinitionUnknown, ExecuteAtUnknown, DepsUnknown, Unknown, CleaningUp), Invalidated (Status.Invalidated, CleaningUp), ; @@ -267,12 +267,12 @@ public static SaveStatus enrich(SaveStatus status, Known known) switch (status) { default: throw new AssertionError("Unexpected status: " + status); - case ErasedOrInvalidated: + case ErasedOrInvalidOrVestigial: if (known.outcome.isInvalidated()) return Invalidated; if (!known.outcome.isOrWasApply() || known.executeAt == ExecuteAtKnown) - return ErasedOrInvalidated; + return ErasedOrInvalidOrVestigial; case Erased: if (!known.outcome.isOrWasApply() || known.executeAt != ExecuteAtKnown) diff --git a/accord-core/src/main/java/accord/local/Status.java b/accord-core/src/main/java/accord/local/Status.java index c80918fa8..ccb5add46 100644 --- a/accord-core/src/main/java/accord/local/Status.java +++ b/accord-core/src/main/java/accord/local/Status.java @@ -78,7 +78,7 @@ public enum Status */ PreCommitted (Accept, Maybe, DefinitionUnknown, ExecuteAtKnown, DepsUnknown, Unknown), - Committed (Commit, Full, DefinitionKnown, ExecuteAtKnown, DepsCommitted, Unknown), + Committed (Commit, Full, DefinitionKnown, ExecuteAtKnown, DepsCommitted,Unknown), Stable (Execute, Full, DefinitionKnown, ExecuteAtKnown, DepsKnown, Unknown), PreApplied (Persist, Full, DefinitionKnown, ExecuteAtKnown, DepsKnown, Outcome.Apply), Applied (Persist, Full, DefinitionKnown, ExecuteAtKnown, DepsKnown, Outcome.Apply), @@ -886,6 +886,7 @@ public static Durability merge(Durability a, Durability b) if (c < 0) { Durability tmp = a; a = b; b = tmp; } // if we know we are applied, we can remove the OrInvalidated qualifier if (a == UniversalOrInvalidated && (b == Majority || b == ShardUniversal || b == Local)) a = Universal; + // TODO (required, minor cleanup): should ShardUniversal+NotDurable=Local? It might be that we are stale. if ((a == ShardUniversal) && (b == Local || b == NotDurable)) a = Local; if (b == NotDurable && a.compareTo(MajorityOrInvalidated) < 0) a = NotDurable; return a; diff --git a/accord-core/src/main/java/accord/local/cfk/CommandsForKey.java b/accord-core/src/main/java/accord/local/cfk/CommandsForKey.java index 162b4fe89..5af043057 100644 --- a/accord-core/src/main/java/accord/local/cfk/CommandsForKey.java +++ b/accord-core/src/main/java/accord/local/cfk/CommandsForKey.java @@ -100,7 +100,7 @@ * *

Dependency Encoding

* The byId list implies the contents of the deps of all commands in the collection - that is, it is assumed that in - * the normal course of events every transaction will include the full set of {@code TxnId} we know that would be + * the normal course of events every transaction will include the full set of {@code TxnId} we know that could be * witnessed by the command. We only encode divergences from this, stored in each command's {@code missing} collection. * * We then go one step further, exploiting the fact that the missing collection exists solely to implement recovery, @@ -125,7 +125,7 @@ * applied or invalidated. * * [We only do this if there also exists some later transactions we are not pruning that collectively have a superset of - * its witnessed collection, so that recovery decisions will be unaffected by the removal of the transaction.] + * its {@code missing} collection, so that recovery decisions will be unaffected by the removal of the transaction.] * * The complexity here is that, by virtue of being a local decision point, we cannot guarantee that no coordinator will * contact us in future with either a new TxnId that is lower than this, or a dependency collection containing a TxnId @@ -166,6 +166,7 @@ * TODO (expected): minimise repeated notification, either by logic or marking a command as notified once ready-to-execute * TODO (required): linearizability violation detection * TODO (desired): introduce a new status or other fast and simple mechanism for filtering treatment of range or unmanaged transactions + * TODO (expected): use locallyAppliedOrInvalidatedBefore to advance minUndecided and as a lower bound for triggering execution */ public class CommandsForKey extends CommandsForKeyUpdate implements CommandsSummary { @@ -521,7 +522,7 @@ public enum InternalStatus convert.put(SaveStatus.TruncatedApplyWithDeps, INVALID_OR_TRUNCATED_OR_UNMANAGED_COMMITTED); convert.put(SaveStatus.TruncatedApplyWithOutcome, INVALID_OR_TRUNCATED_OR_UNMANAGED_COMMITTED); convert.put(SaveStatus.TruncatedApply, INVALID_OR_TRUNCATED_OR_UNMANAGED_COMMITTED); - convert.put(SaveStatus.ErasedOrInvalidated, INVALID_OR_TRUNCATED_OR_UNMANAGED_COMMITTED); + convert.put(SaveStatus.ErasedOrInvalidOrVestigial, INVALID_OR_TRUNCATED_OR_UNMANAGED_COMMITTED); convert.put(SaveStatus.Erased, INVALID_OR_TRUNCATED_OR_UNMANAGED_COMMITTED); convert.put(SaveStatus.Invalidated, INVALID_OR_TRUNCATED_OR_UNMANAGED_COMMITTED); } @@ -786,6 +787,14 @@ public TxnId shardRedundantBefore() return redundantBefore.shardRedundantBefore(); } + public TxnId nextWaitingToApply(Kinds kinds) + { + int i = maxAppliedWriteByExecuteAt + 1; + while (i < committedByExecuteAt.length && (committedByExecuteAt[i].status != APPLIED || !kinds.test(committedByExecuteAt[i].kind()))) + ++i; + return i >= committedByExecuteAt.length ? null : committedByExecuteAt[i]; + } + /** * All commands before/after (exclusive of) the given timestamp, excluding those that are redundant, * or have locally applied prior to some other command that is stable, will be returned by the collection. diff --git a/accord-core/src/main/java/accord/local/cfk/Pruning.java b/accord-core/src/main/java/accord/local/cfk/Pruning.java index 86d3f0e1d..f5e91979b 100644 --- a/accord-core/src/main/java/accord/local/cfk/Pruning.java +++ b/accord-core/src/main/java/accord/local/cfk/Pruning.java @@ -35,7 +35,7 @@ import static accord.local.cfk.CommandsForKey.NO_TXNIDS; import static accord.local.cfk.CommandsForKey.managesExecution; import static accord.local.cfk.Pruning.LoadingPruned.LOADINGF; -import static accord.utils.ArrayBuffers.cachedTimestamps; +import static accord.utils.ArrayBuffers.cachedAny; import static accord.utils.ArrayBuffers.cachedTxnIds; public class Pruning @@ -203,7 +203,7 @@ static CommandsForKey pruneBefore(CommandsForKey cfk, TxnInfo newPrunedBefore, i TxnInfo[] byId = cfk.byId; int minUndecidedById = -1; int retainCount = 0, removedCommittedCount = 0; - Timestamp[] removedExecuteAts = NO_TXNIDS; + Object[] removedExecuteAts = NO_TXNIDS; int removedExecuteAtCount = 0; TxnInfo[] newInfos; { @@ -239,7 +239,7 @@ static CommandsForKey pruneBefore(CommandsForKey cfk, TxnInfo newPrunedBefore, i if (missing != NO_TXNIDS) { if (removedExecuteAtCount == removedExecuteAts.length) - removedExecuteAts = cachedTimestamps().resize(removedExecuteAts, removedExecuteAtCount, Math.max(8, removedExecuteAtCount + (removedExecuteAtCount >> 1))); + removedExecuteAts = cachedAny().resize(removedExecuteAts, removedExecuteAtCount, Math.max(8, removedExecuteAtCount + (removedExecuteAtCount >> 1))); removedExecuteAts[removedExecuteAtCount++] = txn.executeAt; } ++removedCommittedCount; @@ -325,7 +325,7 @@ static CommandsForKey pruneBefore(CommandsForKey cfk, TxnInfo newPrunedBefore, i System.arraycopy(committedByExecuteAt, sourcePos, newCommittedByExecuteAt, sourcePos - removedCommittedCount, committedByExecuteAt.length - sourcePos); } - cachedTimestamps().forceDiscard(removedExecuteAts, removedExecuteAtCount); + cachedAny().forceDiscard(removedExecuteAts, removedExecuteAtCount); int newMaxAppliedWriteByExecuteAt = cfk.maxAppliedWriteByExecuteAt - removedCommittedCount; return new CommandsForKey(cfk.key, cfk.redundantBefore, newPrunedBefore, cfk.loadingPruned, newInfos, newCommittedByExecuteAt, minUndecidedById, newMaxAppliedWriteByExecuteAt, cfk.unmanageds); } diff --git a/accord-core/src/main/java/accord/messages/Accept.java b/accord-core/src/main/java/accord/messages/Accept.java index 1bbc87808..d1c469b01 100644 --- a/accord-core/src/main/java/accord/messages/Accept.java +++ b/accord-core/src/main/java/accord/messages/Accept.java @@ -67,8 +67,8 @@ public Accept(Id to, Topologies topologies, Ballot ballot, TxnId txnId, FullRout super(to, topologies, txnId, route); this.ballot = ballot; this.executeAt = executeAt; - this.keys = keys.slice(scope.covering()); - this.partialDeps = deps.slice(scope.covering()); + this.keys = keys.intersecting(scope); + this.partialDeps = deps.intersecting(scope); } private Accept(TxnId txnId, PartialRoute scope, long waitForEpoch, long minEpoch, boolean doNotComputeProgressKey, Ballot ballot, Timestamp executeAt, Seekables keys, PartialDeps partialDeps) @@ -113,7 +113,7 @@ public AcceptReply apply(SafeCommandStore safeStore) private PartialDeps calculatePartialDeps(SafeCommandStore safeStore) { Ranges ranges = safeStore.ranges().allBetween(minUnsyncedEpoch, executeAt); - return PreAccept.calculatePartialDeps(safeStore, txnId, keys, EpochSupplier.constant(minUnsyncedEpoch), executeAt, ranges); + return PreAccept.calculatePartialDeps(safeStore, txnId, keys, scope, EpochSupplier.constant(minUnsyncedEpoch), executeAt, ranges); } @Override diff --git a/accord-core/src/main/java/accord/messages/Apply.java b/accord-core/src/main/java/accord/messages/Apply.java index 96d57f11d..07b7d8168 100644 --- a/accord-core/src/main/java/accord/messages/Apply.java +++ b/accord-core/src/main/java/accord/messages/Apply.java @@ -78,9 +78,9 @@ protected Apply(Kind kind, Id to, Topologies participates, TxnId txnId, FullRout // TODO (desired): it's wasteful to encode the full set of ranges owned by the recipient node; // often it will be cheaper to include the FullRoute for Deps scope (or come up with some other safety-preserving encoding scheme) this.kind = kind; - this.deps = deps.slice(scope.covering()); - this.keys = txn.keys().slice(scope.covering()); - this.txn = kind == Kind.Maximal ? txn.slice(scope.covering(), true) : null; + this.deps = deps.intersecting(scope); + this.keys = txn.keys().intersecting(scope); + this.txn = kind == Kind.Maximal ? txn.intersecting(scope, true) : null; this.fullRoute = kind == Kind.Maximal ? route : null; this.executeAt = executeAt; this.writes = writes; diff --git a/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java b/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java index 58dfeff46..9584eb763 100644 --- a/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java +++ b/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java @@ -37,6 +37,7 @@ import accord.primitives.PartialTxn; import accord.primitives.Participants; import accord.primitives.Ranges; +import accord.primitives.Route; import accord.primitives.Seekables; import accord.primitives.Timestamp; import accord.primitives.Txn; @@ -78,13 +79,13 @@ public ApplyThenWaitUntilApplied(Node.Id to, Topologies topologies, Timestamp ex { super(to, topologies, txnId, readScope, executeAt.epoch()); this.executeAt = executeAt; - Ranges slice = computeScope(to, topologies, null, 0, (i,r)->r, Ranges::with); + Route scope = computeScope(to, topologies, route); this.route = route; - this.txn = txn.slice(slice, true); - this.deps = deps.slice(slice); + this.txn = txn.intersecting(scope, true); + this.deps = deps.intersecting(scope); this.writes = writes; this.result = result; - this.notify = notify == null ? null : notify.slice(slice); + this.notify = notify == null ? null : notify.intersecting(scope); } protected ApplyThenWaitUntilApplied(TxnId txnId, Participants readScope, Timestamp executeAt, FullRoute route, PartialTxn txn, PartialDeps deps, Writes writes, Result result, Seekables notify) diff --git a/accord-core/src/main/java/accord/messages/BeginInvalidation.java b/accord-core/src/main/java/accord/messages/BeginInvalidation.java index e0c1b4896..0a59bb354 100644 --- a/accord-core/src/main/java/accord/messages/BeginInvalidation.java +++ b/accord-core/src/main/java/accord/messages/BeginInvalidation.java @@ -22,9 +22,12 @@ import accord.local.*; import accord.local.Node.Id; import accord.primitives.*; +import accord.topology.Shard; import accord.topology.Topologies; +import accord.utils.SortedList; import javax.annotation.Nullable; +import java.util.Arrays; import java.util.List; import java.util.Objects; @@ -167,24 +170,34 @@ public MessageType type() return MessageType.BEGIN_INVALIDATE_RSP; } - public static FullRoute findRoute(List invalidateOks) + public static FullRoute findRoute(InvalidateReply[] invalidateOks) { for (InvalidateReply ok : invalidateOks) { - if (isFullRoute(ok.route)) + if (ok != null && isFullRoute(ok.route)) return castToFullRoute(ok.route); } return null; } - public static Route mergeRoutes(List invalidateOks) + public static Route mergeRoutes(InvalidateReply[] invalidateOks) { return mapReduceNonNull(ok -> (Route)ok.route, Route::union, invalidateOks); } - public static InvalidateReply max(List invalidateReplies) + public static InvalidateReply max(InvalidateReply[] invalidateReplies, Shard shard, SortedList nodeIds) { - return Status.max(invalidateReplies, r -> r.status, r -> r.accepted, invalidateReply -> true); + return Status.max(nodeIds.select(invalidateReplies, shard.nodes), r -> r.status, r -> r.accepted, Objects::nonNull); + } + + public static InvalidateReply max(InvalidateReply[] invalidateReplies) + { + return Status.max(Arrays.asList(invalidateReplies),r -> r.status, r -> r.accepted, Objects::nonNull); + } + + public static InvalidateReply maxNotTruncated(InvalidateReply[] invalidateReplies) + { + return Status.max(Arrays.asList(invalidateReplies),r -> r.status, r -> r.accepted, r -> r != null && r.status != Status.Truncated); } public static RoutingKey findHomeKey(List invalidateOks) diff --git a/accord-core/src/main/java/accord/messages/BeginRecovery.java b/accord-core/src/main/java/accord/messages/BeginRecovery.java index f04e0ac9d..29b3bd0ca 100644 --- a/accord-core/src/main/java/accord/messages/BeginRecovery.java +++ b/accord-core/src/main/java/accord/messages/BeginRecovery.java @@ -70,7 +70,7 @@ public BeginRecovery(Id to, Topologies topologies, TxnId txnId, Txn txn, FullRou { super(to, topologies, route, txnId); // TODO (expected): only scope.contains(route.homeKey); this affects state eviction and is low priority given size in C* - this.partialTxn = txn.slice(scope.covering(), true); + this.partialTxn = txn.intersecting(scope, true); this.ballot = ballot; this.route = route; } @@ -115,7 +115,7 @@ public RecoverReply apply(SafeCommandStore safeStore) if (!command.known().deps.hasCommittedOrDecidedDeps()) { // TODO (required): consider whether we are safe ignoring the concept of minUnsyncedEpoch here - localDeps = calculatePartialDeps(safeStore, txnId, partialTxn.keys(), txnId, txnId, safeStore.ranges().coordinates(txnId)); + localDeps = calculatePartialDeps(safeStore, txnId, partialTxn.keys(), scope, txnId, txnId, safeStore.ranges().coordinates(txnId)); } LatestDeps deps = LatestDeps.create(safeStore.ranges().allAt(txnId.epoch()), command.known().deps, command.acceptedOrCommitted(), coordinatedDeps, localDeps); @@ -292,9 +292,14 @@ String toString(String kind) '}'; } - public static RecoverOk maxAcceptedOrLater(List recoverOks) + public static RecoverOk maxAccepted(List recoverOks) { - return Status.max(recoverOks, r -> r.status, r -> r.accepted, r -> r.status.phase.compareTo(Phase.Accept) >= 0); + return Status.max(recoverOks, r -> r.status, r -> r.accepted, r -> r != null && r.status.phase.compareTo(Phase.Accept) >= 0); + } + + public static RecoverOk maxAcceptedNotTruncated(List recoverOks) + { + return Status.max(recoverOks, r -> r.status, r -> r.accepted, r -> r != null && r.status.phase.compareTo(Phase.Accept) >= 0 && r.status.phase.compareTo(Phase.Cleanup) < 0); } } diff --git a/accord-core/src/main/java/accord/messages/CheckStatus.java b/accord-core/src/main/java/accord/messages/CheckStatus.java index a6cf86cfd..bf2798842 100644 --- a/accord-core/src/main/java/accord/messages/CheckStatus.java +++ b/accord-core/src/main/java/accord/messages/CheckStatus.java @@ -57,6 +57,7 @@ import javax.annotation.Nonnull; +import static accord.coordinate.Infer.InvalidIfNot.IsNotInvalid; import static accord.coordinate.Infer.InvalidIfNot.NotKnownToBeInvalid; import static accord.coordinate.Infer.IsPreempted.NotPreempted; import static accord.coordinate.Infer.IsPreempted.Preempted; @@ -72,6 +73,7 @@ import static accord.local.Status.Truncated; import static accord.messages.CheckStatus.WithQuorum.HasQuorum; import static accord.messages.TxnRequest.computeScope; +import static accord.primitives.Routables.Slice.Minimal; import static accord.primitives.Route.castToRoute; import static accord.primitives.Route.isRoute; @@ -157,13 +159,12 @@ public CheckStatusReply apply(SafeCommandStore safeStore) if (!command.has(Known.DefinitionOnly) && Route.isRoute(query) && safeStore.ranges().allAt(txnId.epoch()).contains(Route.castToRoute(query).homeKey())) Commands.informHome(safeStore, safeCommand, Route.castToRoute(query)); - InvalidIfNot invalidIfNotAtLeast = invalidIfNot(safeStore, command, command.route()); boolean isCoordinating = isCoordinating(node, command); Durability durability = command.durability(); Route route = command.route(); if (Route.isFullRoute(route)) durability = Durability.mergeAtLeast(durability, safeStore.commandStore().durableBefore().min(txnId, route)); - Ranges ranges = safeStore.ranges().allBetween(command.txnId().epoch(), command.executeAtIfKnownOrTxnId().epoch()); + FoundKnownMap map = foundKnown(safeStore, command, route); switch (includeInfo) { @@ -171,12 +172,24 @@ public CheckStatusReply apply(SafeCommandStore safeStore) case No: case Route: Route respondWithRoute = includeInfo == IncludeInfo.No ? null : route; - return new CheckStatusOk(ranges, isCoordinating, invalidIfNotAtLeast, respondWithRoute, durability, command); + return new CheckStatusOk(map, isCoordinating, durability, respondWithRoute, command); case All: - return new CheckStatusOkFull(ranges, isCoordinating, invalidIfNotAtLeast, durability, command); + return new CheckStatusOkFull(map, isCoordinating, durability, command); } } + private FoundKnownMap foundKnown(SafeCommandStore safeStore, Command command, @Nullable Route route) + { + Unseekables max = Route.isFullRoute(route) ? route : route != null ? Unseekables.merge((Unseekables)query, route.withHomeKey()) : query; + Unseekables local = max.slice(safeStore.ranges().allAt(command.txnId()), Minimal); + FoundKnown known = new FoundKnown(command.saveStatus().known, NotKnownToBeInvalid, Timestamp.max(command.promised(), command.acceptedOrCommitted()).equals(Ballot.ZERO) ? NotPreempted : Preempted); + + if (command.known().isDecidedToExecute()) + return FoundKnownMap.create(local, known.withAtLeast(IsNotInvalid)); + + return Infer.withInvalidIfNot(safeStore, command.txnId(), local, max, known); + } + private static boolean isCoordinating(Node node, Command command) { return node.isCoordinating(command.txnId(), command.promised()); @@ -202,16 +215,6 @@ public void accept(CheckStatusReply ok, Throwable failure) else node.reply(replyTo, replyContext, ok, null); } - private InvalidIfNot invalidIfNot(SafeCommandStore safeStore, Command command, @Nullable Route route) - { - if (command.known().isDecidedToExecute()) - return NotKnownToBeInvalid; - InvalidIfNot invalidIfNot = Infer.invalidIfNot(safeStore, txnId, query); - if (route != null) - invalidIfNot = invalidIfNot.atLeast(Infer.invalidIfNot(safeStore, txnId, route)); - return invalidIfNot; - } - public interface CheckStatusReply extends Reply { boolean isOk(); @@ -252,6 +255,14 @@ public FoundKnown reduce(FoundKnown with) return new FoundKnown(known, invalidIfNot.reduce(with.invalidIfNot), isPreempted.validForBoth(with.isPreempted)); } + public FoundKnown withAtLeast(InvalidIfNot invalidIfNot) + { + invalidIfNot = this.invalidIfNot.atLeast(invalidIfNot); + if (this.invalidIfNot == invalidIfNot) + return this; + return new FoundKnown(this, invalidIfNot, isPreempted); + } + public FoundKnown validForAll() { Known known = super.validForAll(); @@ -335,6 +346,14 @@ public static FoundKnownMap create(Unseekables keysOrRanges, SaveStatus saveS return create(keysOrRanges, known, Builder::new); } + public static FoundKnownMap create(Unseekables keysOrRanges, FoundKnown known) + { + if (keysOrRanges.isEmpty()) + return new FoundKnownMap(); + + return create(keysOrRanges, known, Builder::new); + } + public static FoundKnownMap merge(FoundKnownMap a, FoundKnownMap b) { return ReducingRangeMap.merge(a, b, FoundKnown::atLeast, Builder::new); @@ -465,9 +484,9 @@ public Ranges knownFor(Known required, Ranges expect) }, Ranges.EMPTY, i -> false); } - static class Builder extends AbstractBoundariesBuilder + public static class Builder extends AbstractBoundariesBuilder { - protected Builder(boolean inclusiveEnds, int capacity) + public Builder(boolean inclusiveEnds, int capacity) { super(inclusiveEnds, capacity); } @@ -504,23 +523,23 @@ public static class CheckStatusOk implements CheckStatusReply public final @Nullable Route route; public final @Nullable RoutingKey homeKey; - public CheckStatusOk(Ranges ranges, boolean isCoordinating, InvalidIfNot invalidIfNot, Durability durability, Command command) + public CheckStatusOk(FoundKnownMap map, boolean isCoordinating, Durability durability, Command command) { - this(ranges, isCoordinating, invalidIfNot, command.route(), durability, command); + this(map, isCoordinating, durability, command.route(), command); } - public CheckStatusOk(Ranges ranges, boolean isCoordinating, InvalidIfNot invalidIfNot, Route route, Durability durability, Command command) + public CheckStatusOk(FoundKnownMap map, boolean isCoordinating, Durability durability, Route route, Command command) { - this(ranges, invalidIfNot, command.saveStatus(), command.promised(), command.acceptedOrCommitted(), command.acceptedOrCommitted(), + this(map, command.saveStatus(), command.promised(), command.acceptedOrCommitted(), command.acceptedOrCommitted(), command.executeAt(), isCoordinating, durability, route, command.homeKey()); } - private CheckStatusOk(Ranges ranges, InvalidIfNot invalidIfNot, SaveStatus saveStatus, Ballot maxPromised, + private CheckStatusOk(FoundKnownMap map, SaveStatus saveStatus, Ballot maxPromised, Ballot maxAcceptedOrCommitted, Ballot acceptedOrCommitted, @Nullable Timestamp executeAt, boolean isCoordinating, Durability durability, @Nullable Route route, @Nullable RoutingKey homeKey) { - this(FoundKnownMap.create(ranges, saveStatus, invalidIfNot, maxPromised), saveStatus, saveStatus, maxPromised, maxAcceptedOrCommitted, acceptedOrCommitted, executeAt, isCoordinating, durability, route, homeKey); + this(map, saveStatus, saveStatus, maxPromised, maxAcceptedOrCommitted, acceptedOrCommitted, executeAt, isCoordinating, durability, route, homeKey); } private CheckStatusOk(FoundKnownMap map, SaveStatus maxKnowledgeSaveStatus, SaveStatus maxSaveStatus, Ballot maxPromised, Ballot maxAcceptedOrCommitted, Ballot acceptedOrCommitted, @@ -620,7 +639,7 @@ CheckStatusOk with(@Nonnull FoundKnownMap newMap) } // TODO (required): harden markShardStale against unnecessary actions by utilising inferInvalidated==MAYBE and performing a global query - public Known knownFor(Routables participants) + public Known knownFor(Unseekables participants) { Known known = map.knownFor(participants); Invariants.checkState(!known.hasFullRoute() || Route.isFullRoute(route)); @@ -736,9 +755,9 @@ public static class CheckStatusOkFull extends CheckStatusOk public final Writes writes; public final Result result; - public CheckStatusOkFull(Ranges ranges, boolean isCoordinating, InvalidIfNot invalidIfNot, Durability durability, Command command) + public CheckStatusOkFull(FoundKnownMap map, boolean isCoordinating, Durability durability, Command command) { - super(ranges, isCoordinating, invalidIfNot, durability, command); + super(map, isCoordinating, durability, command); this.partialTxn = command.partialTxn(); this.stableDeps = command.status().compareTo(Stable) >= 0 ? command.partialDeps() : null; this.writes = command.writes(); @@ -757,9 +776,9 @@ protected CheckStatusOkFull(FoundKnownMap map, SaveStatus maxNotTruncatedSaveSta this.result = result; } - public CheckStatusOkFull finish(Route route, WithQuorum withQuorum) + public CheckStatusOkFull finish(Unseekables unseekables, WithQuorum withQuorum) { - return (CheckStatusOkFull) super.finish(route, withQuorum); + return (CheckStatusOkFull) super.finish(unseekables, withQuorum); } public CheckStatusOkFull merge(@Nonnull Route route) @@ -843,11 +862,11 @@ public CheckStatusOk merge(CheckStatusOk that) * nor any knowledge that does not transfer (i.e. Definition or Deps). */ @Override - public Known knownFor(Routables participants) + public Known knownFor(Unseekables participants) { Known known = super.knownFor(participants); - Invariants.checkState(!known.hasDefinition() || (partialTxn != null && partialTxn.covering().containsAll(participants))); - Invariants.checkState(!known.hasDecidedDeps() || (stableDeps != null && stableDeps.covering.containsAll(participants))); + Invariants.checkState(!known.hasDefinition() || (partialTxn != null && partialTxn.covers(participants))); + Invariants.checkState(!known.hasDecidedDeps() || (stableDeps != null && stableDeps.covers(participants))); return known; } diff --git a/accord-core/src/main/java/accord/messages/Commit.java b/accord-core/src/main/java/accord/messages/Commit.java index dd7a392bc..8bc2bfd5b 100644 --- a/accord-core/src/main/java/accord/messages/Commit.java +++ b/accord-core/src/main/java/accord/messages/Commit.java @@ -125,7 +125,7 @@ public Commit(Kind kind, Id to, Topology coordinateTopology, Topologies topologi if (kind.withTxn == HasTxn) { // TODO (desired): only includeQuery if isHome; this affects state eviction and is low priority given size in C* - partialTxn = txn.slice(scope.covering(), true); + partialTxn = txn.intersecting(scope, true); sendRoute = route; } else if (kind.withTxn == HasNewlyOwnedTxnRanges && executeAt.epoch() != txnId.epoch()) @@ -134,14 +134,14 @@ else if (kind.withTxn == HasNewlyOwnedTxnRanges && executeAt.epoch() != txnId.ep Ranges executeRanges = topologies.computeRangesForNode(to); Ranges extraRanges = executeRanges.subtract(coordinateRanges); if (!extraRanges.isEmpty()) - partialTxn = txn.slice(scope.covering().subtract(coordinateRanges), coordinateRanges.contains(route.homeKey())); + partialTxn = txn.intersecting(scope.subtract(coordinateRanges), coordinateRanges.contains(route.homeKey())); } this.kind = kind; this.executeAt = executeAt; - this.keys = txn.keys().slice(scope.covering()); + this.keys = txn.keys().intersecting(scope); this.partialTxn = partialTxn; - this.partialDeps = deps.slice(scope.covering()); + this.partialDeps = deps.intersecting(scope); this.route = sendRoute; this.readData = toExecuteFactory == null ? null : toExecuteFactory.apply(partialTxn != null ? partialTxn : txn, scope, partialDeps); } diff --git a/accord-core/src/main/java/accord/messages/GetDeps.java b/accord-core/src/main/java/accord/messages/GetDeps.java index 1e77c2464..cd895f9f7 100644 --- a/accord-core/src/main/java/accord/messages/GetDeps.java +++ b/accord-core/src/main/java/accord/messages/GetDeps.java @@ -52,7 +52,7 @@ public static GetDeps create(TxnId txnId, PartialRoute scope, long waitForEpo public GetDeps(Id to, Topologies topologies, FullRoute route, TxnId txnId, Seekables keys, Timestamp executeAt) { super(to, topologies, txnId, route); - this.keys = keys.slice(scope.covering()); + this.keys = keys.intersecting(scope); this.executeAt = executeAt; } @@ -73,7 +73,7 @@ public void process() public PartialDeps apply(SafeCommandStore instance) { Ranges ranges = instance.ranges().allBetween(minUnsyncedEpoch, executeAt); - return calculatePartialDeps(instance, txnId, keys, constant(minUnsyncedEpoch), executeAt, ranges); + return calculatePartialDeps(instance, txnId, keys, scope, constant(minUnsyncedEpoch), executeAt, ranges); } @Override diff --git a/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java b/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java index 3943d5dd4..f086abba6 100644 --- a/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java +++ b/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java @@ -52,7 +52,7 @@ public static GetEphemeralReadDeps create(TxnId txnId, PartialRoute scope, lo public GetEphemeralReadDeps(Id to, Topologies topologies, FullRoute route, TxnId txnId, Seekables keys, long executionEpoch) { super(to, topologies, txnId, route); - this.keys = keys.slice(scope.covering()); + this.keys = keys.intersecting(scope); this.executionEpoch = executionEpoch; } @@ -73,7 +73,7 @@ public void process() public GetEphemeralReadDepsOk apply(SafeCommandStore safeStore) { Ranges ranges = safeStore.ranges().allBetween(minUnsyncedEpoch, executionEpoch); - PartialDeps deps = calculatePartialDeps(safeStore, txnId, keys, constant(minUnsyncedEpoch), Timestamp.MAX, ranges); + PartialDeps deps = calculatePartialDeps(safeStore, txnId, keys, scope, constant(minUnsyncedEpoch), Timestamp.MAX, ranges); return new GetEphemeralReadDepsOk(deps, Math.max(safeStore.time().epoch(), node.epoch())); } diff --git a/accord-core/src/main/java/accord/messages/GetMaxConflict.java b/accord-core/src/main/java/accord/messages/GetMaxConflict.java index c5cc7f4a1..47173a739 100644 --- a/accord-core/src/main/java/accord/messages/GetMaxConflict.java +++ b/accord-core/src/main/java/accord/messages/GetMaxConflict.java @@ -48,7 +48,7 @@ public static GetMaxConflict create(PartialRoute scope, long waitForEpoch, lo public GetMaxConflict(Node.Id to, Topologies topologies, FullRoute route, Seekables keys, long executionEpoch) { super(to, topologies, executionEpoch, route); - this.keys = keys.slice(scope.covering()); + this.keys = keys.intersecting(scope); this.executionEpoch = executionEpoch; } diff --git a/accord-core/src/main/java/accord/messages/InformDurable.java b/accord-core/src/main/java/accord/messages/InformDurable.java index d79b50af2..7d1ee829d 100644 --- a/accord-core/src/main/java/accord/messages/InformDurable.java +++ b/accord-core/src/main/java/accord/messages/InformDurable.java @@ -17,6 +17,8 @@ */ package accord.messages; +import javax.annotation.Nullable; + import accord.local.Commands; import accord.local.Node.Id; import accord.local.PreLoadContext; @@ -24,8 +26,8 @@ import accord.local.SafeCommandStore; import accord.local.Status; import accord.local.Status.Durability; -import accord.primitives.FullRoute; import accord.primitives.PartialRoute; +import accord.primitives.Route; import accord.primitives.Timestamp; import accord.primitives.TxnId; import accord.topology.Topologies; @@ -44,17 +46,17 @@ public static InformDurable create(TxnId txnId, PartialRoute scope, long wait } } - public final Timestamp executeAt; + public final @Nullable Timestamp executeAt; public final Durability durability; - public InformDurable(Id to, Topologies topologies, FullRoute route, TxnId txnId, Timestamp executeAt, Durability durability) + public InformDurable(Id to, Topologies topologies, Route route, TxnId txnId, @Nullable Timestamp executeAt, Durability durability) { super(to, topologies, route, txnId); this.executeAt = executeAt; this.durability = durability; } - private InformDurable(TxnId txnId, PartialRoute scope, long waitForEpoch, Timestamp executeAt, Durability durability) + private InformDurable(TxnId txnId, PartialRoute scope, long waitForEpoch, @Nullable Timestamp executeAt, Durability durability) { super(txnId, scope, waitForEpoch); this.executeAt = executeAt; diff --git a/accord-core/src/main/java/accord/messages/InformHomeDurable.java b/accord-core/src/main/java/accord/messages/InformHomeDurable.java index ae871fd85..213c569e3 100644 --- a/accord-core/src/main/java/accord/messages/InformHomeDurable.java +++ b/accord-core/src/main/java/accord/messages/InformHomeDurable.java @@ -18,10 +18,6 @@ package accord.messages; -import java.util.Set; - -import com.google.common.collect.ImmutableSet; - import accord.local.Commands; import accord.local.Node; import accord.local.Node.Id; @@ -42,15 +38,13 @@ public class InformHomeDurable implements Request public final Route route; public final Timestamp executeAt; public final Durability durability; - public final Set persistedOn; - public InformHomeDurable(TxnId txnId, Route route, Timestamp executeAt, Durability durability, Set persistedOn) + public InformHomeDurable(TxnId txnId, Route route, Timestamp executeAt, Durability durability) { this.txnId = txnId; this.route = route; this.executeAt = executeAt; this.durability = durability; - this.persistedOn = ImmutableSet.copyOf(persistedOn); // Persisted on might be mutated later } @Override diff --git a/accord-core/src/main/java/accord/messages/PreAccept.java b/accord-core/src/main/java/accord/messages/PreAccept.java index ad964ff1a..6af9c1708 100644 --- a/accord-core/src/main/java/accord/messages/PreAccept.java +++ b/accord-core/src/main/java/accord/messages/PreAccept.java @@ -55,7 +55,7 @@ public PreAccept(Id to, Topologies topologies, TxnId txnId, Txn txn, FullRoute keys, EpochSupplier minEpoch, Timestamp executeAt, Ranges ranges) + static PartialDeps calculatePartialDeps(SafeCommandStore safeStore, TxnId txnId, Seekables keys, Route route, EpochSupplier minEpoch, Timestamp executeAt, Ranges ranges) { // TODO (expected): do not build covering ranges; no longer especially valuable given use of FullRoute // NOTE: ExclusiveSyncPoint *relies* on STARTED_BEFORE to ensure it reports a dependency on *every* earlier TxnId that may execute after it. // This is necessary for reporting to a bootstrapping replica which TxnId it must not prune from dependencies // i.e. the source replica reports to the target replica those TxnId that STARTED_BEFORE and EXECUTES_AFTER. - try (Deps.AbstractBuilder builder = new PartialDeps.Builder(ranges); - Deps.AbstractBuilder redundantBuilder = new PartialDeps.Builder(ranges)) + + Route scope = route.slice(ranges); + try (Deps.AbstractBuilder builder = new PartialDeps.Builder(scope); + Deps.AbstractBuilder redundantBuilder = new PartialDeps.Builder(scope)) { safeStore.mapReduceActive(keys, ranges, executeAt, txnId.kind().witnesses(), (p1, keyOrRange, testTxnId, testExecuteAt, in) -> { diff --git a/accord-core/src/main/java/accord/messages/Propagate.java b/accord-core/src/main/java/accord/messages/Propagate.java index cb23080eb..edb362ed2 100644 --- a/accord-core/src/main/java/accord/messages/Propagate.java +++ b/accord-core/src/main/java/accord/messages/Propagate.java @@ -152,7 +152,7 @@ public static void propagate(Node node, TxnId txnId, long sourceEpoch, long toEp return; } - Invariants.checkState(sourceEpoch == txnId.epoch() || (full.executeAt != null && sourceEpoch == full.executeAt.epoch()) || full.maxKnowledgeSaveStatus == SaveStatus.Erased || full.maxKnowledgeSaveStatus == SaveStatus.ErasedOrInvalidated); + Invariants.checkState(sourceEpoch == txnId.epoch() || (full.executeAt != null && sourceEpoch == full.executeAt.epoch()) || full.maxKnowledgeSaveStatus == SaveStatus.Erased || full.maxKnowledgeSaveStatus == SaveStatus.ErasedOrInvalidOrVestigial); full = full.finish(route, withQuorum); route = Invariants.nonNull(full.route); @@ -161,14 +161,14 @@ public static void propagate(Node node, TxnId txnId, long sourceEpoch, long toEp RoutingKey progressKey = node.trySelectProgressKey(txnId, route); - Ranges covering = route.sliceCovering(sliceRanges, Minimal); - Participants participatingKeys = route.participants().slice(covering, Minimal); - Status.Known achieved = full.knownFor(participatingKeys); + Route covering = route.slice(sliceRanges, Minimal); + Status.Known achieved = full.knownFor(covering); if (achieved.executeAt.isDecidedAndKnownToExecute() && full.executeAt.epoch() > toEpoch) { + Invariants.checkState(Route.isFullRoute(route)); Ranges acceptRanges; if (!node.topology().hasEpoch(full.executeAt.epoch()) || - (!route.covers(acceptRanges = node.topology().localRangesForEpochs(txnId.epoch(), full.executeAt.epoch())))) + (!covering.containsAll(route.slice(acceptRanges = node.topology().localRangesForEpochs(txnId.epoch(), full.executeAt.epoch()))))) { // we don't know what the execution epoch requires, so we cannot be sure we can replicate it locally // we *could* wait until we have the local epoch before running this @@ -179,9 +179,8 @@ public static void propagate(Node node, TxnId txnId, long sourceEpoch, long toEp { // TODO (expected): this should only be the two precise epochs, not the full range of epochs sliceRanges = acceptRanges; - covering = route.sliceCovering(sliceRanges, Minimal); - participatingKeys = route.participants().slice(covering, Minimal); - Status.Known knownForExecution = full.knownFor(participatingKeys); + covering = route.slice(sliceRanges, Minimal); + Status.Known knownForExecution = full.knownFor(covering); if ((target != null && target.isSatisfiedBy(knownForExecution)) || achieved.isSatisfiedBy(knownForExecution)) { achieved = knownForExecution; @@ -199,11 +198,11 @@ public static void propagate(Node node, TxnId txnId, long sourceEpoch, long toEp PartialTxn partialTxn = full.partialTxn; if (achieved.definition.isKnown()) - partialTxn = full.partialTxn.slice(sliceRanges, true).reconstitutePartial(covering); + partialTxn = full.partialTxn.intersecting(route, true).reconstitutePartial(covering); PartialDeps stableDeps = full.stableDeps; if (achieved.deps.hasDecidedDeps()) - stableDeps = full.stableDeps.slice(sliceRanges).reconstitutePartial(covering); + stableDeps = full.stableDeps.intersecting(route).reconstitutePartial(covering); Propagate propagate = new Propagate(txnId, route, full.maxKnowledgeSaveStatus, full.maxSaveStatus, full.acceptedOrCommitted, full.durability, full.homeKey, progressKey, achieved, full.map, isShardTruncated, partialTxn, stableDeps, toEpoch, full.executeAtIfKnown(), full.writes, full.result); @@ -261,19 +260,20 @@ public Void apply(SafeCommandStore safeStore) if (achieved == null) return null; - Ranges needed = safeStore.ranges().allBetween(txnId.epoch(), (executeAtIfKnown == null ? txnId : executeAtIfKnown).epoch()); - if (achieved.isDefinitionKnown() && partialTxn == null) + Participants needed = route.slice(safeStore.ranges().allBetween(txnId.epoch(), (executeAtIfKnown == null ? txnId : executeAtIfKnown).epoch())); + if (achieved.isDefinitionKnown() && partialTxn == null && this.partialTxn != null) { PartialTxn existing = command.partialTxn(); - Ranges neededForDefinition = existing == null ? needed : needed.subtract(existing.covering()); - partialTxn = this.partialTxn.slice(needed, true).reconstitutePartial(neededForDefinition); + Participants neededExtra = needed; + if (existing != null) neededExtra = neededExtra.subtract(existing.keys().toParticipants()); + partialTxn = this.partialTxn.intersecting(neededExtra, true).reconstitutePartial(neededExtra); } - if (achieved.hasDecidedDeps() && stableDeps == null) + if (achieved.hasDecidedDeps() && stableDeps == null && this.stableDeps != null) { Invariants.checkState(executeAtIfKnown != null); // we don't subtract existing partialDeps, as they cannot be committed deps; we only permit committing deps covering all participating ranges - stableDeps = this.stableDeps.slice(needed).reconstitutePartial(needed); + stableDeps = this.stableDeps.intersecting(needed).reconstitutePartial(needed); } } @@ -281,7 +281,7 @@ public Void apply(SafeCommandStore safeStore) if (command.hasBeen(propagate)) { if (maxSaveStatus.phase == Cleanup && durability.isDurableOrInvalidated() && Infer.safeToCleanup(safeStore, command, route, executeAtIfKnown)) - Commands.setTruncatedApply(safeStore, safeCommand); + Commands.setTruncatedApplyOrErasedVestigial(safeStore, safeCommand); // TODO (expected): maybe stale? return updateDurability(safeStore, safeCommand); @@ -410,7 +410,7 @@ private Status.Known applyOrUpgradeTruncated(SafeCommandStore safeStore, SafeCom if (ranges.isEmpty()) { // TODO (expected): we might prefer to adopt Redundant status, and permit ourselves to later accept the result of the execution and/or definition - Commands.setTruncatedApply(safeStore, safeCommand, executeAtIfKnown, route); + Commands.setTruncatedApplyOrErasedVestigial(safeStore, safeCommand, executeAtIfKnown, route); return null; } @@ -419,7 +419,7 @@ private Status.Known applyOrUpgradeTruncated(SafeCommandStore safeStore, SafeCom { // we only coordinate this transaction, so being unable to retrieve its state does not imply any staleness // TODO (now): double check this doesn't stop us coordinating the transaction (it shouldn't, as doesn't imply durability) - Commands.setTruncatedApply(safeStore, safeCommand, executeAtIfKnown, route); + Commands.setTruncatedApplyOrErasedVestigial(safeStore, safeCommand, executeAtIfKnown, route); return null; } @@ -445,7 +445,7 @@ private Status.Known applyOrUpgradeTruncated(SafeCommandStore safeStore, SafeCom return required; // TODO (expected): we might prefer to adopt Redundant status, and permit ourselves to later accept the result of the execution and/or definition - Commands.setTruncatedApply(safeStore, safeCommand, executeAtIfKnown, route); + Commands.setTruncatedApplyOrErasedVestigial(safeStore, safeCommand, executeAtIfKnown, route); return null; } diff --git a/accord-core/src/main/java/accord/messages/ReadData.java b/accord-core/src/main/java/accord/messages/ReadData.java index f59216131..f3c4b2e46 100644 --- a/accord-core/src/main/java/accord/messages/ReadData.java +++ b/accord-core/src/main/java/accord/messages/ReadData.java @@ -46,6 +46,7 @@ import static accord.messages.ReadData.CommitOrReadNack.Insufficient; import static accord.messages.ReadData.CommitOrReadNack.Redundant; import static accord.messages.TxnRequest.latestRelevantEpochIndex; +import static accord.primitives.Routables.Slice.Minimal; import static accord.utils.Invariants.illegalState; import static accord.utils.MapReduceConsume.forEach; @@ -191,6 +192,8 @@ protected synchronized CommitOrReadNack apply(SafeCommandStore safeStore, SafeCo waitingOn.set(safeStore.commandStore().id()); ++waitingOnCount; safeCommand.addListener(this); + // TODO (expected): should we invoke waiting directly? We depend on NotifyWaitingOn to make progress, and it will call this. + // Though we might get PreApplied info earlier. safeStore.progressLog().waiting(safeCommand, executeOn().min.execution, null, readScope); beginWaiting(safeStore, false); return status.compareTo(SaveStatus.Stable) >= 0 ? null : Insufficient; @@ -318,7 +321,7 @@ protected void onOneSuccess(@Nullable CommandStore commandStore, @Nullable Range if (newUnavailable != null && !newUnavailable.isEmpty()) { - newUnavailable = newUnavailable.intersecting(readScope); + newUnavailable = newUnavailable.intersecting(readScope, Minimal); if (this.unavailable == null) this.unavailable = newUnavailable; else this.unavailable = newUnavailable.with(this.unavailable); } diff --git a/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java b/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java index e170c3139..4ebba6624 100644 --- a/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java +++ b/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java @@ -32,6 +32,7 @@ import accord.primitives.PartialTxn; import accord.primitives.Participants; import accord.primitives.Ranges; +import accord.primitives.Route; import accord.primitives.Timestamp; import accord.primitives.Txn; import accord.primitives.TxnId; @@ -63,15 +64,15 @@ public ReadEphemeralTxnData(Id to, Topologies topologies, TxnId txnId, Participa private ReadEphemeralTxnData(Id to, Topologies topologies, TxnId txnId, Participants readScope, long executeAtEpoch, @Nonnull Txn txn, @Nonnull Deps deps, @Nonnull FullRoute route, int latestRelevantIndex) { - this(txnId, readScope, computeScope(to, topologies, null, latestRelevantIndex, (i, r) -> r, Ranges::with), executeAtEpoch, txn, deps, route); + this(txnId, readScope, computeScope(to, topologies, route, latestRelevantIndex), executeAtEpoch, txn, deps, route); } - private ReadEphemeralTxnData(TxnId txnId, Participants readScope, Ranges slice, long executeAtEpoch, @Nonnull Txn txn, @Nonnull Deps deps, @Nonnull FullRoute route) + private ReadEphemeralTxnData(TxnId txnId, Participants readScope, Route scope, long executeAtEpoch, @Nonnull Txn txn, @Nonnull Deps deps, @Nonnull FullRoute route) { - super(txnId, readScope.slice(slice), executeAtEpoch); + super(txnId, readScope.intersecting(scope), executeAtEpoch); this.route = route; - this.partialTxn = txn.slice(slice, false); - this.partialDeps = deps.slice(slice); + this.partialTxn = txn.intersecting(scope, false); + this.partialDeps = deps.intersecting(scope); } public ReadEphemeralTxnData(TxnId txnId, Participants readScope, long executeAtEpoch, @Nonnull PartialTxn partialTxn, @Nonnull PartialDeps partialDeps, @Nonnull FullRoute route) @@ -107,7 +108,7 @@ void read(SafeCommandStore safeStore, Command command) long retryInLaterEpoch = retryInLaterEpoch(executeAtEpoch, safeStore, command); if (retryInLaterEpoch > 0) { - // TODO (expected): wait for all stores' results and report only the ranges that execute later to be retried + // TODO (desired): wait for all stores' results and report only the ranges that execute later to be retried beginCancel(safeStore); node.reply(replyTo, replyContext, new ReadOkWithFutureEpoch(null, null, retryInLaterEpoch), null); } diff --git a/accord-core/src/main/java/accord/messages/TxnRequest.java b/accord-core/src/main/java/accord/messages/TxnRequest.java index db7de00c2..65e444a54 100644 --- a/accord-core/src/main/java/accord/messages/TxnRequest.java +++ b/accord-core/src/main/java/accord/messages/TxnRequest.java @@ -37,6 +37,7 @@ import accord.utils.Invariants; import accord.utils.MapReduceConsume; +import static accord.utils.Invariants.illegalArgument; import static java.lang.Long.min; public abstract class TxnRequest implements Request, PreLoadContext, MapReduceConsume @@ -107,6 +108,7 @@ RoutingKey progressKey() public final TxnId txnId; public final PartialRoute scope; public final long waitForEpoch; + // set on receive only protected transient RoutingKey progressKey; protected transient Node node; protected transient Id replyTo; @@ -284,7 +286,7 @@ public static O computeScope(Node.Id node, Topologies topologies, I keys, last = ranges; } if (scope == null) - throw new IllegalArgumentException("No intersection"); + throw illegalArgument("No intersection between " + topologies + " and " + keys + " on " + node); return scope; } diff --git a/accord-core/src/main/java/accord/messages/WaitUntilApplied.java b/accord-core/src/main/java/accord/messages/WaitUntilApplied.java index 74cd57820..6a41023eb 100644 --- a/accord-core/src/main/java/accord/messages/WaitUntilApplied.java +++ b/accord-core/src/main/java/accord/messages/WaitUntilApplied.java @@ -38,8 +38,6 @@ */ public class WaitUntilApplied extends ReadData { - private static final Logger logger = LoggerFactory.getLogger(WaitUntilApplied.class); - public static class SerializerSupport { public static WaitUntilApplied create(TxnId txnId, Participants scope, long executeAtEpoch) @@ -85,7 +83,9 @@ void read(SafeCommandStore safeStore, Command command) @Override protected ReadOk constructReadOk(Ranges unavailable, Data data) { - return new ReadOkWithFutureEpoch(unavailable, data, retryInLaterEpoch); + if (retryInLaterEpoch > 0) + return new ReadOkWithFutureEpoch(unavailable, data, retryInLaterEpoch); + return new ReadOk(unavailable, data); } @Override diff --git a/accord-core/src/main/java/accord/primitives/AbstractKeys.java b/accord-core/src/main/java/accord/primitives/AbstractKeys.java index 59ead2038..29b894677 100644 --- a/accord-core/src/main/java/accord/primitives/AbstractKeys.java +++ b/accord-core/src/main/java/accord/primitives/AbstractKeys.java @@ -155,7 +155,7 @@ public Stream stream() @Override public Iterator iterator() { - return new Iterator() + return new Iterator<>() { int i = 0; @Override @@ -180,7 +180,7 @@ public String toString() // TODO (expected, efficiency): accept cached buffers - protected K[] slice(Ranges ranges, IntFunction factory) + protected K[] slice(AbstractRanges ranges, IntFunction factory) { return SortedArrays.sliceWithMultipleMatches(keys, ranges.ranges, factory, (k, r) -> -r.compareTo(k), Range::compareTo); } @@ -204,12 +204,12 @@ protected static K[] subtract(Range range, K[] keys) return Arrays.copyOfRange(keys, start, end); } - protected K[] intersect(AbstractKeys that, ObjectBuffers buffers) + protected K[] intersecting(AbstractKeys that, ObjectBuffers buffers) { return SortedArrays.linearIntersection(this.keys, that.keys, buffers); } - protected K[] intersect(AbstractRanges ranges, ObjectBuffers buffers) + protected K[] intersecting(AbstractRanges ranges, ObjectBuffers buffers) { return SortedArrays.intersectWithMultipleMatches(keys, keys.length, ranges.ranges, ranges.ranges.length, (k, r) -> -r.compareTo(k), buffers); } @@ -294,15 +294,12 @@ public void forEach(Consumer forEach) public final FullKeyRoute toRoute(RoutingKey homeKey) { - if (isEmpty()) - return new FullKeyRoute(homeKey, false, new RoutingKey[] { homeKey }); - - return toRoutingKeysArray(homeKey, (routingKeys, homeKeyIndex, isParticipatingHomeKey) -> new FullKeyRoute(routingKeys[homeKeyIndex], isParticipatingHomeKey, routingKeys)); + return toRoutingKeysArray(homeKey, false, (routingKeys, homeKeyIndex, isParticipatingHomeKey) -> new FullKeyRoute(routingKeys[homeKeyIndex], routingKeys)); } - protected RoutingKey[] toRoutingKeysArray(RoutingKey withKey) + protected RoutingKey[] toRoutingKeysArray(RoutingKey withKey, boolean permitInsert) { - return toRoutingKeysArray(withKey, (routingKeys, homeKeyIndex, isParticipatingHomeKey) -> routingKeys); + return toRoutingKeysArray(withKey, permitInsert, (routingKeys, homeKeyIndex, isParticipatingHomeKey) -> routingKeys); } interface ToRoutingKeysFactory @@ -310,42 +307,36 @@ interface ToRoutingKeysFactory T apply(RoutingKey[] keys, int insertPos, boolean includesKey); } - @SuppressWarnings("SuspiciousSystemArraycopy") - protected T toRoutingKeysArray(RoutingKey withKey, ToRoutingKeysFactory toRoutingKeysFactory) + protected T toRoutingKeysArray(RoutingKey withKey, boolean permitInsert, ToRoutingKeysFactory toRoutingKeysFactory) { + RoutingKey[] copy; if (keys.getClass() == RoutingKey[].class) { - int insertPos = Arrays.binarySearch(keys, withKey); - if (insertPos >= 0) - { - Invariants.checkState(keys[insertPos].equals(withKey)); - return toRoutingKeysFactory.apply((RoutingKey[])keys, insertPos, true); - } - - insertPos = -1 - insertPos; - RoutingKey[] result = new RoutingKey[1 + keys.length]; - System.arraycopy(keys, 0, result, 0, insertPos); - result[insertPos] = withKey; - System.arraycopy(keys, insertPos, result, insertPos + 1, keys.length - insertPos); - return toRoutingKeysFactory.apply(result, insertPos, false); + copy = (RoutingKey[])keys; } else { - RoutingKey[] result = new RoutingKey[keys.length]; + copy = new RoutingKey[keys.length]; for (int i = 0; i < keys.length; i++) - result[i] = keys[i].toUnseekable(); - int insertPos = Arrays.binarySearch(result, withKey); - if (insertPos >= 0) - return toRoutingKeysFactory.apply(result, insertPos, true); - else - insertPos = -1 - insertPos; - - RoutingKey[] newResult = new RoutingKey[1 + keys.length]; - System.arraycopy(result, 0, newResult, 0, insertPos); - newResult[insertPos] = withKey; - System.arraycopy(result, insertPos, newResult, insertPos + 1, result.length - insertPos); - return toRoutingKeysFactory.apply(newResult, insertPos, false); + copy[i] = keys[i].toUnseekable(); } + + int insertPos = Arrays.binarySearch(copy, withKey); + if (insertPos >= 0) + { + Invariants.checkState(copy[insertPos].equals(withKey)); + return toRoutingKeysFactory.apply(copy, insertPos, true); + } + + if (!permitInsert) + throw new IllegalArgumentException(withKey + " is expected to be a member of " + Arrays.toString(copy)); + + insertPos = -1 - insertPos; + RoutingKey[] newResult = new RoutingKey[1 + copy.length]; + System.arraycopy(copy, 0, newResult, 0, insertPos); + newResult[insertPos] = withKey; + System.arraycopy(copy, insertPos, newResult, insertPos + 1, copy.length - insertPos); + return toRoutingKeysFactory.apply(newResult, insertPos, false); } public final RoutingKeys toParticipants() diff --git a/accord-core/src/main/java/accord/primitives/AbstractRanges.java b/accord-core/src/main/java/accord/primitives/AbstractRanges.java index 5fa94de37..781165837 100644 --- a/accord-core/src/main/java/accord/primitives/AbstractRanges.java +++ b/accord-core/src/main/java/accord/primitives/AbstractRanges.java @@ -100,6 +100,12 @@ public boolean containsAll(AbstractRanges that) return ((int) supersetLinearMerge(this.ranges, that.ranges)) == that.size(); } + @Override + public boolean intersectsAll(Unseekables keysOrRanges) + { + return intersectsAll((Routables) keysOrRanges); + } + public boolean intersectsAll(Routables that) { switch (that.domain()) @@ -178,13 +184,6 @@ public Ranges subtract(Unseekables keysOrRanges) return subtract((AbstractRanges) keysOrRanges); } - public Ranges intersect(Unseekables keysOrRanges) - { - if (keysOrRanges.domain() == Routable.Domain.Key) - keysOrRanges = ((AbstractUnseekableKeys)keysOrRanges).toRanges(); - return sliceMinimal((AbstractRanges)keysOrRanges, this, (AbstractRanges)keysOrRanges, (i1, i2, rs) -> i2.ranges == rs && i2 instanceof Ranges ? (Ranges)i2 : Ranges.ofSortedAndDeoverlapped(rs)); - } - // returns ri in low 32 bits, ki in top, or -1 if no match found @Override public final long findNextIntersection(int ri, AbstractKeys keys, int ki) @@ -297,21 +296,12 @@ else if (count == result.length) /** * Returns the inputs that intersect with any of the members of the keysOrRanges. - * DOES NOT MODIFY THE RANGES. + * DOES NOT MODIFY THE INPUT. */ - static I intersecting(I input, Routables keysOrRanges, P param, SliceConstructor constructor) + static I intersecting(AbstractUnseekableKeys intersecting, I input, P param, SliceConstructor constructor) { - switch (keysOrRanges.domain()) - { - default: throw new AssertionError(); - case Range: return sliceOverlapping((AbstractRanges)keysOrRanges, input, param, constructor); - case Key: - { - AbstractKeys that = (AbstractKeys) keysOrRanges; - Range[] result = SortedArrays.intersectWithMultipleMatches(input.ranges, input.ranges.length, that.keys, that.keys.length, Range::compareTo, cachedRanges()); - return result == input.ranges ? input : constructor.construct(input, param, result); - } - } + Range[] result = SortedArrays.intersectWithMultipleMatches(input.ranges, input.ranges.length, intersecting.keys, intersecting.keys.length, Range::compareTo, cachedRanges()); + return result == input.ranges ? input : constructor.construct(input, param, result); } interface SliceConstructor diff --git a/accord-core/src/main/java/accord/primitives/AbstractUnseekableKeys.java b/accord-core/src/main/java/accord/primitives/AbstractUnseekableKeys.java index 15e0be1a2..34d22d4ee 100644 --- a/accord-core/src/main/java/accord/primitives/AbstractUnseekableKeys.java +++ b/accord-core/src/main/java/accord/primitives/AbstractUnseekableKeys.java @@ -40,20 +40,32 @@ public final int indexOf(RoutingKey key) } @Override - public final AbstractUnseekableKeys intersect(Unseekables keysOrRanges) + public final boolean intersectsAll(Unseekables keysOrRanges) { - switch (keysOrRanges.domain()) + return containsAll(keysOrRanges); + } + + @Override + public AbstractUnseekableKeys intersecting(Unseekables intersecting, Slice slice) + { + return intersecting(intersecting); + } + + @Override + public AbstractUnseekableKeys intersecting(Unseekables intersecting) + { + switch (intersecting.domain()) { - default: throw new AssertionError("Unhandled domain: " + keysOrRanges.domain()); + default: throw new AssertionError("Unhandled domain: " + intersecting.domain()); case Key: { - AbstractUnseekableKeys that = (AbstractUnseekableKeys) keysOrRanges; - return weakWrap(intersect(that, ArrayBuffers.cachedRoutingKeys()), that); + AbstractUnseekableKeys that = (AbstractUnseekableKeys) intersecting; + return weakWrap(intersecting(that, ArrayBuffers.cachedRoutingKeys()), that); } case Range: { - AbstractRanges that = (AbstractRanges) keysOrRanges; - return wrap(intersect(that, ArrayBuffers.cachedRoutingKeys())); + AbstractRanges that = (AbstractRanges) intersecting; + return wrap(intersecting(that, ArrayBuffers.cachedRoutingKeys())); } } } diff --git a/accord-core/src/main/java/accord/primitives/Deps.java b/accord-core/src/main/java/accord/primitives/Deps.java index 2286dca47..ead3cbd6e 100644 --- a/accord-core/src/main/java/accord/primitives/Deps.java +++ b/accord-core/src/main/java/accord/primitives/Deps.java @@ -212,14 +212,9 @@ public Deps without(Predicate remove) return new Deps(keyDeps.without(remove), rangeDeps.without(remove), directKeyDeps.without(remove)); } - public PartialDeps slice(Ranges covering) + public PartialDeps intersecting(Participants participants) { - return slice(covering, covering); - } - - public PartialDeps slice(Ranges covering, Ranges slice) - { - return new PartialDeps(covering, keyDeps.slice(slice), rangeDeps.slice(slice), directKeyDeps.slice(slice)); + return new PartialDeps(participants, keyDeps.intersecting(participants), rangeDeps.intersecting(participants), directKeyDeps.intersecting(participants)); } public boolean isEmpty() @@ -315,6 +310,14 @@ public boolean equals(Deps that) return TxnId.nonNullOrMax(TxnId.nonNullOrMax(maxKeyDep, maxRangeDep), maxDirectKeyDep); } + public @Nullable TxnId minTxnId() + { + TxnId minKeyDep = keyDeps.isEmpty() ? null : keyDeps.txnId(0); + TxnId minRangeDep = rangeDeps.isEmpty() ? null : rangeDeps.txnId(0); + TxnId minDirectKeyDep = directKeyDeps.isEmpty() ? null : directKeyDeps.txnId(0); + return TxnId.nonNullOrMin(TxnId.nonNullOrMin(minKeyDep, minRangeDep), minDirectKeyDep); + } + @Override public int hashCode() { diff --git a/accord-core/src/main/java/accord/primitives/FullKeyRoute.java b/accord-core/src/main/java/accord/primitives/FullKeyRoute.java index dfe5fc3ea..ded732d25 100644 --- a/accord-core/src/main/java/accord/primitives/FullKeyRoute.java +++ b/accord-core/src/main/java/accord/primitives/FullKeyRoute.java @@ -25,15 +25,15 @@ public class FullKeyRoute extends KeyRoute implements FullRoute { public static class SerializationSupport { - public static FullKeyRoute create(RoutingKey homeKey, boolean isParticipatingHomeKey, RoutingKey[] keys) + public static FullKeyRoute create(RoutingKey homeKey, RoutingKey[] keys) { - return new FullKeyRoute(homeKey, isParticipatingHomeKey, keys); + return new FullKeyRoute(homeKey, keys); } } - public FullKeyRoute(RoutingKey homeKey, boolean isParticipatingHomeKey, RoutingKey[] keys) + public FullKeyRoute(RoutingKey homeKey, RoutingKey[] keys) { - super(homeKey, isParticipatingHomeKey, keys); + super(homeKey, keys); } @Override @@ -42,12 +42,6 @@ public UnseekablesKind kind() return UnseekablesKind.FullKeyRoute; } - @Override - public boolean covers(Ranges ranges) - { - return true; - } - @Override public FullKeyRoute with(RoutingKey withKey) { @@ -55,18 +49,6 @@ public FullKeyRoute with(RoutingKey withKey) return this; } - @Override - public PartialKeyRoute slice(Ranges newRanges) - { - return new PartialKeyRoute(newRanges, homeKey, isParticipatingHomeKey, slice(newRanges, RoutingKey[]::new)); - } - - @Override - public PartialKeyRoute sliceStrict(Ranges ranges) - { - return slice(ranges); - } - @Override public FullKeyRoute withHomeKey() { @@ -78,5 +60,4 @@ public String toString() { return "{homeKey:" + homeKey + ',' + super.toString() + '}'; } - } diff --git a/accord-core/src/main/java/accord/primitives/FullRangeRoute.java b/accord-core/src/main/java/accord/primitives/FullRangeRoute.java index 8a614a23e..ea755ddc6 100644 --- a/accord-core/src/main/java/accord/primitives/FullRangeRoute.java +++ b/accord-core/src/main/java/accord/primitives/FullRangeRoute.java @@ -24,15 +24,15 @@ public class FullRangeRoute extends RangeRoute implements FullRoute { public static class SerializationSupport { - public static FullRangeRoute create(RoutingKey homeKey, boolean isParticipatingHomeKey, Range[] ranges) + public static FullRangeRoute create(RoutingKey homeKey, Range[] ranges) { - return new FullRangeRoute(homeKey, isParticipatingHomeKey, ranges); + return new FullRangeRoute(homeKey, ranges); } } - public FullRangeRoute(RoutingKey homeKey, boolean isParticipatingHomeKey, Range[] ranges) + public FullRangeRoute(RoutingKey homeKey, Range[] ranges) { - super(homeKey, isParticipatingHomeKey, ranges); + super(homeKey, ranges); } @Override @@ -41,18 +41,6 @@ public UnseekablesKind kind() return UnseekablesKind.FullRangeRoute; } - @Override - public boolean covers(Ranges ranges) - { - return true; - } - - @Override - public PartialRangeRoute sliceStrict(Ranges ranges) - { - return slice(ranges); - } - @Override public FullRangeRoute withHomeKey() { diff --git a/accord-core/src/main/java/accord/primitives/FullRoute.java b/accord-core/src/main/java/accord/primitives/FullRoute.java index addddd9c7..be0ba22a8 100644 --- a/accord-core/src/main/java/accord/primitives/FullRoute.java +++ b/accord-core/src/main/java/accord/primitives/FullRoute.java @@ -21,5 +21,4 @@ public interface FullRoute extends Route, Unseekables { @Override default FullRoute union(Route route) { return this; } - @Override default Ranges sliceCovering(Ranges newRanges, Slice slice) { return newRanges; } } diff --git a/accord-core/src/main/java/accord/primitives/KeyDeps.java b/accord-core/src/main/java/accord/primitives/KeyDeps.java index eb9c5b225..2bfe35ffe 100644 --- a/accord-core/src/main/java/accord/primitives/KeyDeps.java +++ b/accord-core/src/main/java/accord/primitives/KeyDeps.java @@ -25,6 +25,7 @@ import accord.utils.IndexedConsumer; import accord.utils.IndexedTriConsumer; import accord.utils.RelationMultiMap; +import accord.utils.SortedArrays; import accord.utils.SortedArrays.SortedArrayList; import accord.utils.SymmetricComparator; import accord.utils.TriFunction; @@ -191,9 +192,21 @@ 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); + return select(keys.slice(ranges)); + + } + public KeyDeps intersecting(Unseekables participants) + { + if (isEmpty()) + return new KeyDeps(keys, txnIds, keysToTxnIds); + + return select(keys.intersecting(participants)); + } + + private KeyDeps select(Keys select) + { + // TODO (low priority, efficiency): can slice in parallel with selecting keyToTxnId contents to avoid duplicate merging if (select.isEmpty()) return new KeyDeps(Keys.EMPTY, NO_TXNIDS, NO_INTS); diff --git a/accord-core/src/main/java/accord/primitives/KeyRoute.java b/accord-core/src/main/java/accord/primitives/KeyRoute.java index 6d413b813..dbefd32c8 100644 --- a/accord-core/src/main/java/accord/primitives/KeyRoute.java +++ b/accord-core/src/main/java/accord/primitives/KeyRoute.java @@ -18,8 +18,6 @@ package accord.primitives; -import java.util.Arrays; - import accord.utils.Invariants; import accord.api.RoutingKey; @@ -32,31 +30,17 @@ public abstract class KeyRoute extends AbstractUnseekableKeys implements Route { public final RoutingKey homeKey; - public final boolean isParticipatingHomeKey; - KeyRoute(@Nonnull RoutingKey homeKey, boolean isParticipatingHomeKey, RoutingKey[] keys) + KeyRoute(@Nonnull RoutingKey homeKey, RoutingKey[] keys) { super(keys); this.homeKey = Invariants.nonNull(homeKey); - this.isParticipatingHomeKey = isParticipatingHomeKey; } @Override public boolean participatesIn(Ranges ranges) { - if (isParticipatingHomeKey()) - return intersects(ranges); - - long ij = findNextIntersection(0, ranges, 0); - if (ij < 0) - return false; - - int i = (int)ij; - if (!get(i).equals(homeKey)) - return true; - - int j = (int)(ij >>> 32); - return findNextIntersection(i + 1, ranges, j) >= 0; + return intersects(ranges); } @SuppressWarnings("unchecked") @@ -78,36 +62,14 @@ public Participants with(Participants with) @Override public Participants participants() { - if (isParticipatingHomeKey) - return this; - - int removePos = Arrays.binarySearch(keys, homeKey); - if (removePos < 0) - return this; - - RoutingKey[] result = new RoutingKey[keys.length - 1]; - System.arraycopy(keys, 0, result, 0, removePos); - System.arraycopy(keys, removePos + 1, result, removePos, keys.length - (1 + removePos)); - // TODO (expected): this should return a PartialKeyRoute, but we need to remove covering() - return new RoutingKeys(result); + return this; } @Override public Participants participants(Ranges ranges) { RoutingKey[] keys = slice(ranges, RoutingKey[]::new); - if (isParticipatingHomeKey) - return keys == this.keys ? this : new RoutingKeys(keys); - - int removePos = Arrays.binarySearch(keys, homeKey); - if (removePos < 0) - return new RoutingKeys(keys); - - RoutingKey[] result = new RoutingKey[keys.length - 1]; - System.arraycopy(keys, 0, result, 0, removePos); - System.arraycopy(keys, removePos + 1, result, removePos, keys.length - (1 + removePos)); - // TODO (expected): this should return a PartialKeyRoute, but we need to remove covering() - return new RoutingKeys(result); + return keys == this.keys ? this : new RoutingKeys(keys); } @Override @@ -116,37 +78,40 @@ public Participants participants(Ranges ranges, Slice slice) return participants(ranges); } - public Ranges toRanges() - { - Invariants.checkState(isParticipatingHomeKey); - return super.toRanges(); - } - @Override public RoutingKey homeKey() { return homeKey; } + @Override - public boolean isParticipatingHomeKey() + public PartialKeyRoute slice(Ranges select) { - return isParticipatingHomeKey; + return new PartialKeyRoute(homeKey, slice(select, RoutingKey[]::new)); } @Override - public RoutingKey someParticipatingKey() + public PartialKeyRoute slice(Ranges ranges, Slice slice) { - return isParticipatingHomeKey ? homeKey : keys[0]; + return slice(ranges); } @Override - public abstract PartialKeyRoute slice(Ranges ranges); + public PartialKeyRoute intersecting(Unseekables intersecting) + { + switch (intersecting.domain()) + { + default: throw new AssertionError("Unhandled domain: " + intersecting.domain()); + case Key: return new PartialKeyRoute(homeKey, intersecting((AbstractUnseekableKeys)intersecting, cachedRoutingKeys())); + case Range: return new PartialKeyRoute(homeKey, slice((AbstractRanges)intersecting, RoutingKey[]::new)); + } + } @Override - public PartialKeyRoute slice(Ranges ranges, Slice slice) + public PartialKeyRoute intersecting(Unseekables intersecting, Slice slice) { - return slice(ranges); + return intersecting(intersecting); } private AbstractUnseekableKeys wrap(RoutingKey[] wrap, AbstractKeys that) diff --git a/accord-core/src/main/java/accord/primitives/Keys.java b/accord-core/src/main/java/accord/primitives/Keys.java index b6fa5624b..72bdc88bd 100644 --- a/accord-core/src/main/java/accord/primitives/Keys.java +++ b/accord-core/src/main/java/accord/primitives/Keys.java @@ -26,8 +26,9 @@ import java.util.function.Function; import accord.api.Key; -import accord.utils.ArrayBuffers; +import accord.primitives.Routable.Domain; import accord.utils.ArrayBuffers.ObjectBuffers; +import accord.utils.Invariants; import accord.utils.SortedArrays; import static accord.utils.ArrayBuffers.cachedKeys; @@ -84,9 +85,37 @@ public Keys slice(Ranges ranges, Slice slice) return wrap(slice(ranges, Key[]::new)); } - public final Keys intersect(Keys that) + @Override + public final boolean intersectsAll(Unseekables keysOrRanges) + { + Invariants.checkArgument(keysOrRanges.domain() == Domain.Key); + AbstractUnseekableKeys that = (AbstractUnseekableKeys) keysOrRanges; + return SortedArrays.isSubset((rk, k) -> -k.compareAsRoutingKey(rk), that.keys, 0, that.keys.length, this.keys, 0, this.keys.length); + } + + public final Keys intersecting(Unseekables intersecting) + { + switch (intersecting.domain()) + { + default: throw new AssertionError("Unhandled domain: " + intersecting.domain()); + case Key: return intersecting((AbstractUnseekableKeys) intersecting); + case Range: return wrap(intersecting((AbstractRanges) intersecting, cachedKeys())); + } + } + + public final Keys intersecting(Unseekables intersecting, Slice slice) + { + return intersecting(intersecting); + } + + public final Keys intersecting(Keys that) + { + return wrap(SortedArrays.linearIntersection(this.keys, that.keys, cachedKeys()), that); + } + + public final Keys intersecting(AbstractUnseekableKeys that) { - return wrap(SortedArrays.linearIntersection(this.keys, that.keys, ArrayBuffers.cachedKeys()), that); + return wrap(SortedArrays.intersectWithMultipleMatches(this.keys, this.keys.length, that.keys, that.keys.length, Key::compareAsRoutingKey, cachedKeys()), this); } public Keys with(Key key) @@ -168,7 +197,7 @@ public static Keys of(List input, Function tran public static Keys ofMergeSorted(List as, Function fa, List bs, Function fb) { - ObjectBuffers cache = ArrayBuffers.cachedKeys(); + ObjectBuffers cache = cachedKeys(); int asSize = as.size(), bsSize = bs.size(); Key[] array = cache.get(asSize + bsSize); int count = 0; diff --git a/accord-core/src/main/java/accord/primitives/PartialDeps.java b/accord-core/src/main/java/accord/primitives/PartialDeps.java index 7b17b3c71..4c73cd967 100644 --- a/accord-core/src/main/java/accord/primitives/PartialDeps.java +++ b/accord-core/src/main/java/accord/primitives/PartialDeps.java @@ -24,14 +24,14 @@ public class PartialDeps extends Deps { public static final PartialDeps NONE = new PartialDeps(Ranges.EMPTY, KeyDeps.NONE, RangeDeps.NONE, KeyDeps.NONE); - public static Builder builder(Ranges covering) + public static Builder builder(Participants covering) { return new Builder(covering); } public static class Builder extends AbstractBuilder { - final Ranges covering; - public Builder(Ranges covering) + final Participants covering; + public Builder(Participants covering) { this.covering = covering; } @@ -50,20 +50,17 @@ public PartialDeps build() // could also retain a simple bitset over the original FullRoute // TODO (required) remove this and related concepts, as can cause problems with topology changes for a single store // where the store has some ranges that we participate in, and some we do not; we will not correctly construct covering in some cases - public final Ranges covering; - public PartialDeps(Ranges covering, KeyDeps keyDeps, RangeDeps rangeDeps, KeyDeps directKeyDeps) + public final Participants covering; // set only if this is a range transaction, containing the minimal ranges of the original transaction that we cover + public PartialDeps(Participants covering, KeyDeps keyDeps, RangeDeps rangeDeps, KeyDeps directKeyDeps) { super(keyDeps, rangeDeps, directKeyDeps); this.covering = covering; - Invariants.checkState(covering.containsAll(keyDeps.keys)); - Invariants.checkState(covering.containsAll(directKeyDeps.keys)); - Invariants.checkState(rangeDeps.isCoveredBy(covering)); } - public boolean covers(Participants participants) + public boolean covers(Unseekables participants) { - return covering.containsAll(participants); + return covering.intersectsAll(participants); } public Deps with(Deps that) @@ -75,10 +72,11 @@ public Deps with(Deps that) public PartialDeps with(PartialDeps that) { - return new PartialDeps(that.covering.with(this.covering), - this.keyDeps.with(that.keyDeps), - this.rangeDeps.with(that.rangeDeps), - this.directKeyDeps.with(that.directKeyDeps) + return new PartialDeps( + this.covering.with((Participants)that.covering), + this.keyDeps.with(that.keyDeps), + this.rangeDeps.with(that.rangeDeps), + this.directKeyDeps.with(that.directKeyDeps) ); } @@ -90,35 +88,12 @@ public Deps reconstitute(FullRoute route) } // covering might cover a wider set of ranges, some of which may have no involved keys - public PartialDeps reconstitutePartial(Ranges covering) + public PartialDeps reconstitutePartial(Participants covering) { if (!covers(covering)) throw new IllegalArgumentException(); if (covers(covering)) return this; - else throw Invariants.illegalArgument(this.covering + " does not cover " + covering); - } - - @Override - public boolean equals(Object that) - { - return this == that || (that instanceof PartialDeps && equals((PartialDeps) that)); - } - - @Override - public boolean equals(Deps that) - { - return that instanceof PartialDeps && equals((PartialDeps) that); - } - - public boolean equals(PartialDeps that) - { - return that != null && this.covering.equals(that.covering) && super.equals(that); - } - - @Override - public String toString() - { - return covering + ":" + super.toString(); + else throw Invariants.illegalArgument(this + " does not cover " + covering); } } diff --git a/accord-core/src/main/java/accord/primitives/PartialKeyRoute.java b/accord-core/src/main/java/accord/primitives/PartialKeyRoute.java index 781c050a5..79e6c0a40 100644 --- a/accord-core/src/main/java/accord/primitives/PartialKeyRoute.java +++ b/accord-core/src/main/java/accord/primitives/PartialKeyRoute.java @@ -18,6 +18,8 @@ package accord.primitives; +import java.util.Arrays; + import accord.utils.Invariants; import accord.api.RoutingKey; @@ -30,34 +32,25 @@ public class PartialKeyRoute extends KeyRoute implements PartialRoute= 0) return this; - return new PartialKeyRoute(covering.with(Ranges.of(homeKey.asRange())), homeKey, isParticipatingHomeKey, toRoutingKeysArray(homeKey)); - } - @Override - public PartialKeyRoute slice(Ranges newRanges, Slice slice) - { - if (newRanges.containsAll(covering)) - return this; + insertPos = -1 - insertPos; + RoutingKey[] keys = new RoutingKey[1 + this.keys.length]; + System.arraycopy(this.keys, 0, keys, 0, insertPos); + keys[insertPos] = homeKey; + System.arraycopy(this.keys, insertPos, keys, insertPos + 1, this.keys.length - insertPos); - RoutingKey[] keys = slice(newRanges, RoutingKey[]::new); - return new PartialKeyRoute(newRanges, homeKey, isParticipatingHomeKey, keys); + return new PartialKeyRoute(homeKey, keys); } @Override - public Ranges covering() + public PartialKeyRoute slice(Ranges select, Slice slice) { - return covering; + RoutingKey[] keys = slice(select, RoutingKey[]::new); + if (keys == this.keys) + return this; + + return new PartialKeyRoute(homeKey, keys); } @Override @@ -123,13 +112,13 @@ public PartialKeyRoute union(PartialRoute with) PartialKeyRoute that = (PartialKeyRoute) with; Invariants.checkState(homeKey.equals(that.homeKey)); - Invariants.checkState(isParticipatingHomeKey == that.isParticipatingHomeKey); RoutingKey[] keys = SortedArrays.linearUnion(this.keys, that.keys, RoutingKey[]::new); - Ranges covering = this.covering.with(that.covering); - if (covering == this.covering && keys == this.keys) + if (keys == this.keys || keys == that.keys) + { + if (keys != that.keys) return this; + if (keys != this.keys) return that; return this; - if (covering == that.covering && keys == that.keys) - return that; - return new PartialKeyRoute(covering, homeKey, isParticipatingHomeKey, keys); + } + return new PartialKeyRoute(homeKey, keys); } } diff --git a/accord-core/src/main/java/accord/primitives/PartialRangeRoute.java b/accord-core/src/main/java/accord/primitives/PartialRangeRoute.java index e657758db..b32959e41 100644 --- a/accord-core/src/main/java/accord/primitives/PartialRangeRoute.java +++ b/accord-core/src/main/java/accord/primitives/PartialRangeRoute.java @@ -18,8 +18,6 @@ package accord.primitives; -import javax.annotation.Nonnull; - import accord.api.RoutingKey; import accord.utils.Invariants; @@ -32,45 +30,27 @@ public class PartialRangeRoute extends RangeRoute implements PartialRoute { public static class SerializationSupport { - public static PartialRangeRoute create(Ranges covering, RoutingKey homeKey, boolean isParticipatingHomeKey, Range[] ranges) + public static PartialRangeRoute create(RoutingKey homeKey, Range[] ranges) { - return new PartialRangeRoute(covering, homeKey, isParticipatingHomeKey, ranges); + return new PartialRangeRoute(homeKey, ranges); } } - public final Ranges covering; - - public PartialRangeRoute(Ranges covering, RoutingKey homeKey, boolean isParticipatingHomeKey, Range[] ranges) + public PartialRangeRoute(RoutingKey homeKey, Range[] ranges) { - super(homeKey, isParticipatingHomeKey, ranges); - this.covering = covering; + super(homeKey, ranges); } - @Override - public UnseekablesKind kind() - { - return UnseekablesKind.PartialRangeRoute; - } - - @Override - public Ranges covering() - { - return covering; - } - - @Override - public boolean covers(Ranges ranges) + // + private PartialRangeRoute(Object ignore, RoutingKey homeKey, Range[] ranges) { - return covering.containsAll(ranges); + super(homeKey, ranges); } @Override - public PartialRangeRoute sliceStrict(Ranges newRanges) + public UnseekablesKind kind() { - if (!covering.containsAll(newRanges)) - throw new IllegalArgumentException("Not covered"); - - return slice(newRanges); + return UnseekablesKind.PartialRangeRoute; } @Override @@ -80,7 +60,7 @@ public PartialRangeRoute withHomeKey() return this; Ranges with = Ranges.of(homeKey.asRange()); - return new PartialRangeRoute(covering.with(with), homeKey, isParticipatingHomeKey, union(MERGE_OVERLAPPING, this, with, null, null, (i1, i2, rs) -> rs)); + return new PartialRangeRoute(homeKey, union(MERGE_OVERLAPPING, this, with, null, null, (i1, i2, rs) -> rs)); } @Override @@ -98,28 +78,7 @@ public PartialRangeRoute union(PartialRoute with) PartialRangeRoute that = (PartialRangeRoute) with; Invariants.checkState(homeKey.equals(that.homeKey)); - Ranges covering = this.covering.with(that.covering); - if (covering == this.covering) return this; - else if (covering == that.covering) return that; - return union(MERGE_OVERLAPPING, this, that, covering, homeKey, - isParticipatingHomeKey ? PartialRangeRoute::withParticipatingHomeKey - : PartialRangeRoute::withNonParticipatingHomeKey); - } - - @Override - public boolean equals(Object that) - { - return super.equals(that) && covering.equals(((PartialRangeRoute)that).covering); - } - - static PartialRangeRoute withParticipatingHomeKey(Ranges covering, @Nonnull RoutingKey homeKey, Range[] ranges) - { - return new PartialRangeRoute(covering, homeKey, true, ranges); - } - - static PartialRangeRoute withNonParticipatingHomeKey(Ranges covering, @Nonnull RoutingKey homeKey, Range[] ranges) - { - return new PartialRangeRoute(covering, homeKey, false, ranges); + return union(MERGE_OVERLAPPING, this, that, null, homeKey, (ignore, homeKey, ranges) -> new PartialRangeRoute(homeKey, ranges)); } } diff --git a/accord-core/src/main/java/accord/primitives/PartialRoute.java b/accord-core/src/main/java/accord/primitives/PartialRoute.java index 4cfee21c3..74c7b5e05 100644 --- a/accord-core/src/main/java/accord/primitives/PartialRoute.java +++ b/accord-core/src/main/java/accord/primitives/PartialRoute.java @@ -22,16 +22,9 @@ public interface PartialRoute extends Route { @Override boolean isEmpty(); - Ranges covering(); /** * Expected to be compatible PartialRoute type, i.e. both split from the same FullRoute */ PartialRoute union(PartialRoute route); - - @Override - default Ranges sliceCovering(Ranges newRanges, Slice slice) - { - return covering().slice(newRanges, slice); - } } diff --git a/accord-core/src/main/java/accord/primitives/PartialTxn.java b/accord-core/src/main/java/accord/primitives/PartialTxn.java index 6bcb9ca86..54ff06537 100644 --- a/accord-core/src/main/java/accord/primitives/PartialTxn.java +++ b/accord-core/src/main/java/accord/primitives/PartialTxn.java @@ -29,32 +29,22 @@ public interface PartialTxn extends Txn { // TODO (expected): we no longer need this if everyone has a FullRoute - Ranges covering(); // TODO (low priority, efficiency): efficient merge when more than one input PartialTxn with(PartialTxn add); Txn reconstitute(FullRoute route); - PartialTxn reconstitutePartial(Ranges covering); + PartialTxn reconstitutePartial(Participants covering); default boolean covers(Route route) { if (query() == null && route.contains(route.homeKey())) return false; - return covers(route.participants()); + return keys().intersectsAll(route); } - default boolean covers(Participants participants) + default boolean covers(Unseekables participants) { - if (query() == null) - { - // The home shard is expected to store the query contents - // So if the query is null, and we are being asked if we - // cover a range that includes a home shard, we should say no - Route asRoute = Route.tryCastToRoute(participants); - if (asRoute != null && asRoute.contains(asRoute.homeKey())) - return false; - } - return covering().containsAll(participants); + return keys().intersectsAll(participants); } static PartialTxn merge(@Nullable PartialTxn a, @Nullable PartialTxn b) @@ -65,18 +55,9 @@ static PartialTxn merge(@Nullable PartialTxn a, @Nullable PartialTxn b) // TODO (low priority, clarity): override toString class InMemory extends Txn.InMemory implements PartialTxn { - public final Ranges covering; - - public InMemory(Ranges covering, Kind kind, Seekables keys, Read read, Query query, Update update) + public InMemory(Kind kind, Seekables keys, Read read, Query query, Update update) { super(kind, keys, read, query, update); - this.covering = covering; - } - - @Override - public Ranges covering() - { - return covering; } @Override @@ -85,27 +66,21 @@ public PartialTxn with(PartialTxn add) if (!add.kind().equals(kind())) throw new IllegalArgumentException(); - Ranges covering = this.covering.with(add.covering()); Seekables keys = ((Seekables)this.keys()).with(add.keys()); Read read = this.read().merge(add.read()); Query query = this.query() == null ? add.query() : this.query(); Update update = this.update() == null ? null : this.update().merge(add.update()); if (keys == this.keys()) { - if (covering == this.covering && read == this.read() && query == this.query() && update == this.update()) + if (read == this.read() && query == this.query() && update == this.update()) return this; } else if (keys == add.keys()) { - if (covering == add.covering() && read == add.read() && query == add.query() && update == add.update()) + if (read == add.read() && query == add.query() && update == add.update()) return add; } - return new PartialTxn.InMemory(covering, kind(), keys, read, query, update); - } - - public boolean covers(Ranges ranges) - { - return covering.containsAll(ranges); + return new PartialTxn.InMemory(kind(), keys, read, query, update); } @Override @@ -118,15 +93,15 @@ public Txn reconstitute(FullRoute route) } @Override - public PartialTxn reconstitutePartial(Ranges covering) + public PartialTxn reconstitutePartial(Participants covering) { if (!covers(covering)) throw illegalState("Incomplete PartialTxn: " + this + ", covering: " + covering); - if (this.covering.containsAll(covering)) + if (this.keys().containsAll(covering)) return this; - return new PartialTxn.InMemory(covering, kind(), keys(), read(), query(), update()); + return new PartialTxn.InMemory(kind(), keys(), read(), query(), update()); } } } diff --git a/accord-core/src/main/java/accord/primitives/Participants.java b/accord-core/src/main/java/accord/primitives/Participants.java index 1f84ae3be..3cda77c1b 100644 --- a/accord-core/src/main/java/accord/primitives/Participants.java +++ b/accord-core/src/main/java/accord/primitives/Participants.java @@ -24,12 +24,15 @@ * that these classes will only be used via interfaces such as Route that do not extend this interface, * so that the implementation may return itself when suitable, and a converted copy otherwise. */ +// TODO (desired): so we need this abstraction anymore, now that we have removed the concept of non-participating home keys? public interface Participants extends Unseekables { Participants with(Participants with); @Override - Participants intersect(Unseekables with); + Participants intersecting(Unseekables intersecting); + @Override + Participants intersecting(Unseekables intersecting, Slice slice); @Override Participants slice(Ranges ranges); @Override @@ -40,39 +43,6 @@ public interface Participants extends Unseekables Ranges toRanges(); - static boolean isParticipants(Unseekables unseekables) - { - switch (unseekables.kind()) - { - default: throw new AssertionError("Unhandled Unseekables.Kind: " + unseekables.kind()); - case FullKeyRoute: - case FullRangeRoute: - case PartialKeyRoute: - case PartialRangeRoute: - return Route.castToRoute(unseekables).isParticipatingHomeKey(); - case RoutingRanges: - case RoutingKeys: - return true; - } - } - - static Participants participants(Unseekables unseekables) - { - switch (unseekables.kind()) - { - default: throw new AssertionError("Unhandled Unseekables.Kind: " + unseekables.kind()); - case FullKeyRoute: - case FullRangeRoute: - case PartialKeyRoute: - case PartialRangeRoute: - return Route.castToRoute(unseekables).participants(); - case RoutingRanges: - return (Ranges) unseekables; - case RoutingKeys: - return (RoutingKeys) unseekables; - } - } - /** * If both left and right are a Route, invoke {@link Route#union} on them. Otherwise invoke {@link #with}. */ diff --git a/accord-core/src/main/java/accord/primitives/RangeDeps.java b/accord-core/src/main/java/accord/primitives/RangeDeps.java index 4cc18d293..0ea6899a1 100644 --- a/accord-core/src/main/java/accord/primitives/RangeDeps.java +++ b/accord-core/src/main/java/accord/primitives/RangeDeps.java @@ -188,7 +188,7 @@ public void forEach(RoutableKey key, Consumer forEach) @Inline public int forEach(RoutableKey key, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4, int minIndex) { - return ensureSearchable().forEach(key, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); + return ensureSearchable().forEachKey(key, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); } private int forEach(RoutableKey key, BiConsumer forEach, P1 p1, int minIndex, @Nullable BitSet visited) @@ -211,22 +211,29 @@ private int forEach(RoutableKey key, IndexedBiConsumer forEach, @Inline public int forEach(Range range, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4, int minIndex) { - return ensureSearchable().forEach(range, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); + return ensureSearchable().forEachRange(range, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); } @Inline public int forEach(RoutingKey start, RoutingKey end, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4, int minIndex) { - return ensureSearchable().forEach(start, end, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); + return ensureSearchable().forEachRange(start, end, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); } - private void forEach(Ranges ranges, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4) + private void forEach(AbstractRanges ranges, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4) { int minIndex = 0; for (int i = 0; i < ranges.size() ; ++i) minIndex = forEach(ranges.get(i), forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); } + private void forEach(AbstractUnseekableKeys keys, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4) + { + int minIndex = 0; + for (int i = 0; i < keys.size() ; ++i) + minIndex = forEach(keys.get(i), forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); + } + private int forEach(Range range, BiConsumer forEach, P1 p1, int minIndex, @Nullable BitSet visited) { return forEach(range, RangeDeps::visitTxnIdsForRangeIndex, RangeDeps::visitTxnIdsForRangeIndex, @@ -574,7 +581,22 @@ void ensureTxnIdToRange() txnIdsToRanges = invert(rangesToTxnIds, rangesToTxnIds.length, ranges.length, txnIds.length); } + public RangeDeps intersecting(Unseekables select) + { + switch (select.domain()) + { + default: throw new AssertionError("Unhandled domain: " + select.domain()); + case Key: return intersecting((AbstractUnseekableKeys) select); + case Range: return slice((AbstractRanges) select); + } + } + public RangeDeps slice(Ranges select) + { + return slice((AbstractRanges) select); + } + + private RangeDeps slice(AbstractRanges select) { if (isEmpty()) return new RangeDeps(NO_RANGES, txnIds, NO_INTS); @@ -582,20 +604,36 @@ public RangeDeps slice(Ranges select) try (RangeAndMapCollector collector = new RangeAndMapCollector(ensureSearchable().maxScanAndCheckpointMatches)) { forEach(select, collector, collector, ranges, rangesToTxnIds, null, null); + return build(collector); + } + } - if (collector.rangesCount == 0) - return new RangeDeps(NO_RANGES, NO_TXNIDS, NO_INTS); - - if (collector.rangesCount == this.ranges.length) - return this; + private RangeDeps intersecting(AbstractUnseekableKeys select) + { + if (isEmpty()) + return new RangeDeps(NO_RANGES, txnIds, NO_INTS); - Range[] ranges = collector.getRanges(); - int[] rangesToTxnIds = collector.getRangesToTxnIds(); - TxnId[] txnIds = trimUnusedValues(ranges, this.txnIds, rangesToTxnIds, TxnId[]::new); - return new RangeDeps(ranges, txnIds, rangesToTxnIds); + try (RangeAndMapCollector collector = new RangeAndMapCollector(ensureSearchable().maxScanAndCheckpointMatches)) + { + forEach(select, collector, collector, ranges, rangesToTxnIds, null, null); + return build(collector); } } + private RangeDeps build(RangeAndMapCollector collector) + { + 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()) @@ -624,26 +662,6 @@ 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 SortedArrayList txnIds() { return new SortedArrayList<>(txnIds); diff --git a/accord-core/src/main/java/accord/primitives/RangeRoute.java b/accord-core/src/main/java/accord/primitives/RangeRoute.java index 39c41c56a..5994d7053 100644 --- a/accord-core/src/main/java/accord/primitives/RangeRoute.java +++ b/accord-core/src/main/java/accord/primitives/RangeRoute.java @@ -29,14 +29,11 @@ public abstract class RangeRoute extends AbstractRanges implements Route, Unseekables, Participants { public final RoutingKey homeKey; - public final boolean isParticipatingHomeKey; - RangeRoute(@Nonnull RoutingKey homeKey, boolean isParticipatingHomeKey, Range[] ranges) + RangeRoute(@Nonnull RoutingKey homeKey, Range[] ranges) { super(ranges); this.homeKey = Invariants.nonNull(homeKey); - this.isParticipatingHomeKey = isParticipatingHomeKey; - Invariants.checkArgument(isParticipatingHomeKey || !contains(homeKey) || get(indexOf(homeKey)).equals(homeKey.asRange())); } @Override @@ -78,47 +75,42 @@ public Unseekables with(RoutingKey withKey) @Override public boolean participatesIn(Ranges ranges) { - if (isParticipatingHomeKey()) - return intersects(ranges); - - long ij = findNextIntersection(0, ranges, 0); - if (ij < 0) - return false; - - int i = (int)ij; - if (!get(i).contains(homeKey)) - return true; - - Invariants.checkState(get(i).equals(homeKey.asRange())); - int j = (int)(ij >>> 32); - return findNextIntersection(i + 1, ranges, j) >= 0; + return intersects(ranges); } @Override public PartialRangeRoute slice(Ranges ranges) { - return slice(ranges, Overlapping, this, homeKey, - isParticipatingHomeKey ? PartialRangeRoute::withParticipatingHomeKey - : PartialRangeRoute::withNonParticipatingHomeKey); + return slice(ranges, Overlapping); } @Override public PartialRangeRoute slice(Ranges ranges, Slice slice) { - return slice(ranges, slice, this, homeKey, - isParticipatingHomeKey ? PartialRangeRoute::withParticipatingHomeKey - : PartialRangeRoute::withNonParticipatingHomeKey); + return slice(ranges, slice, this, homeKey, (ignore, hk, rs) -> new PartialRangeRoute(hk, rs)); } @Override - public Participants participants() + public RangeRoute intersecting(Unseekables intersecting) { - if (isParticipatingHomeKey || !contains(homeKey)) - return this; + return intersecting(intersecting, Overlapping); + } - // TODO (desired): efficiency (lots of unnecessary allocations) - // TODO (expected): this should return a PartialRangeRoute, but we need to remove Route.covering() - return toRanges().subtract(Ranges.of(homeKey().asRange())); + @Override + public RangeRoute intersecting(Unseekables intersecting, Slice slice) + { + switch (intersecting.domain()) + { + default: throw new AssertionError("Unhandled domain: " + intersecting.domain()); + case Key: return intersecting((AbstractUnseekableKeys)intersecting, this, homeKey, (ignore, hk, ranges) -> new PartialRangeRoute(hk, ranges)); + case Range: return slice((Ranges)intersecting, slice, this, homeKey, (ignore, homeKey, ranges) -> new PartialRangeRoute(homeKey, ranges)); + } + } + + @Override + public Participants participants() + { + return this; } @Override @@ -129,17 +121,7 @@ public Participants participants(Ranges slice) public Participants participants(Ranges slice, Slice kind) { - Range[] ranges = slice(slice, kind, this, null, (i1, i2, rs) -> rs); - if (ranges == this.ranges && isParticipatingHomeKey) - return this; - - Ranges result = Ranges.ofSortedAndDeoverlapped(ranges); - if (isParticipatingHomeKey || !result.contains(homeKey)) - return result; - - // TODO (desired): efficiency (lots of unnecessary allocations) - // TODO (expected): this should return a PartialRangeRoute, but we need to remove Route.covering() - return result.subtract(Ranges.of(homeKey().asRange())); + return slice(slice, kind, this, null, (i1, i2, rs) -> i1.ranges == rs ? i1 : Ranges.ofSortedAndDeoverlapped(rs)); } public Ranges toRanges() @@ -153,18 +135,6 @@ public RoutingKey homeKey() return homeKey; } - @Override - public boolean isParticipatingHomeKey() - { - return isParticipatingHomeKey; - } - - @Override - public RoutingKey someParticipatingKey() - { - return isParticipatingHomeKey ? homeKey : ranges[0].someIntersectingRoutingKey(null); - } - @Override public boolean equals(Object that) { diff --git a/accord-core/src/main/java/accord/primitives/Ranges.java b/accord-core/src/main/java/accord/primitives/Ranges.java index ac293dc5b..f26022f0f 100644 --- a/accord-core/src/main/java/accord/primitives/Ranges.java +++ b/accord-core/src/main/java/accord/primitives/Ranges.java @@ -98,9 +98,26 @@ public final Ranges slice(Ranges ranges, Slice slice) return slice(ranges, slice, this, null, (i1, i2, rs) -> i1.ranges == rs ? i1 : Ranges.ofSortedAndDeoverlapped(rs)); } - public Ranges intersecting(Routables keysOrRanges) + private Ranges slice(AbstractRanges ranges, Slice slice) { - return intersecting(this, keysOrRanges, this, (i1, i2, rs) -> i2.ranges == rs ? i2 : new Ranges(rs)); + return slice(ranges, slice, this, this, (i1, i2, rs) -> i2.ranges == rs ? i2 : Ranges.ofSortedAndDeoverlapped(rs)); + } + + @Override + public Ranges intersecting(Unseekables intersecting) + { + return intersecting(intersecting, Overlapping); + } + + @Override + public final Ranges intersecting(Unseekables intersecting, Slice slice) + { + switch (intersecting.domain()) + { + default: throw new AssertionError("Unhandled domain: " + intersecting.domain()); + case Range: return slice((AbstractRanges) intersecting, slice); + case Key: return intersecting((AbstractUnseekableKeys) intersecting, this, null, (i1, i2, rs) -> i1.ranges == rs ? i1 : new Ranges(rs)); + } } @Override @@ -157,11 +174,9 @@ public Ranges toRanges() public FullRangeRoute toRoute(RoutingKey homeKey) { if (!contains(homeKey)) - { - Range[] ranges = with(Ranges.of(homeKey.asRange())).ranges; - return new FullRangeRoute(homeKey, false, ranges); - } - return new FullRangeRoute(homeKey, true, ranges); + throw new IllegalArgumentException("Home key must be contained in the route: " + homeKey + " not in " + this); + + return new FullRangeRoute(homeKey, ranges); } public Ranges union(UnionMode mode, Ranges that) diff --git a/accord-core/src/main/java/accord/primitives/Routables.java b/accord-core/src/main/java/accord/primitives/Routables.java index b37bcc91e..7a7603374 100644 --- a/accord-core/src/main/java/accord/primitives/Routables.java +++ b/accord-core/src/main/java/accord/primitives/Routables.java @@ -34,6 +34,10 @@ */ public interface Routables extends Iterable { + /** + * How to slice an input range that partially overlaps a slice range. + * This modifier applies only when both input collections (to either {@link #slice} or {@link #intersecting} contain Ranges) + */ enum Slice { /** (Default) Overlapping ranges are returned unmodified */ @@ -66,9 +70,12 @@ default boolean intersects(Routables routables) boolean contains(RoutableKey key); boolean containsAll(Routables keysOrRanges); + boolean intersectsAll(Unseekables keysOrRanges); Routables slice(Ranges ranges); Routables slice(Ranges ranges, Slice slice); + Routables intersecting(Unseekables intersecting); + Routables intersecting(Unseekables intersecting, Slice slice); /** * Search forwards from {code thisIndex} and {@code withIndex} to find the first entries in each collection diff --git a/accord-core/src/main/java/accord/primitives/Route.java b/accord-core/src/main/java/accord/primitives/Route.java index 775c2b3b3..65a26db40 100644 --- a/accord-core/src/main/java/accord/primitives/Route.java +++ b/accord-core/src/main/java/accord/primitives/Route.java @@ -22,20 +22,12 @@ import javax.annotation.Nullable; -public interface Route extends Unseekables +public interface Route extends Participants { RoutingKey homeKey(); - /** - * @return true iff homeKey() is involved in the transaction, not only in its coordination (i.e. txn.keys().contains(homeKey()) - */ - boolean isParticipatingHomeKey(); - RoutingKey someParticipatingKey(); - default boolean isRoute() { return true; } - boolean covers(Ranges ranges); - /** * Return an object containing any {@code K} present in either of the original collections, * and covering the union of the ranges. @@ -47,9 +39,13 @@ public interface Route extends Unseekables @Override PartialRoute slice(Ranges ranges); + @Override PartialRoute slice(Ranges ranges, Slice slice); - PartialRoute sliceStrict(Ranges ranges); - Ranges sliceCovering(Ranges ranges, Slice slice); + + @Override + Route intersecting(Unseekables intersecting); + @Override + Route intersecting(Unseekables intersecting, Slice slice); Route withHomeKey(); @@ -73,11 +69,6 @@ public interface Route extends Unseekables */ Participants participants(Ranges ranges, Slice slice); - default boolean hasParticipants() - { - return size() > (isParticipatingHomeKey() || !contains(homeKey()) ? 0 : 1); - } - // this method exists solely to circumvent JDK bug with testing and casting interfaces static boolean isFullRoute(@Nullable Unseekables unseekables) { return unseekables != null && unseekables.kind().isFullRoute(); } @@ -116,8 +107,8 @@ static FullRoute tryCastToFullRoute(@Nullable Unseekables unseekables) switch (unseekables.domain()) { default: return null; - case Key: return (FullKeyRoute) unseekables; - case Range: return (FullRangeRoute) unseekables; + case Key: return unseekables instanceof FullKeyRoute ? (FullKeyRoute) unseekables : null; + case Range: return unseekables instanceof FullRangeRoute ? (FullRangeRoute) unseekables : null; } } diff --git a/accord-core/src/main/java/accord/primitives/RoutingKeys.java b/accord-core/src/main/java/accord/primitives/RoutingKeys.java index 81212ef75..19147403c 100644 --- a/accord-core/src/main/java/accord/primitives/RoutingKeys.java +++ b/accord-core/src/main/java/accord/primitives/RoutingKeys.java @@ -90,7 +90,7 @@ public RoutingKeys with(RoutingKey with) if (contains(with)) return this; - return wrap(toRoutingKeysArray(with)); + return wrap(toRoutingKeysArray(with, true)); } @Override diff --git a/accord-core/src/main/java/accord/primitives/Seekables.java b/accord-core/src/main/java/accord/primitives/Seekables.java index 17a2f075c..4d8112688 100644 --- a/accord-core/src/main/java/accord/primitives/Seekables.java +++ b/accord-core/src/main/java/accord/primitives/Seekables.java @@ -31,8 +31,13 @@ public interface Seekables> extend @Override default U slice(Ranges ranges) { return slice(ranges, Overlapping); } + @Override + default U intersecting(Unseekables intersecting) { return intersecting(intersecting, Overlapping); } + @Override U slice(Ranges ranges, Slice slice); + U intersecting(Unseekables intersecting, Slice slice); + Seekables subtract(Ranges ranges); Seekables subtract(U without); Seekables with(U with); diff --git a/accord-core/src/main/java/accord/primitives/SyncPoint.java b/accord-core/src/main/java/accord/primitives/SyncPoint.java index a9abc19f3..825234ed7 100644 --- a/accord-core/src/main/java/accord/primitives/SyncPoint.java +++ b/accord-core/src/main/java/accord/primitives/SyncPoint.java @@ -65,16 +65,15 @@ private SyncPoint(TxnId syncId, Deps waitFor, S keysOrRanges, RoutingKey homeKey this.homeKey = homeKey; } - public FullRoute route() + // TODO (required): this is not safe to use as a "sourceEpoch", as a transaction in the dependencies may execute in a future epoch + public long sourceEpoch() { - return keysOrRanges.toRoute(homeKey); + return syncId.epoch(); } - // TODO (required): document this and its usages; make sure call-sites make sense - public long sourceEpoch() + public FullRoute route() { - TxnId maxDep = waitFor.maxTxnId(); - return TxnId.nonNullOrMax(maxDep, syncId).epoch(); + return keysOrRanges.toRoute(homeKey); } @Override diff --git a/accord-core/src/main/java/accord/primitives/Txn.java b/accord-core/src/main/java/accord/primitives/Txn.java index d10943358..2bb53efea 100644 --- a/accord-core/src/main/java/accord/primitives/Txn.java +++ b/accord-core/src/main/java/accord/primitives/Txn.java @@ -306,9 +306,20 @@ public InMemory(@Nonnull Kind kind, @Nonnull Seekables keys, @Nonnull Read public PartialTxn slice(Ranges ranges, boolean includeQuery) { return new PartialTxn.InMemory( - ranges, kind(), keys().slice(ranges), - read().slice(ranges), includeQuery ? query() : null, - update() == null ? null : update().slice(ranges) + kind(), keys().slice(ranges), + read().slice(ranges), includeQuery ? query() : null, + update() == null ? null : update().slice(ranges) + ); + } + + @Nonnull + @Override + public PartialTxn intersecting(Participants participants, boolean includeQuery) + { + return new PartialTxn.InMemory( + kind(), keys().intersecting(participants), + read().intersecting(participants), includeQuery ? query() : null, + update() == null ? null : update().intersecting(participants) ); } @@ -374,6 +385,7 @@ public String toString() @Nullable Update update(); @Nonnull PartialTxn slice(Ranges ranges, boolean includeQuery); + @Nonnull PartialTxn intersecting(Participants participants, boolean includeQuery); default boolean isWrite() { diff --git a/accord-core/src/main/java/accord/primitives/TxnId.java b/accord-core/src/main/java/accord/primitives/TxnId.java index 5647ae9c1..63824421c 100644 --- a/accord-core/src/main/java/accord/primitives/TxnId.java +++ b/accord-core/src/main/java/accord/primitives/TxnId.java @@ -166,7 +166,7 @@ public static TxnId minForEpoch(long epoch) return new TxnId(epochMsb(epoch), 0, Id.NONE); } - private static final Pattern PARSE = Pattern.compile("\\[(?[0-9]+),(?[0-9]+),(?[0-9]+)\\([KREWNSXL]\\),(?[0-9]+)]"); + private static final Pattern PARSE = Pattern.compile("\\[(?[0-9]+),(?[0-9]+),(?[0-9]+)\\([KR][REWSXL]\\),(?[0-9]+)]"); public static TxnId parse(String txnIdString) { Matcher m = PARSE.matcher(txnIdString); diff --git a/accord-core/src/main/java/accord/primitives/Unseekable.java b/accord-core/src/main/java/accord/primitives/Unseekable.java index 2c40e95d6..e5f8e3035 100644 --- a/accord-core/src/main/java/accord/primitives/Unseekable.java +++ b/accord-core/src/main/java/accord/primitives/Unseekable.java @@ -23,4 +23,5 @@ */ public interface Unseekable extends Routable { + Range asRange(); } diff --git a/accord-core/src/main/java/accord/primitives/Unseekables.java b/accord-core/src/main/java/accord/primitives/Unseekables.java index 51b80e7d4..f59cc1985 100644 --- a/accord-core/src/main/java/accord/primitives/Unseekables.java +++ b/accord-core/src/main/java/accord/primitives/Unseekables.java @@ -41,14 +41,17 @@ public boolean isFullRoute() } @Override - Unseekables slice(Ranges ranges); + Unseekables intersecting(Unseekables intersecting); + @Override + Unseekables intersecting(Unseekables intersecting, Slice slice); + @Override + Unseekables slice(Ranges ranges); @Override Unseekables slice(Ranges ranges, Slice slice); Unseekables subtract(Ranges ranges); Unseekables subtract(Unseekables subtract); - Unseekables intersect(Unseekables intersect); /** * Return an object containing any {@code K} present in either of the original collections. diff --git a/accord-core/src/main/java/accord/topology/Shard.java b/accord-core/src/main/java/accord/topology/Shard.java index 28c9866a6..8cb8edbf9 100644 --- a/accord-core/src/main/java/accord/topology/Shard.java +++ b/accord-core/src/main/java/accord/topology/Shard.java @@ -26,9 +26,9 @@ import accord.primitives.Range; import accord.primitives.RoutableKey; import accord.utils.SortedArrays.ExtendedSortedArrayList; +import accord.utils.SortedArrays.SortedArrayList; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; @@ -38,7 +38,7 @@ public class Shard { public final Range range; - public final List nodes; + public final SortedArrayList nodes; public final ExtendedSortedArrayList sortedNodes; public final Set fastPathElectorate; public final Set joining; @@ -47,10 +47,10 @@ public class Shard public final int fastPathQuorumSize; public final int slowPathQuorumSize; - public Shard(Range range, List nodes, Set fastPathElectorate, Set joining) + public Shard(Range range, SortedArrayList nodes, Set fastPathElectorate, Set joining) { this.range = range; - this.nodes = ImmutableList.copyOf(nodes); + this.nodes = nodes; this.sortedNodes = ExtendedSortedArrayList.sortedCopyOf(nodes, Id[]::new); this.maxFailures = maxToleratedFailures(nodes.size()); this.fastPathElectorate = ImmutableSet.copyOf(fastPathElectorate); @@ -62,7 +62,7 @@ public Shard(Range range, List nodes, Set fastPathElectorate, Set jo this.fastPathQuorumSize = fastPathQuorumSize(nodes.size(), e, maxFailures); } - public Shard(Range range, List nodes, Set fastPathElectorate) + public Shard(Range range, SortedArrayList nodes, Set fastPathElectorate) { this(range, nodes, fastPathElectorate, Collections.emptySet()); } diff --git a/accord-core/src/main/java/accord/topology/Topologies.java b/accord-core/src/main/java/accord/topology/Topologies.java index ee4aa7200..d3b1576e3 100644 --- a/accord-core/src/main/java/accord/topology/Topologies.java +++ b/accord-core/src/main/java/accord/topology/Topologies.java @@ -22,9 +22,13 @@ import accord.local.Node; import accord.local.Node.Id; import accord.primitives.Ranges; +import accord.utils.ArrayBuffers; +import accord.utils.ArrayBuffers.RecursiveObjectBuffers; import accord.utils.IndexedConsumer; -import com.google.common.collect.Sets; import accord.utils.Invariants; +import accord.utils.SortedArrays; +import accord.utils.SortedArrays.SortedArrayList; +import accord.utils.SortedList; import java.util.*; @@ -62,9 +66,8 @@ default boolean isEmpty() boolean contains(Id to); - Set nodes(); - - Set copyOfNodes(); + // note this can be expensive to evaluate + SortedList nodes(); int estimateUniqueNodes(); @@ -201,17 +204,11 @@ public boolean contains(Id to) } @Override - public Set nodes() + public SortedList nodes() { return topology.nodes(); } - @Override - public Set copyOfNodes() - { - return new HashSet<>(nodes()); - } - @Override public int estimateUniqueNodes() { @@ -362,18 +359,34 @@ public int estimateUniqueNodes() } @Override - public Set nodes() + public SortedList nodes() { - Set result = Sets.newLinkedHashSetWithExpectedSize(estimateUniqueNodes()); - for (int i=0,mi=size(); i copyOfNodes() - { - return nodes(); + if (topologies.isEmpty()) + return new SortedArrayList<>(new Id[0]); + + RecursiveObjectBuffers merging = new RecursiveObjectBuffers<>(ArrayBuffers.cachedAny()); + + int bufferSize = topologies.get(0).nodes().size(); + Object[] buffer = merging.get(bufferSize); + buffer = topologies.get(0).nodes().toArray(buffer); + + for (int i = 1; i < topologies.size() ; ++i) + { + Topology topology = topologies.get(i); + Node.Id[] input = topology.nodes().backingArrayUnsafe(); + + buffer = SortedArrays.linearUnion(buffer, 0, bufferSize, input, 0, input.length, (a, b) -> ((Id)a).compareTo((Id)b), merging); + bufferSize = merging.sizeOfLast(buffer); + } + + Id[] array = new Id[bufferSize]; + System.arraycopy(buffer, 0, array, 0, bufferSize); + SortedArrayList result = new SortedArrayList<>(array); + merging.discardBuffers(); + return result; } @Override diff --git a/accord-core/src/main/java/accord/topology/Topology.java b/accord-core/src/main/java/accord/topology/Topology.java index 99336ec2f..a64c84863 100644 --- a/accord-core/src/main/java/accord/topology/Topology.java +++ b/accord-core/src/main/java/accord/topology/Topology.java @@ -22,13 +22,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.stream.IntStream; @@ -47,7 +44,10 @@ import accord.utils.IndexedConsumer; import accord.utils.IndexedIntFunction; import accord.utils.IndexedTriFunction; +import accord.utils.SimpleBitSet; +import accord.utils.SortedArrays.SortedArrayList; import accord.utils.Utils; +import org.agrona.collections.Int2ObjectHashMap; import org.agrona.collections.IntArrayList; import javax.annotation.Nullable; @@ -60,14 +60,14 @@ public class Topology { public static final long EMPTY_EPOCH = 0; private static final int[] EMPTY_SUBSET = new int[0]; - public static final Topology EMPTY = new Topology(null, EMPTY_EPOCH, new Shard[0], Ranges.EMPTY, Collections.emptyMap(), Ranges.EMPTY, EMPTY_SUBSET); + public static final Topology EMPTY = new Topology(null, EMPTY_EPOCH, new Shard[0], Ranges.EMPTY, new SortedArrayList<>(new Id[0]), new Int2ObjectHashMap<>(0, 0.9f), Ranges.EMPTY, EMPTY_SUBSET); final long epoch; final Shard[] shards; final Ranges ranges; - /** - * TODO (desired, efficiency): do not recompute nodeLookup for sub-topologies - */ - final Map nodeLookup; + + final SortedArrayList nodeIds; + final Int2ObjectHashMap nodeLookup; + /** * This array is used to permit cheaper sharing of Topology objects between requests, as we must only specify * the indexes within the parent Topology that we contain. This also permits us to perform efficient merges with @@ -124,29 +124,35 @@ public Topology(long epoch, Shard... shards) this.shards = shards; this.subsetOfRanges = ranges; this.supersetIndexes = IntStream.range(0, shards.length).toArray(); - this.nodeLookup = new LinkedHashMap<>(); + this.nodeLookup = new Int2ObjectHashMap<>(); Map build = new HashMap<>(); for (int i = 0 ; i < shards.length ; ++i) { for (Id node : shards[i].nodes) build.computeIfAbsent(node, ignore -> new IntArrayList()).add(i); } + Id[] nodeIds = new Id[build.size()]; + int count = 0; for (Map.Entry e : build.entrySet()) { int[] supersetIndexes = e.getValue().toIntArray(); Ranges ranges = this.ranges.select(supersetIndexes); - nodeLookup.put(e.getKey(), new NodeInfo(ranges, supersetIndexes)); + nodeLookup.put(e.getKey().id, new NodeInfo(ranges, supersetIndexes)); + nodeIds[count++] = e.getKey(); } + Arrays.sort(nodeIds); + this.nodeIds = new SortedArrayList<>(nodeIds); } @VisibleForTesting - Topology(@Nullable Topology global, long epoch, Shard[] shards, Ranges ranges, Map nodeLookup, Ranges subsetOfRanges, int[] supersetIndexes) + Topology(@Nullable Topology global, long epoch, Shard[] shards, Ranges ranges, SortedArrayList nodeIds, Int2ObjectHashMap nodeById, Ranges subsetOfRanges, int[] supersetIndexes) { this.global = global; this.epoch = epoch; this.shards = shards; this.ranges = ranges; - this.nodeLookup = nodeLookup; + this.nodeIds = nodeIds; + this.nodeLookup = nodeById; this.subsetOfRanges = subsetOfRanges; this.supersetIndexes = supersetIndexes; } @@ -195,6 +201,11 @@ private static Topology select(long epoch, Shard[] shards, int[] indexes) return new Topology(epoch, subset); } + public boolean isEmpty() + { + return supersetIndexes.length == 0; + } + public boolean isSubset() { return supersetIndexes.length < shards.length; @@ -207,13 +218,12 @@ public long epoch() public Topology forNode(Id node) { - NodeInfo info = nodeLookup.get(node); + NodeInfo info = nodeLookup.get(node.id); if (info == null) return Topology.EMPTY; - Map lookup = new LinkedHashMap<>(); - lookup.put(node, info); - return new Topology(global(), epoch, shards, ranges, lookup, info.ranges, info.supersetIndexes); + SortedArrayList nodeIds = new SortedArrayList<>(new Id[] { node }); + return new Topology(global(), epoch, shards, ranges, nodeIds, nodeLookup, info.ranges, info.supersetIndexes); } public Topology trim() @@ -223,7 +233,7 @@ public Topology trim() public Ranges rangesForNode(Id node) { - NodeInfo info = nodeLookup.get(node); + NodeInfo info = nodeLookup.get(node.id); return info != null ? info.ranges : Ranges.EMPTY; } @@ -249,44 +259,53 @@ public Topology withEmptySubset() public Topology forSelection(Unseekables select) { - return forSubset(subsetFor(select)); + return forSelection(select, false); } - public Topology forSelection(Unseekables select, Collection nodes) + public Topology forSelection(Unseekables select, boolean permitMissing) { - return forSubset(subsetFor(select), nodes); + return forSubset(subsetFor(select, permitMissing)); } @VisibleForTesting Topology forSubset(int[] newSubset) { Ranges rangeSubset = ranges.select(newSubset); - - Map nodeLookup = new LinkedHashMap<>(); + SimpleBitSet nodes = new SimpleBitSet(nodeIds.size()); for (int shardIndex : newSubset) { Shard shard = shards[shardIndex]; for (Id id : shard.nodes) - nodeLookup.putIfAbsent(id, this.nodeLookup.get(id).forSubset(newSubset)); + { + nodes.set(nodeIds.find(id)); + // TODO (expected): do we need to shrink to the subset? I don't think we do anymore, and if not we can avoid copying the nodeLookup entirely + nodeLookup.putIfAbsent(id.id, this.nodeLookup.get(id.id).forSubset(newSubset)); + } } - return new Topology(global(), epoch, shards, ranges, nodeLookup, rangeSubset, newSubset); + Id[] nodeIds = new Id[nodes.getSetBitCount()]; + int count = 0; + for (int i = nodes.firstSetBit() ; i >= 0 ; i = nodes.nextSetBit(i + 1, -1)) + nodeIds[count++] = this.nodeIds.get(i); + return new Topology(global(), epoch, shards, ranges, new SortedArrayList<>(nodeIds), nodeLookup, rangeSubset, newSubset); } @VisibleForTesting Topology forSubset(int[] newSubset, Collection nodes) { Ranges rangeSubset = ranges.select(newSubset); - Map nodeLookup = new HashMap<>(); + Id[] nodeIds = nodes.toArray(new Id[nodes.size()]); + Arrays.sort(nodeIds); + Int2ObjectHashMap nodeLookup = new Int2ObjectHashMap<>(nodes.size(), 0.8f); for (Id id : nodes) { - NodeInfo info = this.nodeLookup.get(id).forSubset(newSubset); + NodeInfo info = this.nodeLookup.get(id.id).forSubset(newSubset); if (info.ranges.isEmpty()) continue; - nodeLookup.put(id, info); + nodeLookup.put(id.id, info); } - return new Topology(global(), epoch, shards, ranges, nodeLookup, rangeSubset, newSubset); + return new Topology(global(), epoch, shards, ranges, new SortedArrayList<>(nodeIds), nodeLookup, rangeSubset, newSubset); } - private int[] subsetFor(Unseekables select) + private int[] subsetFor(Unseekables select, boolean permitMissing) { int count = 0; IntBuffers cachedInts = ArrayBuffers.cachedInts(); @@ -307,13 +326,13 @@ private int[] subsetFor(Unseekables select) long abi = as.findNextIntersection(ai, bs, bi); if (abi < 0) { - if (ailim < as.size()) + if (ailim < as.size() && !permitMissing) throw new IllegalArgumentException("Range not found for " + as.get(ailim)); break; } ai = (int)abi; - if (ailim < ai) + if (ailim < ai && !permitMissing) throw new IllegalArgumentException("Range not found for " + as.get(ailim)); bi = (int)(abi >>> 32); @@ -353,7 +372,7 @@ private int[] subsetFor(Unseekables select) public void visitNodeForKeysOnceOrMore(Unseekables select, Consumer nodes) { - for (int shardIndex : subsetFor(select)) + for (int shardIndex : subsetFor(select, false)) { Shard shard = shards[shardIndex]; for (Id id : shard.nodes) @@ -385,7 +404,7 @@ public T foldl(Unseekables select, IndexedBiFunction functio public void forEachOn(Id on, IndexedConsumer consumer) { - NodeInfo info = nodeLookup.get(on); + NodeInfo info = nodeLookup.get(on.id); if (info == null) return; int[] a = supersetIndexes, b = info.supersetIndexes; @@ -412,7 +431,7 @@ else if (a[ai] < b[bi]) public O mapReduceOn(Id on, int offset, IndexedTriFunction function, P1 p1, P2 p2, P3 p3, BiFunction reduce, O initialValue) { - NodeInfo info = nodeLookup.get(on); + NodeInfo info = nodeLookup.get(on.id); if (info == null) return initialValue; int[] a = supersetIndexes, b = info.supersetIndexes; @@ -442,7 +461,7 @@ else if (a[ai] < b[bi]) public

int foldlIntOn(Id on, IndexedIntFunction

consumer, P param, int offset, int initialValue, int terminalValue) { // TODO (low priority, efficiency/clarity): use findNextIntersection? - NodeInfo info = nodeLookup.get(on); + NodeInfo info = nodeLookup.get(on.id); if (info == null) return initialValue; int[] a = supersetIndexes, b = info.supersetIndexes; @@ -496,7 +515,7 @@ public Shard get(int index) public boolean contains(Id id) { - return nodeLookup.containsKey(id); + return nodeLookup.containsKey(id.id); } public List shards() @@ -523,9 +542,9 @@ public void forEach(Consumer forEach) forEach.accept(shards[i]); } - public Set nodes() + public SortedArrayList nodes() { - return nodeLookup.keySet(); + return nodeIds; } public Ranges ranges() diff --git a/accord-core/src/main/java/accord/topology/TopologyManager.java b/accord-core/src/main/java/accord/topology/TopologyManager.java index d2d09afb8..66db3529b 100644 --- a/accord-core/src/main/java/accord/topology/TopologyManager.java +++ b/accord-core/src/main/java/accord/topology/TopologyManager.java @@ -81,10 +81,10 @@ static class EpochState private final BitSet curShardSyncComplete; private final Ranges addedRanges, removedRanges; private EpochReady ready; - private Ranges curSyncComplete, prevSyncComplete, syncComplete; - Ranges closed = Ranges.EMPTY, complete = Ranges.EMPTY; + private Ranges curSynced, prevSynced, synced; + private Ranges closed = Ranges.EMPTY, complete = Ranges.EMPTY; - EpochState(Id node, Topology global, TopologySorter sorter, Ranges prevRanges, Ranges prevSyncComplete) + EpochState(Id node, Topology global, TopologySorter sorter, Ranges prevRanges, Ranges prevSynced) { this.self = node; this.global = checkArgument(global, !global.isSubset()); @@ -98,24 +98,24 @@ static class EpochState this.addedRanges = global.ranges.subtract(prevRanges).mergeTouching(); this.removedRanges = prevRanges.mergeTouching().subtract(global.ranges); - this.prevSyncComplete = addedRanges.union(MERGE_ADJACENT, prevSyncComplete.subtract(removedRanges)); - this.curSyncComplete = this.syncComplete = addedRanges; + this.prevSynced = addedRanges.union(MERGE_ADJACENT, prevSynced.subtract(removedRanges)); + this.curSynced = this.synced = addedRanges; } boolean markPrevSynced(Ranges newPrevSyncComplete) { newPrevSyncComplete = newPrevSyncComplete.union(MERGE_ADJACENT, addedRanges).subtract(removedRanges); - if (prevSyncComplete.containsAll(newPrevSyncComplete)) + if (prevSynced.containsAll(newPrevSyncComplete)) return false; - Invariants.checkState(newPrevSyncComplete.containsAll(prevSyncComplete), "Expected %s to contain all ranges in %s; but did not", newPrevSyncComplete, prevSyncComplete); - prevSyncComplete = newPrevSyncComplete; - syncComplete = curSyncComplete.slice(newPrevSyncComplete, Minimal).union(MERGE_ADJACENT, addedRanges); + Invariants.checkState(newPrevSyncComplete.containsAll(prevSynced), "Expected %s to contain all ranges in %s; but did not", newPrevSyncComplete, prevSynced); + prevSynced = newPrevSyncComplete; + synced = curSynced.slice(newPrevSyncComplete, Minimal).union(MERGE_ADJACENT, addedRanges); return true; } public boolean hasReachedQuorum() { - return syncTracker == null ? true : syncTracker.hasReachedQuorum(); + return syncTracker == null || syncTracker.hasReachedQuorum(); } public boolean recordSyncComplete(Id node) @@ -125,8 +125,8 @@ public boolean recordSyncComplete(Id node) if (syncTracker.recordSuccess(node) == Success) { - curSyncComplete = global.ranges.mergeTouching(); - syncComplete = prevSyncComplete; + curSynced = global.ranges.mergeTouching(); + synced = prevSynced; return true; } else @@ -137,8 +137,8 @@ public boolean recordSyncComplete(Id node) { if (syncTracker.get(i).hasReachedQuorum() && !curShardSyncComplete.get(i)) { - curSyncComplete = curSyncComplete.union(MERGE_ADJACENT, Ranges.of(global.shards[i].range)); - syncComplete = curSyncComplete.slice(prevSyncComplete, Minimal); + curSynced = curSynced.union(MERGE_ADJACENT, Ranges.of(global.shards[i].range)); + synced = curSynced.slice(prevSynced, Minimal); curShardSyncComplete.set(i); updated = true; } @@ -181,7 +181,7 @@ long epoch() boolean syncComplete() { - return syncComplete.containsAll(global.ranges); + return synced.containsAll(global.ranges); } /** @@ -189,7 +189,7 @@ boolean syncComplete() */ boolean syncCompleteFor(Unseekables intersect) { - return syncComplete.containsAll(intersect); + return synced.containsAll(intersect); } @Override @@ -294,7 +294,7 @@ public void syncComplete(Id node, long epoch) if (i < 0 || !epochs[i].recordSyncComplete(node)) return; - while (--i >= 0 && epochs[i].markPrevSynced(epochs[i + 1].syncComplete)) {} + while (--i >= 0 && epochs[i].markPrevSynced(epochs[i + 1].synced)) {} } } @@ -398,7 +398,7 @@ public synchronized EpochReady onTopologyUpdate(Topology topology, Supplier removed) @VisibleForTesting public Ranges syncComplete(long epoch) { - return epochs.get(epoch).syncComplete; + return epochs.get(epoch).synced; } public synchronized void truncateTopologyUntil(long epoch) @@ -530,6 +530,7 @@ public Topologies preciseEpochs(long epoch) return new Single(sorter, epochs.get(epoch).global); } + // TODO (required): test all of these methods when asking for epochs that have been cleaned up (and other code paths) public Topologies withUnsyncedEpochs(Unseekables select, Timestamp min, Timestamp max) { return withUnsyncedEpochs(select, min.epoch(), max.epoch()); @@ -538,15 +539,20 @@ public Topologies withUnsyncedEpochs(Unseekables select, Timestamp min, Times public Topologies withUnsyncedEpochs(Unseekables select, long minEpoch, long maxEpoch) { Invariants.checkArgument(minEpoch <= maxEpoch, "min epoch %d > max %d", minEpoch, maxEpoch); - return withSufficientEpochs(select, minEpoch, maxEpoch, epochState -> epochState.syncComplete); + return withSufficientEpochsAtLeast(select, minEpoch, maxEpoch, epochState -> epochState.synced); } - public Topologies withOpenEpochs(Unseekables select, EpochSupplier min, EpochSupplier max) + public Topologies withOpenEpochs(Unseekables select, @Nullable EpochSupplier min, EpochSupplier max) { - return withSufficientEpochs(select, min.epoch(), max.epoch(), epochState -> epochState.closed); + return withSufficientEpochsAtMost(select, min == null ? Long.MIN_VALUE : min.epoch(), max.epoch(), epochState -> epochState.global.ranges); } - private Topologies withSufficientEpochs(Unseekables select, long minEpoch, long maxEpoch, Function isSufficientFor) + public Topologies withUncompletedEpochs(Unseekables select, @Nullable EpochSupplier min, EpochSupplier max) + { + return withSufficientEpochsAtMost(select, min == null ? Long.MIN_VALUE : min.epoch(), max.epoch(), epochState -> epochState.complete); + } + + private Topologies withSufficientEpochsAtLeast(Unseekables select, long minEpoch, long maxEpoch, Function isSufficientFor) { Invariants.checkArgument(minEpoch <= maxEpoch); Epochs snapshot = epochs; @@ -560,7 +566,7 @@ private Topologies withSufficientEpochs(Unseekables select, long minEpoch, lo EpochState maxEpochState = nonNull(snapshot.get(maxEpoch)); if (minEpoch == maxEpoch && isSufficientFor.apply(maxEpochState).containsAll(select)) - return new Single(sorter, maxEpochState.global.forSelection(select)); + return new Single(sorter, maxEpochState.global.forSelection(select, false)); int i = (int)(snapshot.currentEpoch - maxEpoch); int maxi = (int)(Math.min(1 + snapshot.currentEpoch - minEpoch, snapshot.epochs.length)); @@ -570,10 +576,10 @@ private Topologies withSufficientEpochs(Unseekables select, long minEpoch, lo // An issue was found where a range was removed from a replica and min selection picked the epoch before that, // which caused a node to get included in the txn that actually lost the range // See CASSANDRA-18804 - while (i < maxi) + while (i < maxi && !select.isEmpty()) { EpochState epochState = snapshot.epochs[i++]; - topologies.add(epochState.global.forSelection(select)); + topologies.add(epochState.global.forSelection(select, false)); select = select.subtract(epochState.addedRanges); } @@ -603,7 +609,7 @@ private Topologies withSufficientEpochs(Unseekables select, long minEpoch, lo return topologies.build(sorter); EpochState next = snapshot.epochs[i++]; - topologies.add(next.global.forSelection(select)); + topologies.add(next.global.forSelection(select, false)); prev = next; } while (i < snapshot.epochs.length); // needd to remove sufficent / added else remaining may not be empty when the final matches are the last epoch @@ -615,6 +621,40 @@ private Topologies withSufficientEpochs(Unseekables select, long minEpoch, lo return topologies.build(sorter); } + private Topologies withSufficientEpochsAtMost(Unseekables select, long minEpoch, long maxEpoch, Function isSufficientFor) + { + Invariants.checkArgument(minEpoch <= maxEpoch); + Epochs snapshot = epochs; + + minEpoch = Math.max(snapshot.minEpoch(), minEpoch); + if (maxEpoch == Long.MAX_VALUE) maxEpoch = snapshot.currentEpoch; + else + { + Invariants.checkState(snapshot.currentEpoch >= maxEpoch, "current epoch %d < provided max %d", snapshot.currentEpoch, maxEpoch); + Invariants.checkState(snapshot.minEpoch() <= maxEpoch, "minimum known epoch %d > provided max %d", snapshot.minEpoch(), maxEpoch); + } + + EpochState maxEpochState = nonNull(snapshot.get(maxEpoch)); + if (minEpoch == maxEpoch && isSufficientFor.apply(maxEpochState).containsAll(select)) + return new Single(sorter, maxEpochState.global.forSelection(select, true)); + + int i = (int)(snapshot.currentEpoch - maxEpoch); + int maxi = (int)(Math.min(1 + snapshot.currentEpoch - minEpoch, snapshot.epochs.length)); + Topologies.Builder topologies = new Topologies.Builder(maxi - i); + + while (i < maxi && !select.isEmpty()) + { + EpochState epochState = snapshot.epochs[i++]; + Topology topology = epochState.global.forSelection(select, true); + if (!topology.isEmpty()) + topologies.add(topology); + select = select.subtract(epochState.addedRanges); + select = select.subtract(isSufficientFor.apply(epochState)); + } + + return topologies.build(sorter); + } + public Topologies preciseEpochs(Unseekables select, long minEpoch, long maxEpoch) { Epochs snapshot = epochs; @@ -636,7 +676,6 @@ public Topologies preciseEpochs(Unseekables select, long minEpoch, long maxEp topologies.add(epochState.global.forSelection(select)); select = select.subtract(epochState.addedRanges); } - Invariants.checkState(!topologies.isEmpty(), "Unable to find an epoch that contained %s", select); return topologies.build(sorter); } diff --git a/accord-core/src/main/java/accord/utils/ArrayBuffers.java b/accord-core/src/main/java/accord/utils/ArrayBuffers.java index 2e66d2e57..388c5e2cf 100644 --- a/accord-core/src/main/java/accord/utils/ArrayBuffers.java +++ b/accord-core/src/main/java/accord/utils/ArrayBuffers.java @@ -21,7 +21,6 @@ import accord.api.Key; import accord.api.RoutingKey; import accord.primitives.Range; -import accord.primitives.Timestamp; import accord.primitives.TxnId; import java.lang.reflect.Array; @@ -58,7 +57,7 @@ public class ArrayBuffers private static final ThreadLocal> ROUTINGKEYS = ThreadLocal.withInitial(() -> new ObjectBufferCache<>(3, 1 << 9, RoutingKey[]::new)); private static final ThreadLocal> KEYRANGES = ThreadLocal.withInitial(() -> new ObjectBufferCache<>(3, 1 << 7, Range[]::new)); private static final ThreadLocal> TXN_IDS = ThreadLocal.withInitial(() -> new ObjectBufferCache<>(3, 1 << 12, TxnId[]::new)); - private static final ThreadLocal> TIMESTAMPS = ThreadLocal.withInitial(() -> new ObjectBufferCache<>(3, 1 << 12, Timestamp[]::new)); + private static final ThreadLocal> OBJECTS = ThreadLocal.withInitial(() -> new ObjectBufferCache<>(3, 1 << 12, Object[]::new)); public static IntBuffers cachedInts() { @@ -85,9 +84,9 @@ public static ObjectBuffers cachedTxnIds() return TXN_IDS.get(); } - public static ObjectBuffers cachedTimestamps() + public static ObjectBuffers cachedAny() { - return TIMESTAMPS.get(); + return OBJECTS.get(); } public static ObjectBuffers uncached(IntFunction allocator) { return new UncachedObjectBuffers<>(allocator); } @@ -611,7 +610,8 @@ public int sizeOfLast(T[] buffer) } /** - * Returns the buffer to the caller, saving the length if necessary + * Used to perform a sequence of merges over the same data, i.e. a collection of arrays + * where we merge the first with the second, then the result of that with the third and so on */ public static class RecursiveObjectBuffers implements ObjectBuffers { @@ -630,30 +630,32 @@ public T[] get(int minSize) if (alt != null) { if (alt.length >= minSize) - return alt; + { + T[] result = alt; + alt = cur; + return cur = result; + } wrapped.forceDiscard(alt, Math.min(maxBufferSize, alt.length)); } - return alt = wrapped.get(minSize); + + T[] result = wrapped.get(minSize); + alt = cur; + return cur = result; } @Override public T[] getAndCompleteExact(int size) { updateSize(size); - T[] buf = get(size); - this.alt = cur; - this.cur = buf; - return buf; + return get(size); } @Override public T[] complete(T[] buffer, int usedSize) { updateSize(usedSize); - Invariants.checkArgument(buffer == alt); - alt = cur; - cur = buffer; + Invariants.checkArgument(buffer == cur); return buffer; } diff --git a/accord-core/src/main/java/accord/utils/CheckpointIntervalArray.java b/accord-core/src/main/java/accord/utils/CheckpointIntervalArray.java index 14a7d4c59..df9ffe1a8 100644 --- a/accord-core/src/main/java/accord/utils/CheckpointIntervalArray.java +++ b/accord-core/src/main/java/accord/utils/CheckpointIntervalArray.java @@ -24,6 +24,7 @@ import net.nicoulaj.compilecommand.annotations.Inline; import static accord.utils.SortedArrays.Search.CEIL; +import static accord.utils.SortedArrays.Search.FLOOR; public class CheckpointIntervalArray { @@ -92,12 +93,12 @@ public CheckpointIntervalArray(Accessor accessor, Ranges ran } @Inline - public int forEach(Range range, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4, int minIndex) + public int forEachRange(Range range, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4, int minIndex) { - return forEach(accessor.start(range), accessor.end(range), forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); + return forEachRange(accessor.start(range), accessor.end(range), forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); } - public int forEach(Key startKey, Key endKey, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4, int minIndex) + public int forEachRange(Key startKey, Key endKey, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4, int minIndex) { if (accessor.size(ranges) == 0 || minIndex == accessor.size(ranges)) return minIndex; @@ -127,6 +128,36 @@ else if (c.compare(accessor.end(ranges, start), startKey) <= 0) return forEach(start, end, floor, startKey, 0, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); } + public int forEachKey(Key key, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, P1 p1, P2 p2, P3 p3, P4 p4, int minIndex) + { + if (accessor.size(ranges) == 0 || minIndex == accessor.size(ranges)) + return minIndex; + + var c = accessor.keyComparator(); + int end = accessor.binarySearch(ranges, minIndex, accessor.size(ranges), key, (a, b) -> c.compare(a, accessor.start(b)), FLOOR); + if (end < 0) end = -1 - end; + else ++end; + if (end <= minIndex) return minIndex; + + int floor = accessor.binarySearch(ranges, minIndex, accessor.size(ranges), key, (a, b) -> c.compare(a, accessor.start(b)), CEIL); + int start = floor; + if (floor < 0) + { + // if there's no precise match on start, step backwards; + // if this range does not overlap us, step forwards again for start + // but retain the floor index for performing scan and checkpoint searches from + // as this contains all ranges that might overlap us (whereas those that end + // after us but before the next range's start would be missed by the next range index) + start = floor = -2 - floor; + if (start < 0) + start = floor = 0; + else if (c.compare(accessor.end(ranges, start), key) <= 0) + ++start; + } + + return forEach(start, end, floor, key, 0, forEachScanOrCheckpoint, forEachRange, p1, p2, p3, p4, minIndex); + } + @Inline protected int forEach(int start, int end, int floor, Key startBound, int cmpStartBoundWithEnd, IndexedQuadConsumer forEachScanOrCheckpoint, IndexedRangeQuadConsumer forEachRange, diff --git a/accord-core/src/main/java/accord/utils/Functions.java b/accord-core/src/main/java/accord/utils/Functions.java index 2df491bc5..cab8995db 100644 --- a/accord-core/src/main/java/accord/utils/Functions.java +++ b/accord-core/src/main/java/accord/utils/Functions.java @@ -57,6 +57,22 @@ public static O mapReduceNonNull(Function map, BiFunction return result; } + public static O mapReduceNonNull(Function map, BiFunction reduce, I[] input) + { + O result = null; + for (I i : input) + { + if (i == null) continue; + + O o = map.apply(i); + if (o == null) continue; + + if (result == null) result = o; + else result = reduce.apply(result, o); + } + return result; + } + public static O foldl(List list, BiFunction foldl, O zero) { O result = zero; diff --git a/accord-core/src/main/java/accord/utils/ImmutableBitSet.java b/accord-core/src/main/java/accord/utils/ImmutableBitSet.java index 78a15d267..281c0d16e 100644 --- a/accord-core/src/main/java/accord/utils/ImmutableBitSet.java +++ b/accord-core/src/main/java/accord/utils/ImmutableBitSet.java @@ -50,6 +50,11 @@ public ImmutableBitSet(SimpleBitSet copy) super(copy); } + public ImmutableBitSet(SimpleBitSet copy, boolean share) + { + super(copy, share); + } + ImmutableBitSet(long[] bits) { super(bits); diff --git a/accord-core/src/main/java/accord/utils/Invariants.java b/accord-core/src/main/java/accord/utils/Invariants.java index 2f951d21e..d007505f7 100644 --- a/accord-core/src/main/java/accord/utils/Invariants.java +++ b/accord-core/src/main/java/accord/utils/Invariants.java @@ -41,7 +41,7 @@ public enum ParanoiaCostFactor private static final int PARANOIA_COMPUTE = Paranoia.valueOf(System.getProperty("accord.paranoia.cpu", "NONE").toUpperCase()).ordinal(); private static final int PARANOIA_MEMORY = Paranoia.valueOf(System.getProperty("accord.paranoia.memory", "NONE").toUpperCase()).ordinal(); private static final int PARANOIA_FACTOR = ParanoiaCostFactor.valueOf(System.getProperty("accord.paranoia.costfactor", "LOW").toUpperCase()).ordinal(); - private static boolean IS_PARANOID = PARANOIA_COMPUTE > 0 || PARANOIA_MEMORY > 0; + private static final boolean IS_PARANOID = PARANOIA_COMPUTE > 0 || PARANOIA_MEMORY > 0; private static final boolean DEBUG = System.getProperty("accord.debug", "false").equals("true"); public static boolean isParanoid() diff --git a/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java b/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java index 1f2c3162a..5f726c868 100644 --- a/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java +++ b/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java @@ -435,6 +435,11 @@ protected AbstractBoundariesBuilder(boolean inclusiveEnds, int capacity) this.values = new ArrayList<>(capacity + 1); } + public boolean isEmpty() + { + return values.isEmpty(); + } + /** * null is a valid value to represent no knowledge, and is the *expected* final value, representing * the bound of our knowledge (any higher key will find no associated information) @@ -443,8 +448,8 @@ public void append(K start, @Nullable V value, BiFunction reduce) { int tailIdx = starts.size() - 1; - assert starts.size() == values.size(); - assert tailIdx < 0 || start.compareTo(starts.get(tailIdx)) >= 0; + Invariants.checkState(starts.size() == values.size()); + Invariants.checkState( tailIdx < 0 || start.compareTo(starts.get(tailIdx)) >= 0); boolean sameAsTailKey, sameAsTailValue; V tailValue; diff --git a/accord-core/src/main/java/accord/utils/ReducingRangeMap.java b/accord-core/src/main/java/accord/utils/ReducingRangeMap.java index 13beb8c35..09a440175 100644 --- a/accord-core/src/main/java/accord/utils/ReducingRangeMap.java +++ b/accord-core/src/main/java/accord/utils/ReducingRangeMap.java @@ -77,6 +77,11 @@ public V2 foldl(Routables routables, TriFunction fold return foldl(routables, (a, b, f, p) -> f.apply(a, b, p), accumulator, fold, p1, terminate); } + public V2 foldlWithKey(Routables routables, IndexedTriFunction fold, V2 accumulator, P1 p1, Predicate terminate) + { + return foldl(routables, (v, v2, f, p, i, j, k) -> f.apply(v, v2, p, i), accumulator, fold, p1, terminate); + } + public V2 foldl(Routables routables, QuadFunction fold, V2 accumulator, P1 p1, P2 p2, Predicate terminate) { return foldl(routables, (v, v2, param1, param2, i, j, k) -> fold.apply(v, v2, param1, param2), accumulator, p1, p2, terminate); @@ -107,7 +112,7 @@ public V2 foldlWithDefault(Routables routables, QuadFunction fold.apply(v, v2, param1, param2), defaultValue, accumulator, p1, p2, terminate); } - private V2 foldl(Routables routables, ReduceFunction fold, V2 accumulator, P1 p1, P2 p2, Predicate terminate) + public V2 foldl(Routables routables, ReduceFunction fold, V2 accumulator, P1 p1, P2 p2, Predicate terminate) { switch (routables.domain()) { @@ -117,7 +122,7 @@ private V2 foldl(Routables routables, ReduceFunction V2 foldl(AbstractKeys keys, ReduceFunction fold, V2 accumulator, P1 p1, P2 p2, Predicate terminate) + public V2 foldl(AbstractKeys keys, ReduceFunction fold, V2 accumulator, P1 p1, P2 p2, Predicate terminate) { if (values.length == 0) return accumulator; @@ -151,7 +156,7 @@ private V2 foldl(AbstractKeys keys, ReduceFunction V2 foldl(AbstractRanges ranges, ReduceFunction fold, V2 accumulator, P1 p1, P2 p2, Predicate terminate) + public V2 foldl(AbstractRanges ranges, ReduceFunction fold, V2 accumulator, P1 p1, P2 p2, Predicate terminate) { if (values.length == 0) return accumulator; @@ -314,7 +319,7 @@ public V get(RoutableKey key) return values[idx]; } - public static ReducingRangeMap create(Ranges ranges, V value) + public static ReducingRangeMap create(AbstractRanges ranges, V value) { if (value == null) throw new IllegalArgumentException("value is null"); @@ -330,7 +335,7 @@ public static > M create(Unseekables keysOrR switch (keysOrRanges.domain()) { default: throw new AssertionError("Unhandled domain: " + keysOrRanges.domain()); - case Range: return create((Ranges) keysOrRanges, value, builder); + case Range: return create((AbstractRanges) keysOrRanges, value, builder); case Key: return create((AbstractUnseekableKeys) keysOrRanges, value, builder); } } @@ -340,12 +345,12 @@ public static > M create(Seekables keysOr switch (keysOrRanges.domain()) { default: throw new AssertionError("Unhandled domain: " + keysOrRanges.domain()); - case Range: return create((Ranges) keysOrRanges, value, builder); + case Range: return create((AbstractRanges) keysOrRanges, value, builder); case Key: return create((Keys) keysOrRanges, value, builder); } } - public static > M create(Ranges ranges, V value, BuilderFactory factory) + public static > M create(AbstractRanges ranges, V value, BuilderFactory factory) { if (value == null) throw new IllegalArgumentException("value is null"); diff --git a/accord-core/src/main/java/accord/utils/SimpleBitSet.java b/accord-core/src/main/java/accord/utils/SimpleBitSet.java index 652f3c2fc..db61c1cc0 100644 --- a/accord-core/src/main/java/accord/utils/SimpleBitSet.java +++ b/accord-core/src/main/java/accord/utils/SimpleBitSet.java @@ -60,6 +60,11 @@ public SimpleBitSet(int size, boolean set) } public SimpleBitSet(SimpleBitSet copy) + { + this(copy, false); + } + + public SimpleBitSet(SimpleBitSet copy, boolean share) { bits = copy.bits.clone(); count = copy.count; diff --git a/accord-core/src/main/java/accord/utils/SortedArrays.java b/accord-core/src/main/java/accord/utils/SortedArrays.java index 8da30df17..3b5871657 100644 --- a/accord-core/src/main/java/accord/utils/SortedArrays.java +++ b/accord-core/src/main/java/accord/utils/SortedArrays.java @@ -52,6 +52,19 @@ public SortedArrayList(T[] array) this.array = checkArgument(array, SortedArrays::isSortedUnique); } + public static > SortedArrayList copySorted(Collection copy, IntFunction allocator) + { + T[] array = copy.toArray(allocator); + return new SortedArrayList<>(array); + } + + public static > SortedArrayList copyUnsorted(Collection copy, IntFunction allocator) + { + T[] array = copy.toArray(allocator); + Arrays.sort(array); + return new SortedArrayList<>(array); + } + @Override public T get(int index) { @@ -78,7 +91,12 @@ public int find(Comparable find) public boolean containsAll(SortedArrayList test) { - return test.array.length == SortedArrays.foldlIntersection(Comparable::compareTo, array, 0, array.length, test.array, 0, test.array.length, (t, p, v, li, ri) -> v + 1, 0, 0, test.array.length); + return isSubset(Comparable::compareTo, test.array, 0, test.array.length, array, 0, array.length); + } + + public T[] backingArrayUnsafe() + { + return array; } public static class Builder> @@ -1338,22 +1356,18 @@ public static > boolean isSubset(T[] test, int t } @Inline - public static boolean isSubset(AsymmetricComparator comparator, T[] test, int testFrom, int testTo, T[] superset, int supersetFrom, int supersetTo) + public static boolean isSubset(AsymmetricComparator comparator, T1[] test, int testFrom, int testTo, T2[] superset, int supersetFrom, int supersetTo) { - while (true) + while (testFrom < testTo) { - long abi = findNextIntersection(test, testFrom, testTo, superset, supersetFrom, supersetTo, comparator); - if (abi < 0) - return true; - - int nextai = (int)(abi); - if (testFrom != nextai) + supersetFrom = SortedArrays.exponentialSearch(superset, supersetFrom, supersetTo, test[testFrom], comparator, FAST); + if (supersetFrom < 0) return false; - supersetFrom = (int)(abi >>> 32); ++testFrom; ++supersetFrom; } + return true; } public static > void assertSorted(T[] array) diff --git a/accord-core/src/main/java/accord/utils/SortedList.java b/accord-core/src/main/java/accord/utils/SortedList.java index 5ce5fe18d..e51a0e852 100644 --- a/accord-core/src/main/java/accord/utils/SortedList.java +++ b/accord-core/src/main/java/accord/utils/SortedList.java @@ -18,10 +18,49 @@ package accord.utils; +import java.util.AbstractList; import java.util.List; +import java.util.RandomAccess; +import java.util.Set; +import java.util.Spliterator; -public interface SortedList> extends List +public interface SortedList> extends List, Set, RandomAccess { int findNext(int i, Comparable find); int find(Comparable find); + + @Override + default boolean contains(Object o) + { + return o != null && find((Comparable) o) >= 0; + } + + @Override + default int indexOf(Object o) + { + return o == null ? -1 : Math.max(-1, find((Comparable) o)); + } + + @Override + default Spliterator spliterator() + { + return List.super.spliterator(); + } + + @Override + default int lastIndexOf(Object o) + { + return indexOf(o); + } + + default List select(V[] selectFrom, List select) + { + return new AbstractList<>() + { + @Override + public V get(int index) { return selectFrom[find(select.get(index))]; } + @Override + public int size() { return select.size(); } + }; + } } diff --git a/accord-core/src/main/java/accord/utils/Utils.java b/accord-core/src/main/java/accord/utils/Utils.java index 7ce33a9ba..97039109a 100644 --- a/accord-core/src/main/java/accord/utils/Utils.java +++ b/accord-core/src/main/java/accord/utils/Utils.java @@ -118,7 +118,7 @@ public static SimpleBitSet ensureMutable(SimpleBitSet set) public static ImmutableBitSet ensureImmutable(SimpleBitSet set) { if (set == null) return null; - return set instanceof ImmutableBitSet ? (ImmutableBitSet) set : new ImmutableBitSet(set); + return set instanceof ImmutableBitSet ? (ImmutableBitSet) set : new ImmutableBitSet(set, true); } public static T[] addAll(T[] first, T[] second) diff --git a/accord-core/src/test/java/accord/Utils.java b/accord-core/src/test/java/accord/Utils.java index 3cc082be8..5f531d4b8 100644 --- a/accord-core/src/test/java/accord/Utils.java +++ b/accord-core/src/test/java/accord/Utils.java @@ -20,11 +20,14 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import accord.api.Agent; import com.google.common.collect.Sets; import accord.api.Key; @@ -45,6 +48,7 @@ import accord.impl.mock.MockConfigurationService; import accord.impl.mock.MockStore; import accord.local.Node; +import accord.local.Node.Id; import accord.local.NodeTimeService; import accord.local.ShardDistributor; import accord.messages.LocalRequest; @@ -58,6 +62,7 @@ import accord.utils.DefaultRandom; import accord.utils.EpochFunction; import accord.utils.Invariants; +import accord.utils.SortedArrays.SortedArrayList; import accord.utils.ThreadPoolScheduler; import org.awaitility.Awaitility; import org.awaitility.core.ThrowingRunnable; @@ -66,14 +71,14 @@ public class Utils { - public static Node.Id id(int i) + public static Id id(int i) { - return new Node.Id(i); + return new Id(i); } - public static List ids(int num) + public static List ids(int num) { - List rlist = new ArrayList<>(num); + List rlist = new ArrayList<>(num); for (int i=0; i ids(int num) return rlist; } - public static List ids(int first, int last) + public static SortedArrayList ids(int first, int last) { Invariants.checkArgument(last >= first); - List rlist = new ArrayList<>(last - first + 1); + Id[] rlist = new Id[last - first + 1]; for (int i=first; i<=last; i++) - rlist.add(id(i)); + rlist[first - i] = id(i); - return rlist; + return new SortedArrayList<>(rlist); } - public static List idList(int... ids) + public static SortedArrayList idList(int... ids) { - List list = new ArrayList<>(ids.length); - for (int i : ids) - list.add(new Node.Id(i)); - return list; + Id[] list = new Id[ids.length]; + for (int i = 0 ; i < ids.length ; ++i) + list[i] = new Id(ids[i]); + Arrays.sort(list); + return new SortedArrayList<>(list); } - public static Set idSet(int... ids) + public static Set idSet(int... ids) { - Set set = Sets.newHashSetWithExpectedSize(ids.length); + Set set = Sets.newHashSetWithExpectedSize(ids.length); for (int i : ids) - set.add(new Node.Id(i)); + set.add(new Id(i)); return set; } @@ -121,7 +127,7 @@ public static Txn writeTxn(Ranges ranges) return new Txn.InMemory(ranges, MockStore.read(ranges), MockStore.QUERY, MockStore.update(ranges)); } - public static Txn listWriteTxn(Node.Id client, Keys keys) + public static Txn listWriteTxn(Id client, Keys keys) { ListUpdate update = new ListUpdate(Function.identity()); for (Key k : keys) @@ -136,11 +142,16 @@ public static Txn readTxn(Keys keys) return new Txn.InMemory(keys, MockStore.read(keys), MockStore.QUERY); } - public static Shard shard(Range range, List nodes, Set fastPath) + public static Shard shard(Range range, SortedArrayList nodes, Set fastPath) { return new Shard(range, nodes, fastPath); } + public static Shard shard(Range range, SortedArrayList nodes) + { + return shard(range, nodes, new TreeSet<>(nodes)); + } + public static Topology topology(long epoch, Shard... shards) { return new Topology(epoch, shards); @@ -156,7 +167,12 @@ public static Topologies topologies(Topology... topologies) return new Topologies.Multi(SizeOfIntersectionSorter.SUPPLIER, topologies); } - public static Node createNode(Node.Id nodeId, Topology topology, MessageSink messageSink, MockCluster.Clock clock) + public static Node createNode(Id nodeId, Topology topology, MessageSink messageSink, MockCluster.Clock clock) + { + return createNode(nodeId, topology, messageSink, clock, new TestAgent()); + } + + public static Node createNode(Node.Id nodeId, Topology topology, MessageSink messageSink, MockCluster.Clock clock, Agent agent) { MockStore store = new MockStore(); Scheduler scheduler = new ThreadPoolScheduler(); @@ -169,7 +185,7 @@ public static Node createNode(Node.Id nodeId, Topology topology, MessageSink mes NodeTimeService.unixWrapper(TimeUnit.MICROSECONDS, clock), () -> store, new ShardDistributor.EvenSplit(8, ignore -> new IntKey.Splitter()), - new TestAgent(), + agent, new DefaultRandom(), scheduler, SizeOfIntersectionSorter.SUPPLIER, diff --git a/accord-core/src/test/java/accord/burn/BurnTest.java b/accord-core/src/test/java/accord/burn/BurnTest.java index 294023ec0..7a93b4b6a 100644 --- a/accord-core/src/test/java/accord/burn/BurnTest.java +++ b/accord-core/src/test/java/accord/burn/BurnTest.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Queue; @@ -478,7 +479,7 @@ static void burn(RandomSource random, TopologyFactory topologyFactory, List int observedOperations = acks.get() + recovered.get() + nacks.get() + lost.get() + truncated.get(); logger.info("Received {} acks, {} recovered, {} nacks, {} lost, {} truncated ({} total) to {} operations", acks.get(), recovered.get(), nacks.get(), lost.get(), truncated.get(), observedOperations, operations); - logger.info("Message counts: {}", messageStatsMap.entrySet()); + logger.info("Message counts: {}", statsInDescOrder(messageStatsMap)); logger.info("Took {} and in logical time of {}", Duration.ofNanos(System.nanoTime() - startNanos), Duration.ofMillis(queue.nowInMillis() - startLogicalMillis)); if (clock.get() != operations * 2 || observedOperations != operations) { @@ -498,6 +499,13 @@ static void burn(RandomSource random, TopologyFactory topologyFactory, List } } + private static String statsInDescOrder(Map statsMap) + { + List stats = new ArrayList<>(statsMap.values()); + stats.sort(Comparator.comparingInt(s -> -s.count())); + return stats.toString(); + } + private static Verifier createVerifier(String prefix, int keyCount) { if (!ElleVerifier.Support.allowed()) diff --git a/accord-core/src/test/java/accord/coordinate/CoordinateSyncPointTest.java b/accord-core/src/test/java/accord/coordinate/CoordinateSyncPointTest.java new file mode 100644 index 000000000..99614fe2a --- /dev/null +++ b/accord-core/src/test/java/accord/coordinate/CoordinateSyncPointTest.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.coordinate; + +import accord.Utils; +import accord.api.MessageSink; +import accord.coordinate.tracking.AppliedTracker; +import accord.impl.IntKey; +import accord.impl.TestAgent; +import accord.impl.mock.MockCluster; +import accord.local.AgentExecutor; +import accord.local.Node; +import accord.messages.Accept; +import accord.messages.Apply; +import accord.messages.Callback; +import accord.messages.Commit; +import accord.messages.PreAccept; +import accord.messages.ReadData; +import accord.messages.Request; +import accord.messages.WaitUntilApplied; +import accord.primitives.EpochSupplier; +import accord.primitives.PartialDeps; +import accord.primitives.Range; +import accord.primitives.Ranges; +import accord.primitives.Routable; +import accord.primitives.SyncPoint; +import accord.primitives.Txn; +import accord.primitives.TxnId; +import accord.topology.Topology; +import accord.topology.TopologyUtils; +import accord.utils.SortedArrays.SortedArrayList; +import accord.utils.async.AsyncChains; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; + +import java.util.function.Function; + +class CoordinateSyncPointTest +{ + static final Node.Id N1 = new Node.Id(1); + static final Node.Id N2 = new Node.Id(2); + static final SortedArrayList ALL = new SortedArrayList(new Node.Id[] { N1, N2 }); + + @Test + void rangeMovedOffNode() + { + Topology t1 = TopologyUtils.initialTopology(ALL, Ranges.single(IntKey.range(0, 100)), ALL.size()); + Range removed = IntKey.range(10, 100); + Topology t2 = new Topology(t1.epoch() + 1, + Utils.shard(IntKey.range(0, 10), ALL), + Utils.shard(removed, new SortedArrayList<>(new Node.Id[] { N2 }))); + + Node n1 = Utils.createNode(N1, t1, happyPathMessaging(), new MockCluster.Clock(0), new TestAgent.RethrowAgent()); + n1.topology().onTopologyUpdate(t2, () -> null); + for (Node.Id node : ALL) + n1.topology().onEpochSyncComplete(node, t1.epoch()); + + awaitApplied(n1, removed); + } + + @Test + void rangeRemovedGlobally() + { + Topology t1 = TopologyUtils.initialTopology(ALL, Ranges.single(IntKey.range(0, 100)), ALL.size()); + Range removed = IntKey.range(10, 100); + Topology t2 = new Topology(t1.epoch() + 1, + Utils.shard(IntKey.range(0, 10), ALL)); + + Node n1 = Utils.createNode(N1, t1, happyPathMessaging(), new MockCluster.Clock(0), new TestAgent.RethrowAgent()); + n1.topology().onTopologyUpdate(t2, () -> null); + for (Node.Id node : ALL) + n1.topology().onEpochSyncComplete(node, t1.epoch()); + + awaitApplied(n1, removed); + + n1.onEpochRedundant(Ranges.single(removed), t2.epoch()); + awaitApplied(n1, removed); + + n1.onEpochClosed(Ranges.single(removed), t2.epoch()); + awaitApplied(n1, removed); + } + + private static SyncPoint awaitApplied(Node node, Range removed) + { + var await = CoordinateSyncPoint.exclusive(node, new TxnId(1, node.now(), Txn.Kind.ExclusiveSyncPoint, Routable.Domain.Range, node.id()), Ranges.single(removed)) + .flatMap(syncPoint -> + // the test uses an executor that runs everything right away, so this gets called outside the CommandStore + node.commandStores().forId(0).submit(() -> { + ExecuteSyncPoint.ExecuteExclusiveSyncPoint execute = new ExecuteSyncPoint.ExecuteExclusiveSyncPoint(node, syncPoint, AppliedTracker::new); + execute.start(); + return execute; + }) + ).flatMap(Function.identity()); + + return AsyncChains.getUnchecked(await); + } + + private static MessageSink happyPathMessaging() + { + MessageSink msg = Mockito.mock(MessageSink.class); + Mockito.doAnswer(args -> { + Request request = args.getArgument(1); + + if (request instanceof PreAccept) + { + PreAccept preAccept = (PreAccept) request; + onSuccess(args, new PreAccept.PreAcceptOk(preAccept.txnId, preAccept.txnId, PartialDeps.NONE)); + } + else if (request instanceof Accept) + { + onSuccess(args, new Accept.AcceptReply(PartialDeps.NONE)); + } + else if (request instanceof Commit) + { + onSuccess(args, new ReadData.ReadOk(null, null)); + } + else if (request instanceof Apply) + { + onSuccess(args, Apply.ApplyReply.Applied); + } + else if (request instanceof WaitUntilApplied) + { + onSuccess(args, new ReadData.ReadOk(null, null)); + } + else + { + throw new AssertionError("Unexpected request: " + request); + } + return null; + }).when(msg).send(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any()); + return msg; + } + + private static void onSuccess(InvocationOnMock args, T reply) + { + Node.Id to = args.getArgument(0); + AgentExecutor executor = args.getArgument(2); + Callback cb = args.getArgument(3); + executor.execute(() -> cb.onSuccess(to, reply)); + } +} \ No newline at end of file diff --git a/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java b/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java index 90951ecdf..b8846bf26 100644 --- a/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java +++ b/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java @@ -306,7 +306,7 @@ void barrierTest() throws Throwable // Apply the blockingTxn to unblock the rest for (Node n : cluster) assertEquals(ApplyOutcome.Success, getUninterruptibly(n.unsafeForKey(key).submit(blockingTxnContext, store -> { - return Commands.apply(store, store.get(blockingTxnId, homeKey), blockingTxnId, route, null, blockingTxnId, PartialDeps.builder(store.ranges().allAt(blockingTxnId.epoch())).build(), blockingTxn.slice(store.ranges().allAt(blockingTxnId.epoch()), true), blockingTxn.execute(blockingTxnId, blockingTxnId, null), blockingTxn.query().compute(blockingTxnId, blockingTxnId, keys, null, null, null)); + return Commands.apply(store, store.get(blockingTxnId, homeKey), blockingTxnId, route, null, blockingTxnId, PartialDeps.builder(route).build(), blockingTxn.slice(store.ranges().allAt(blockingTxnId.epoch()), true), blockingTxn.execute(blockingTxnId, blockingTxnId, null), blockingTxn.query().compute(blockingTxnId, blockingTxnId, keys, null, null, null)); }))); // Global sync should be unblocked syncPoint = getUninterruptibly(syncInclusiveSyncFuture); diff --git a/accord-core/src/test/java/accord/impl/AbstractConfigurationServiceTest.java b/accord-core/src/test/java/accord/impl/AbstractConfigurationServiceTest.java index f4829f0e2..59eb56b43 100644 --- a/accord-core/src/test/java/accord/impl/AbstractConfigurationServiceTest.java +++ b/accord-core/src/test/java/accord/impl/AbstractConfigurationServiceTest.java @@ -21,7 +21,6 @@ import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -29,9 +28,10 @@ import accord.api.ConfigurationService.EpochReady; import accord.primitives.Ranges; +import accord.utils.SortedArrays.SortedArrayList; import accord.utils.async.AsyncResult; import accord.utils.async.AsyncResults; -import com.google.common.collect.ImmutableList; + import com.google.common.collect.ImmutableSet; import accord.api.ConfigurationService; @@ -175,15 +175,15 @@ public void reportEpochRedundant(Ranges ranges, long epoch) private static final Id ID1 = new Id(1); private static final Id ID2 = new Id(2); private static final Id ID3 = new Id(3); - private static final List NODES = ImmutableList.of(ID1, ID2, ID3); + private static final SortedArrayList NODES = new SortedArrayList<>(new Id[] { ID1, ID2, ID3 }); private static final Range RANGE = IntKey.range(0, 100); - private static Shard shard(Range range, List nodes, Set fastPath) + private static Shard shard(Range range, SortedArrayList nodes, Set fastPath) { return new Shard(range, nodes, fastPath); } - private static Topology topology(long epoch, Range range, List nodes, Set fastPath) + private static Topology topology(long epoch, Range range, SortedArrayList nodes, Set fastPath) { return new Topology(epoch, shard(range, nodes, fastPath)); } diff --git a/accord-core/src/test/java/accord/impl/basic/Cluster.java b/accord-core/src/test/java/accord/impl/basic/Cluster.java index 0ac20cb32..f19d09ece 100644 --- a/accord-core/src/test/java/accord/impl/basic/Cluster.java +++ b/accord-core/src/test/java/accord/impl/basic/Cluster.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.EnumSet; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; @@ -38,13 +39,17 @@ import java.util.function.Function; import java.util.function.IntSupplier; import java.util.function.LongSupplier; +import java.util.function.Predicate; import java.util.function.Supplier; +import javax.annotation.Nullable; + import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import accord.api.BarrierType; +import accord.api.Key; import accord.api.MessageSink; import accord.api.Scheduler; import accord.burn.BurnTestConfigurationService; @@ -58,19 +63,26 @@ import accord.coordinate.Invalidated; import accord.coordinate.Preempted; import accord.coordinate.Timeout; +import accord.coordinate.Truncated; import accord.impl.CoordinateDurabilityScheduling; +import accord.impl.InMemoryCommandStore; import accord.impl.MessageListener; import accord.impl.PrefixedIntHashKey; import accord.impl.SimpleProgressLog; import accord.impl.SizeOfIntersectionSorter; import accord.impl.TopologyFactory; +import accord.impl.basic.DelayedCommandStores.DelayedCommandStore; import accord.impl.list.ListAgent; import accord.impl.list.ListStore; import accord.local.AgentExecutor; +import accord.local.Command; import accord.local.Node.Id; import accord.local.Node; import accord.local.NodeTimeService; +import accord.local.SaveStatus; import accord.local.ShardDistributor; +import accord.local.Status; +import accord.local.cfk.CommandsForKey; import accord.messages.Message; import accord.messages.MessageType; import accord.messages.Reply; @@ -79,8 +91,11 @@ import accord.primitives.Keys; import accord.primitives.Range; import accord.primitives.Ranges; +import accord.primitives.Routables; import accord.primitives.Seekables; import accord.primitives.Timestamp; +import accord.primitives.Txn; +import accord.primitives.TxnId; import accord.topology.Topology; import accord.topology.TopologyRandomizer; import accord.utils.Gens; @@ -109,10 +124,16 @@ public class Cluster implements Scheduler public static class Stats { + final MessageType type; int count; + public Stats(MessageType type) + { + this.type = type; + } + public int count() { return count; } - public String toString() { return Integer.toString(count); } + public String toString() { return type + ": " + count; } } public static class LinkConfig @@ -179,7 +200,7 @@ void add(Packet packet, long delay, TimeUnit unit) { MessageType type = packet.message.type(); if (type != null) - statsMap.computeIfAbsent(type, ignore -> new Stats()).count++; + statsMap.computeIfAbsent(type, Stats::new).count++; if (trace.isTraceEnabled()) trace.trace("{} {} {}", clock++, packet.message instanceof Reply ? "RPLY" : "SEND", packet); if (lookup.apply(packet.dst) == null) responseSink.accept(packet); @@ -589,7 +610,7 @@ public void doRun() if (f != null) { // ignore specific errors - if (f instanceof Invalidated || f instanceof Timeout || f instanceof Preempted || f instanceof Exhausted) + if (f instanceof Invalidated || f instanceof Timeout || f instanceof Preempted || f instanceof Exhausted || f instanceof Truncated) return; node.agent().onUncaughtException(f); } @@ -746,4 +767,143 @@ private LinkConfig defaultLinkConfig(RandomSource random, IntSupplier rf) return new LinkConfig(overrideLinks, defaultLinks); } + public static class BlockingTransaction + { + final TxnId txnId; + final Command command; + final DelayedCommandStore commandStore; + final TxnId blockedOn; + final Routables blockedVia; + + public BlockingTransaction(TxnId txnId, Command command, DelayedCommandStore commandStore, @Nullable TxnId blockedOn, @Nullable Routables blockedVia) + { + this.txnId = txnId; + this.command = command; + this.commandStore = commandStore; + this.blockedOn = blockedOn; + this.blockedVia = blockedVia; + } + + @Override + public String toString() + { + return txnId + ":" + command.saveStatus() + "@" + + commandStore.toString().replaceAll("DelayedCommandStore", "") + + (command.homeKey() != null && commandStore.unsafeRangesForEpoch().allAt(txnId.epoch()).contains(command.homeKey()) ? "(Home)" : ""); + } + } + + public List findBlockedCommitted(@Nullable Txn.Kind first, Txn.Kind ... rest) + { + return findBlocked(SaveStatus.Committed, first, rest); + } + + public List findBlocked(@Nullable SaveStatus minSaveStatus, @Nullable Txn.Kind first, Txn.Kind ... rest) + { + List result = new ArrayList<>(); + BlockingTransaction cur = findMin(SaveStatus.Committed, SaveStatus.ReadyToExecute, first, rest); + while (cur != null) + { + result.add(cur); + Command command = cur.commandStore.unsafeCommands().get(cur.txnId).value(); + if (!command.hasBeen(Status.Stable) || cur.blockedOn == null) + break; + + cur = find(cur.blockedOn, null, SaveStatus.Stable); + } + return result; + } + + public BlockingTransaction findMinUnstable() + { + return findMin(null, SaveStatus.Committed, null); + } + + public BlockingTransaction findMinUnstable(@Nullable Txn.Kind first, Txn.Kind ... rest) + { + return findMin(null, SaveStatus.Committed, first, rest); + } + + public BlockingTransaction findMin(@Nullable SaveStatus minSaveStatus, @Nullable SaveStatus maxSaveStatus, @Nullable Txn.Kind first, Txn.Kind ... rest) + { + Predicate testKind = first == null ? ignore -> true : EnumSet.of(first, rest)::contains; + return findMin(minSaveStatus, maxSaveStatus, id -> testKind.test(id.kind())); + } + + public BlockingTransaction find(TxnId txnId, @Nullable SaveStatus minSaveStatus, @Nullable SaveStatus maxSaveStatus) + { + return findMin(minSaveStatus, maxSaveStatus, txnId::equals); + } + + public BlockingTransaction findMin(@Nullable SaveStatus minSaveStatus, @Nullable SaveStatus maxSaveStatus, Predicate testTxnId) + { + return find(minSaveStatus, maxSaveStatus, testTxnId, (min, test) -> { + int c = -1; + if (min == null || (c = test.txnId.compareTo(min.txnId)) <= 0 && (c < 0 || test.command.saveStatus().compareTo(min.command.saveStatus()) < 0)) + min = test; + return min; + }, null); + } + + public List findAll(TxnId txnId) + { + return findAll(null, null, txnId::equals); + } + + public List findAll(@Nullable SaveStatus minSaveStatus, @Nullable SaveStatus maxSaveStatus, Predicate testTxnId) + { + List result = new ArrayList<>(); + find(minSaveStatus, maxSaveStatus, testTxnId, (r, c) -> { r.add(c); return r; }, result); + return result; + } + + public T find(@Nullable SaveStatus minSaveStatus, @Nullable SaveStatus maxSaveStatus, Predicate testTxnId, BiFunction fold, T accumulate) + { + for (Node.Id id : sinks.keySet()) + { + Node node = lookup.apply(id); + + DelayedCommandStores stores = (DelayedCommandStores) node.commandStores(); + for (DelayedCommandStore store : stores.unsafeStores()) + { + for (Map.Entry e : store.unsafeCommands().entrySet()) + { + Command command = e.getValue().value(); + if ((minSaveStatus == null || command.saveStatus().compareTo(minSaveStatus) >= 0) && + (maxSaveStatus == null || command.saveStatus().compareTo(maxSaveStatus) <= 0) && + testTxnId.test(command.txnId())) + { + accumulate = fold.apply(accumulate, toBlocking(command, store)); + break; + } + } + } + } + return accumulate; + } + + private BlockingTransaction toBlocking(Command command, DelayedCommandStore store) + { + Routables blockedVia = null; + TxnId blockedOn = null; + if (command.hasBeen(Status.Stable) && !command.hasBeen(Status.Truncated)) + { + Command.WaitingOn waitingOn = command.asCommitted().waitingOn(); + Key blockedOnKey = waitingOn.lastWaitingOnKey(); + if (blockedOnKey == null) + { + blockedOn = waitingOn.nextWaitingOn(); + if (blockedOn != null) + blockedVia = command.partialDeps().participants(blockedOn); + } + else + { + CommandsForKey cfk = store.unsafeCommandsForKey().get(blockedOnKey).value(); + blockedOn = cfk.nextWaitingToApply(command.txnId().kind().witnesses()); + blockedVia = Keys.of(blockedOnKey); + } + } + return new BlockingTransaction(command.txnId(), command, store, blockedOn, blockedVia); + } + } diff --git a/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java b/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java index 319330df5..8bba8c00a 100644 --- a/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java +++ b/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java @@ -18,6 +18,7 @@ package accord.impl.basic; +import java.util.ArrayList; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -296,4 +297,12 @@ public void postExecute() }); } } + + public List unsafeStores() + { + List stores = new ArrayList<>(); + for (ShardHolder holder : current().shards) + stores.add((DelayedCommandStore) holder.store); + return stores; + } } diff --git a/accord-core/src/test/java/accord/impl/basic/Journal.java b/accord-core/src/test/java/accord/impl/basic/Journal.java index 7c8c8373d..6696e2861 100644 --- a/accord-core/src/test/java/accord/impl/basic/Journal.java +++ b/accord-core/src/test/java/accord/impl/basic/Journal.java @@ -298,8 +298,8 @@ private static Command.Truncated truncated(CommonAttributes.Mutable attrs, SaveS case TruncatedApplyWithDeps: case TruncatedApply: return Command.Truncated.truncatedApply(attrs, status, executeAt, writes, result, executesAtLeast); - case ErasedOrInvalidated: - return Command.Truncated.erasedOrInvalidated(attrs.txnId(), attrs.durability(), attrs.route()); + case ErasedOrInvalidOrVestigial: + return Command.Truncated.erasedOrInvalidOrVestigial(attrs.txnId(), attrs.durability(), attrs.route()); case Erased: return Command.Truncated.erased(attrs.txnId(), attrs.durability(), attrs.route()); case Invalidated: diff --git a/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java b/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java index 9f7cc28b7..1ca71c9fe 100644 --- a/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java +++ b/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java @@ -53,7 +53,7 @@ public ListFetchCoordinator(Node node, Ranges ranges, SyncPoint syncPoint, DataS @Override protected PartialTxn rangeReadTxn(Ranges ranges) { - return new PartialTxn.InMemory(ranges, Txn.Kind.Read, ranges, new ListRead(Function.identity(), false, ranges, ranges), new ListQuery(Node.Id.NONE, Long.MIN_VALUE, false), null); + return new PartialTxn.InMemory(Txn.Kind.Read, ranges, new ListRead(Function.identity(), false, ranges, ranges), new ListQuery(Node.Id.NONE, Long.MIN_VALUE, false), null); } @Override diff --git a/accord-core/src/test/java/accord/impl/list/ListRead.java b/accord-core/src/test/java/accord/impl/list/ListRead.java index c9215e41f..eb7657672 100644 --- a/accord-core/src/test/java/accord/impl/list/ListRead.java +++ b/accord-core/src/test/java/accord/impl/list/ListRead.java @@ -23,6 +23,7 @@ import java.util.function.Function; import accord.local.SafeCommandStore; +import accord.primitives.Participants; import accord.primitives.Ranges; import accord.primitives.Timestamp; import accord.utils.async.AsyncChain; @@ -98,6 +99,12 @@ public Read slice(Ranges ranges) return new ListRead(executor, isEphemeralRead, userReadKeys.slice(ranges), keys.slice(ranges)); } + @Override + public Read intersecting(Participants participants) + { + return new ListRead(executor, isEphemeralRead, userReadKeys.intersecting(participants), keys.intersecting(participants)); + } + @Override public Read merge(Read other) { diff --git a/accord-core/src/test/java/accord/impl/list/ListRequest.java b/accord-core/src/test/java/accord/impl/list/ListRequest.java index 560cf39a0..64d8d9642 100644 --- a/accord-core/src/test/java/accord/impl/list/ListRequest.java +++ b/accord-core/src/test/java/accord/impl/list/ListRequest.java @@ -108,7 +108,8 @@ protected Action checkSufficient(Id from, CheckStatusOk ok) @Override protected void onDone(CheckShards.Success done, Throwable failure) { - if (failure != null) callback.accept(null, failure); + if (failure instanceof Exhausted) callback.accept(Outcome.Lost, null); + else if (failure != null) callback.accept(null, failure); else if (merged.maxKnowledgeSaveStatus.is(Status.Invalidated)) callback.accept(Outcome.Invalidated, null); else if (merged.maxKnowledgeSaveStatus.is(Status.Truncated)) callback.accept(Outcome.Truncated, null); else if (merged.maxKnowledgeSaveStatus.hasBeen(Applied)) callback.accept(new Outcome(Outcome.Kind.Applied, (ListResult) ((CheckStatus.CheckStatusOkFull) merged).result), null); @@ -189,7 +190,7 @@ else if (success != null) private void checkOnResult(@Nullable RoutingKey homeKey, TxnId txnId, int attempt, Throwable t) { if (homeKey == null) - homeKey = node.computeRoute(txnId, txn.keys()).someParticipatingKey(); + homeKey = node.computeRoute(txnId, txn.keys()).homeKey(); RoutingKey finalHomeKey = homeKey; node.commandStores().select(homeKey).execute(() -> CheckOnResult.checkOnResult(node, txnId, finalHomeKey, (s, f) -> { if (f != null) diff --git a/accord-core/src/test/java/accord/impl/list/ListStore.java b/accord-core/src/test/java/accord/impl/list/ListStore.java index 1a5865a3a..c08130731 100644 --- a/accord-core/src/test/java/accord/impl/list/ListStore.java +++ b/accord-core/src/test/java/accord/impl/list/ListStore.java @@ -31,20 +31,17 @@ import accord.api.DataStore; import accord.api.Key; import accord.coordinate.CoordinateSyncPoint; +import accord.coordinate.ExecuteSyncPoint; import accord.coordinate.ExecuteSyncPoint.SyncPointErased; +import accord.coordinate.Exhausted; import accord.coordinate.Invalidated; import accord.coordinate.Preempted; import accord.coordinate.Timeout; import accord.coordinate.TopologyMismatch; import accord.coordinate.tracking.AppliedTracker; -import accord.coordinate.tracking.RequestStatus; import accord.impl.basic.SimulatedFault; import accord.local.Node; import accord.local.SafeCommandStore; -import accord.messages.Callback; -import accord.messages.ReadData; -import accord.messages.ReadData.ReadReply; -import accord.messages.WaitUntilApplied; import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.RoutableKey; @@ -52,12 +49,10 @@ import accord.primitives.SyncPoint; import accord.primitives.Timestamp; import accord.primitives.TxnId; -import accord.topology.Topologies; import accord.topology.Topology; import accord.utils.Timestamped; import accord.utils.async.AsyncChain; import accord.utils.async.AsyncChains; -import accord.utils.async.AsyncResults; import org.agrona.collections.Int2ObjectHashMap; import org.agrona.collections.LongArrayList; @@ -432,7 +427,7 @@ private void removeLocalWhenReady(Node node, long epoch, Ranges removed) // * api is range -> TxnId, so we still need a TxnId to be referenced off... so we need to create a TxnId to know when we are in-sync! // * if we have a sync point, we really only care if the sync point is applied globally... // * even though CoordinateDurabilityScheduling is part of BurnTest, it was seen that shard/global syncs were only happening in around 1/5 of the tests (mostly due to timeouts/invalidates), which would mean purge was called infrequently. - currentSyncPoint(node, removed).flatMap(sp -> Await.coordinate(node, epoch - 1, sp)).begin((s, f) -> { + currentSyncPoint(node, removed).flatMap(sp -> awaitSyncPoint(node, sp)).begin((s, f) -> { if (f != null) { node.agent().onUncaughtException(f); @@ -492,108 +487,38 @@ private static AsyncChain> currentSyncPoint(Node node, Ranges }); } - // TODO (duplication): this is 95% of accord.coordinate.CoordinateShardDurable - // we already report all this information to EpochState; would be better to use that - private static class Await extends AsyncResults.SettableResult> implements Callback + private static AsyncChain> awaitSyncPoint(Node node, SyncPoint exclusiveSyncPoint) { - private final Node node; - private final AppliedTracker tracker; - private final SyncPoint exclusiveSyncPoint; - - private Await(Node node, long minEpoch, SyncPoint exclusiveSyncPoint) - { - Topologies topologies = node.topology().forEpoch(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.sourceEpoch()); - this.node = node; - this.tracker = new AppliedTracker(topologies); - this.exclusiveSyncPoint = exclusiveSyncPoint; - } - - public static AsyncChain> coordinate(Node node, long minEpoch, SyncPoint sp) - { - Await coordinate = new Await(node, minEpoch, sp); - coordinate.start(); - return coordinate.recover(t -> { - if (t.getClass() == SyncPointErased.class) - return AsyncChains.success(null); - if (t instanceof Timeout || - // TODO (expected): why are we not simply handling Insufficient properly? - t instanceof RuntimeException && "Insufficient".equals(t.getMessage()) || - t instanceof SimulatedFault) - return coordinate(node, minEpoch, sp); - // cannot loop indefinitely - if (t instanceof RuntimeException && "Redundant".equals(t.getMessage())) - return AsyncChains.success(null); - return null; - }); - } - - private void start() - { - node.send(tracker.nodes(), to -> new WaitUntilApplied(to, tracker.topologies(), exclusiveSyncPoint.syncId, exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.syncId.epoch()), this); - } - @Override - public void onSuccess(Node.Id from, ReadReply reply) - { - if (!reply.isOk()) - { - ReadData.CommitOrReadNack nack = (ReadData.CommitOrReadNack) reply; - switch (nack) - { - default: throw new AssertionError("Unhandled: " + reply); - - case Insufficient: - CoordinateSyncPoint.sendApply(node, from, exclusiveSyncPoint); - return; - case Rejected: - tryFailure(new RuntimeException(nack.name())); - case Redundant: - tryFailure(new SyncPointErased()); - return; - case Invalid: - tryFailure(new Invalidated(exclusiveSyncPoint.syncId, exclusiveSyncPoint.homeKey)); - return; - } - } - else - { - if (tracker.recordSuccess(from) == RequestStatus.Success) - { - node.configService().reportEpochRedundant(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.syncId.epoch()); - trySuccess(exclusiveSyncPoint); - } - } - } - - private Throwable cause; + Await e = new Await(node, exclusiveSyncPoint); + e.addCallback(() -> node.configService().reportEpochRedundant(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.syncId.epoch())); + e.start(); + return e.recover(t -> { + if (t.getClass() == SyncPointErased.class) + return AsyncChains.success(null); + if (t instanceof Timeout || + // TODO (expected): why are we not simply handling Insufficient properly? + t instanceof RuntimeException && "Insufficient".equals(t.getMessage()) || + t instanceof SimulatedFault || + t instanceof Exhausted) + return awaitSyncPoint(node, exclusiveSyncPoint); + // cannot loop indefinitely + if (t instanceof RuntimeException && "Redundant".equals(t.getMessage())) + return AsyncChains.success(null); + return null; + }); + } - @Override - public void onFailure(Node.Id from, Throwable failure) + private static class Await extends ExecuteSyncPoint.ExecuteExclusiveSyncPoint + { + public Await(Node node, SyncPoint syncPoint) { - synchronized (this) - { - if (cause == null) cause = failure; - else - { - try - { - cause.addSuppressed(failure); - } - catch (Throwable t) - { - // can not always add suppress - node.agent().onUncaughtException(failure); - } - } - failure = cause; - } - if (tracker.recordFailure(from) == RequestStatus.Failed) - tryFailure(failure); + super(node, syncPoint, AppliedTracker::new); } @Override - public void onCallbackFailure(Node.Id from, Throwable failure) + public void start() { - tryFailure(failure); + super.start(); } } } diff --git a/accord-core/src/test/java/accord/impl/list/ListUpdate.java b/accord-core/src/test/java/accord/impl/list/ListUpdate.java index 6c0b11557..d09b55480 100644 --- a/accord-core/src/test/java/accord/impl/list/ListUpdate.java +++ b/accord-core/src/test/java/accord/impl/list/ListUpdate.java @@ -29,6 +29,7 @@ import accord.api.Update; import accord.local.CommandStore; import accord.primitives.Keys; +import accord.primitives.Participants; import accord.primitives.Ranges; import accord.primitives.Seekables; import accord.primitives.Timestamp; @@ -77,6 +78,18 @@ public Update slice(Ranges ranges) return result; } + @Override + public Update intersecting(Participants participants) + { + ListUpdate result = new ListUpdate(executor); + for (Map.Entry e : entrySet()) + { + if (participants.contains(e.getKey())) + result.put(e.getKey(), e.getValue()); + } + return result; + } + @Override public Update merge(Update other) { diff --git a/accord-core/src/test/java/accord/impl/mock/MockStore.java b/accord-core/src/test/java/accord/impl/mock/MockStore.java index 523856665..e3613fb27 100644 --- a/accord-core/src/test/java/accord/impl/mock/MockStore.java +++ b/accord-core/src/test/java/accord/impl/mock/MockStore.java @@ -27,6 +27,7 @@ import accord.api.Write; import accord.local.Node; import accord.local.SafeCommandStore; +import accord.primitives.Participants; import accord.primitives.Ranges; import accord.primitives.Seekable; import accord.primitives.Seekables; @@ -73,6 +74,12 @@ public Read slice(Ranges ranges) return MockStore.read(keys.slice(ranges)); } + @Override + public Read intersecting(Participants participants) + { + return MockStore.read(keys.intersecting(participants)); + } + @Override public Read merge(Read other) { @@ -109,6 +116,12 @@ public Update slice(Ranges ranges) return MockStore.update(keys.slice(ranges)); } + @Override + public Update intersecting(Participants participants) + { + return MockStore.update(keys.intersecting(participants)); + } + @Override public Update merge(Update other) { diff --git a/accord-core/src/test/java/accord/local/BootstrapLocalTxnTest.java b/accord-core/src/test/java/accord/local/BootstrapLocalTxnTest.java index 22b065454..42533284e 100644 --- a/accord-core/src/test/java/accord/local/BootstrapLocalTxnTest.java +++ b/accord-core/src/test/java/accord/local/BootstrapLocalTxnTest.java @@ -86,7 +86,7 @@ public void process(Node on, Node.Id from, ReplyContext replyContext) SyncPoint syncPoint = new SyncPoint<>(globalSyncId, Deps.NONE, ranges, route); Ranges valid = AccordGens.rangesInsideRanges(ranges, (rs2, r) -> rs2.nextInt(1, 4)).next(rs); Invariants.checkArgument(syncPoint.keysOrRanges.containsAll(valid)); - store.execute(contextFor(localSyncId, syncPoint.waitFor.keyDeps.keys(), KeyHistory.COMMANDS), safe -> Commands.createBootstrapCompleteMarkerTransaction(safe, localSyncId, syncPoint, valid)) + store.execute(contextFor(localSyncId, syncPoint.waitFor.keyDeps.keys(), KeyHistory.COMMANDS), safe -> Commands.createBootstrapCompleteMarkerTransaction(safe, localSyncId, valid)) .flatMap(ignore -> store.execute(contextFor(localSyncId), safe -> validate.accept(safe.get(localSyncId, route.homeKey()).current()))) .flatMap(ignore -> store.execute(contextFor(localSyncId), safe -> Commands.markBootstrapComplete(safe, localSyncId, ranges))) .flatMap(ignore -> store.execute(contextFor(localSyncId), safe -> validate.accept(safe.get(localSyncId, route.homeKey()).current()))) diff --git a/accord-core/src/test/java/accord/local/cfk/CommandsForKeyTest.java b/accord-core/src/test/java/accord/local/cfk/CommandsForKeyTest.java index c893702f2..ffd74861f 100644 --- a/accord-core/src/test/java/accord/local/cfk/CommandsForKeyTest.java +++ b/accord-core/src/test/java/accord/local/cfk/CommandsForKeyTest.java @@ -67,6 +67,7 @@ import accord.primitives.EpochSupplier; import accord.primitives.FullRoute; import accord.primitives.Keys; +import accord.primitives.PartialDeps; import accord.primitives.Participants; import accord.primitives.Range; import accord.primitives.Ranges; @@ -496,20 +497,20 @@ Command acceptedInvalidated(TxnId txnId, SaveStatus saveStatus) Command accepted(TxnId txnId, Timestamp executeAt, SaveStatus saveStatus) { Deps deps = generateDeps(txnId, txnId, Status.Accepted); - return Command.Accepted.accepted(common(txnId, saveStatus.known.definition.isKnown()).partialDeps(deps.slice(RANGES)), + return Command.Accepted.accepted(common(txnId, saveStatus.known.definition.isKnown()).partialDeps(deps.intersecting(txnId.domain() == Domain.Key ? KEY_ROUTE : RANGE_ROUTE)), saveStatus, executeAt, Ballot.ZERO, Ballot.ZERO); } Command committed(TxnId txnId, Timestamp executeAt) { Deps deps = generateDeps(txnId, executeAt, Status.Committed); - return Command.Committed.committed(common(txnId).partialDeps(deps.slice(RANGES)), SaveStatus.Committed, executeAt, Ballot.ZERO, Ballot.ZERO, null); + return Command.Committed.committed(common(txnId).partialDeps(slice(txnId, deps)), SaveStatus.Committed, executeAt, Ballot.ZERO, Ballot.ZERO, null); } Command stable(TxnId txnId, Timestamp executeAt, @Nullable Command.Committed committed) { Deps deps = committed == null ? generateDeps(txnId, executeAt, Status.Stable) : committed.partialDeps(); - CommonAttributes common = common(txnId).partialDeps(deps.slice(RANGES)); + CommonAttributes common = common(txnId).partialDeps(slice(txnId, deps)); Command.WaitingOn waitingOn = initialiseWaitingOn(txnId, executeAt, common.route(), deps); return Command.Committed.committed(common, SaveStatus.Stable, executeAt, Ballot.ZERO, Ballot.ZERO, waitingOn); } @@ -517,7 +518,7 @@ Command stable(TxnId txnId, Timestamp executeAt, @Nullable Command.Committed com Command applied(TxnId txnId, Timestamp executeAt, @Nullable Command.Committed committed) { Deps deps = committed == null ? generateDeps(txnId, executeAt, Status.Applied) : committed.partialDeps(); - CommonAttributes common = common(txnId).partialDeps(deps.slice(RANGES)); + CommonAttributes common = common(txnId).partialDeps(slice(txnId, deps)); Command.WaitingOn waitingOn = committed == null || committed.waitingOn == null ? initialiseWaitingOn(txnId, executeAt, common.route(), deps) : committed.waitingOn; return new Command.Executed(common, SaveStatus.Applied, executeAt, Ballot.ZERO, Ballot.ZERO, waitingOn, new Writes(txnId, executeAt, KEYS, null), new Result(){}); @@ -564,6 +565,11 @@ private Command selectOne(TreeSet from) TxnId txnId = from.floor(bound); return byId.get(txnId); } + + private static PartialDeps slice(TxnId txnId, Deps deps) + { + return deps.intersecting(txnId.domain() == Domain.Key ? KEY_ROUTE : RANGE_ROUTE); + } } @Test @@ -870,6 +876,12 @@ public Read slice(Ranges ranges) return this; } + @Override + public Read intersecting(Participants participants) + { + return this; + } + @Override public Read merge(Read other) { diff --git a/accord-core/src/test/java/accord/messages/ReadDataTest.java b/accord-core/src/test/java/accord/messages/ReadDataTest.java index 34ef1060d..eba37895f 100644 --- a/accord-core/src/test/java/accord/messages/ReadDataTest.java +++ b/accord-core/src/test/java/accord/messages/ReadDataTest.java @@ -263,7 +263,7 @@ private static class State this.partialRoute = route.slice(RANGES); this.progressKey = key.toUnseekable(); this.executeAt = txnId; - this.deps = PartialDeps.builder(RANGES).build(); + this.deps = PartialDeps.builder(partialRoute).build(); this.readResult = readResult; } diff --git a/accord-core/src/test/java/accord/primitives/AbstractRangesTest.java b/accord-core/src/test/java/accord/primitives/AbstractRangesTest.java index 82f3d78e5..e1df60d3b 100644 --- a/accord-core/src/test/java/accord/primitives/AbstractRangesTest.java +++ b/accord-core/src/test/java/accord/primitives/AbstractRangesTest.java @@ -105,6 +105,18 @@ public Routables slice(accord.primitives.Ranges ranges, Slice slice) { throw new UnsupportedOperationException(); } + + @Override + public Routables intersecting(Unseekables intersecting) + { + throw new UnsupportedOperationException(); + } + + @Override + public Routables intersecting(Unseekables intersecting, Slice slice) + { + throw new UnsupportedOperationException(); + } } private static class PrefixKey extends IntKey.Routing diff --git a/accord-core/src/test/java/accord/topology/ShardTest.java b/accord-core/src/test/java/accord/topology/ShardTest.java index 14b1ca228..56df12a92 100644 --- a/accord-core/src/test/java/accord/topology/ShardTest.java +++ b/accord-core/src/test/java/accord/topology/ShardTest.java @@ -19,11 +19,12 @@ package accord.topology; import accord.local.Node; +import accord.utils.SortedArrays.SortedArrayList; + import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.HashSet; -import java.util.List; import java.util.Set; import static accord.Utils.ids; @@ -151,7 +152,7 @@ void fastPathQuorumSizeTest() @Test void pendingNodeValidation() { - List nodes = ids(0, 3); + SortedArrayList nodes = ids(0, 3); Set fpNodes = new HashSet<>(ids(0, 2)); // pending nodes are part of electorate new Shard(range(0, 100), nodes, fpNodes, new HashSet<>(ids(3, 3))); diff --git a/accord-core/src/test/java/accord/topology/TopologyManagerTest.java b/accord-core/src/test/java/accord/topology/TopologyManagerTest.java index e018a26a4..9327c8155 100644 --- a/accord-core/src/test/java/accord/topology/TopologyManagerTest.java +++ b/accord-core/src/test/java/accord/topology/TopologyManagerTest.java @@ -30,6 +30,8 @@ import accord.utils.RandomSource; import com.google.common.collect.AbstractIterator; import com.google.common.collect.Iterators; + +import accord.utils.SortedArrays.SortedArrayList; import org.agrona.collections.Long2ObjectHashMap; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -366,9 +368,9 @@ private void test(Topology topology) void aba() { TopologyManager service = new TopologyManager(SUPPLIER, ID); - List dc1Nodes = idList(1, 2, 3); + SortedArrayList dc1Nodes = idList(1, 2, 3); Set dc1Fp = idSet(1, 2); - List dc2Nodes = idList(4, 5, 6); + SortedArrayList dc2Nodes = idList(4, 5, 6); Set dc2Fp = idSet(4, 5); addAndMarkSynced(service, topology(1, shard(PrefixedIntHashKey.range(0, 0, 100), dc2Nodes, dc2Fp), diff --git a/accord-core/src/test/java/accord/topology/TopologyRandomizer.java b/accord-core/src/test/java/accord/topology/TopologyRandomizer.java index 7b3a919c0..d388f925f 100644 --- a/accord-core/src/test/java/accord/topology/TopologyRandomizer.java +++ b/accord-core/src/test/java/accord/topology/TopologyRandomizer.java @@ -42,12 +42,14 @@ import accord.impl.PrefixedIntHashKey.Hash; import accord.impl.PrefixedIntHashKey.PrefixedIntRoutingKey; import accord.local.Node; +import accord.local.Node.Id; import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.Routables; import accord.primitives.Timestamp; import accord.utils.Invariants; import accord.utils.RandomSource; +import accord.utils.SortedArrays.SortedArrayList; import org.agrona.collections.IntHashSet; import static accord.burn.BurnTest.HASH_RANGE_END; @@ -81,25 +83,25 @@ private static class State } private static final Logger logger = LoggerFactory.getLogger(TopologyRandomizer.class); - private static final Node.Id[] EMPTY_NODES = new Node.Id[0]; + private static final Id[] EMPTY_NODES = new Id[0]; private static final Shard[] EMPTY_SHARDS = new Shard[0]; private final RandomSource random; private final AtomicInteger currentPrefix; private final List epochs = new ArrayList<>(); - private final Function nodeLookup; - private final Map previouslyReplicated = new HashMap<>(); + private final Function nodeLookup; + private final Map previouslyReplicated = new HashMap<>(); private final TopologyUpdates topologyUpdates; private final Listener listener; - public TopologyRandomizer(Supplier randomSupplier, Topology initialTopology, TopologyUpdates topologyUpdates, @Nullable Function nodeLookup, Listener listener) + public TopologyRandomizer(Supplier randomSupplier, Topology initialTopology, TopologyUpdates topologyUpdates, @Nullable Function nodeLookup, Listener listener) { this.random = randomSupplier.get(); this.currentPrefix = new AtomicInteger(random.nextInt(0, 1024)); this.topologyUpdates = topologyUpdates; this.epochs.add(Topology.EMPTY); this.epochs.add(initialTopology); - for (Node.Id node : initialTopology.nodes()) + for (Id node : initialTopology.nodes()) previouslyReplicated.put(node, initialTopology.rangesForNode(node)); this.nodeLookup = nodeLookup; this.listener = listener; @@ -209,13 +211,14 @@ private static Shard[] merge(State state, RandomSource random) System.arraycopy(shards, 0, result, 0, idx); System.arraycopy(shards, idx + 2, result, idx + 1, shards.length - (idx + 2)); Range range = PrefixedIntHashKey.range((Hash)left.range.start(), (Hash)right.range.end()); - List nodes; { - TreeSet tmp = new TreeSet<>(); + SortedArrayList nodes; { + TreeSet tmp = new TreeSet<>(); tmp.addAll(left.nodes); tmp.addAll(right.nodes); - nodes = new ArrayList<>(tmp); + nodes = new SortedArrayList<>(tmp.toArray(new Id[0])); } - Set joining = new TreeSet<>(); + + Set joining = new TreeSet<>(); joining.addAll(left.joining); joining.addAll(right.joining); result[idx] = new Shard(range, nodes, newFastPath(nodes, random), joining); @@ -237,7 +240,7 @@ private static Shard[] updateMembership(State state, RandomSource random) if (Arrays.stream(shards).allMatch(shard -> shard.sortedNodes.containsAll(shardLeft.sortedNodes) || shardLeft.containsAll(shard.sortedNodes))) return shards; - Set joining = new HashSet<>(shardLeft.joining); + Set joining = new HashSet<>(shardLeft.joining); int idxRight; Shard shardRight; @@ -247,8 +250,8 @@ private static Shard[] updateMembership(State state, RandomSource random) joining.addAll(shardRight.joining); } while (idxRight == idxLeft || shardLeft.sortedNodes.containsAll(shardRight.sortedNodes) || shardRight.sortedNodes.containsAll(shardLeft.sortedNodes)); - List nodesLeft; - Node.Id toRight; + List nodesLeft; + Id toRight; for (;;) { nodesLeft = new ArrayList<>(shardLeft.nodes); @@ -257,8 +260,8 @@ private static Shard[] updateMembership(State state, RandomSource random) break; } - List nodesRight; - Node.Id toLeft; + List nodesRight; + Id toLeft; for (;;) { nodesRight = new ArrayList<>(shardRight.nodes); @@ -271,8 +274,8 @@ private static Shard[] updateMembership(State state, RandomSource random) nodesRight.add(toRight); Shard[] newShards = shards.clone(); - newShards[idxLeft] = new Shard(shardLeft.range, nodesLeft, newFastPath(nodesLeft, random), Sets.intersection(joining, new HashSet<>(nodesLeft))); - newShards[idxRight] = new Shard(shardRight.range, nodesRight, newFastPath(nodesRight, random), Sets.intersection(joining, new HashSet<>(nodesRight))); + newShards[idxLeft] = new Shard(shardLeft.range, SortedArrayList.copyUnsorted(nodesLeft, Id[]::new), newFastPath(nodesLeft, random), Sets.intersection(joining, new HashSet<>(nodesLeft))); + newShards[idxRight] = new Shard(shardRight.range, SortedArrayList.copyUnsorted(nodesRight, Id[]::new), newFastPath(nodesRight, random), Sets.intersection(joining, new HashSet<>(nodesRight))); logger.debug("updated membership on {} & {} {} {} to {} {}", idxLeft, idxRight, shardLeft.toString(true), shardRight.toString(true), @@ -281,15 +284,15 @@ private static Shard[] updateMembership(State state, RandomSource random) return newShards; } - private static Set newFastPath(List nodes, RandomSource random) + private static Set newFastPath(List nodes, RandomSource random) { - List available = new ArrayList<>(nodes); + List available = new ArrayList<>(nodes); int rf = available.size(); int f = Shard.maxToleratedFailures(rf); int minSize = rf - f; int newSize = minSize + random.nextInt(f + 1); - Set fastPath = new HashSet<>(); + Set fastPath = new HashSet<>(); for (int i=0; i joining = new HashSet<>(); - Node.Id[] nodes; + Set joining = new HashSet<>(); + Id[] nodes; { - Set uniq = new HashSet<>(); + Set uniq = new HashSet<>(); for (Shard shard : shards) { uniq.addAll(shard.nodes); joining.addAll(shard.joining); } - Node.Id[] result = uniq.toArray(EMPTY_NODES); + Id[] result = uniq.toArray(EMPTY_NODES); Arrays.sort(result); nodes = result; } @@ -355,8 +358,8 @@ private static Shard[] addPrefix(State state, RandomSource random) for (int i = 0; i < ranges.length; i++) { Range range = ranges[i]; - List replicas = select(nodes, rf, random); - Set fastPath = newFastPath(replicas, random); + SortedArrayList replicas = select(nodes, rf, random); + Set fastPath = newFastPath(replicas, random); result.add(new Shard(range, replicas, fastPath, Sets.intersection(joining, new HashSet<>(replicas)))); } return result.toArray(EMPTY_SHARDS); @@ -375,18 +378,18 @@ private static int[] prefixes(Shard[] shards) return prefixes; } - private static List select(Node.Id[] nodes, int rf, RandomSource random) + private static SortedArrayList select(Id[] nodes, int rf, RandomSource random) { Invariants.checkArgument(nodes.length >= rf, "Given %d nodes, which is < rf of %d", nodes.length, rf); - List result = new ArrayList<>(rf); + List result = new ArrayList<>(rf); while (result.size() < rf) { - Node.Id id = random.pick(nodes); + Id id = random.pick(nodes); // TODO (efficiency) : rf is normally "small", so is it worth it to have a set, bitset, or another structure? if (!result.contains(id)) result.add(id); } - return result; + return SortedArrayList.copyUnsorted(result, Id[]::new); } private static int prefix(Shard shard) @@ -394,10 +397,10 @@ private static int prefix(Shard shard) return ((PrefixedIntHashKey) shard.range.start()).prefix; } - private static Map getAdditions(Topology current, Topology next) + private static Map getAdditions(Topology current, Topology next) { - Map additions = new HashMap<>(); - for (Node.Id node : next.nodes()) + Map additions = new HashMap<>(); + for (Id node : next.nodes()) { Ranges prev = current.rangesForNode(node); if (prev == null) prev = Ranges.EMPTY; @@ -411,12 +414,12 @@ private static Map getAdditions(Topology current, Topology next return additions; } - private static boolean reassignsRanges(Topology current, Shard[] nextShards, Map previouslyReplicated) + private static boolean reassignsRanges(Topology current, Shard[] nextShards, Map previouslyReplicated) { Topology next = new Topology(current.epoch + 1, nextShards); - Map additions = getAdditions(current, next); + Map additions = getAdditions(current, next); - for (Map.Entry entry : additions.entrySet()) + for (Map.Entry entry : additions.entrySet()) { if (previouslyReplicated.getOrDefault(entry.getKey(), Ranges.EMPTY).intersects(entry.getValue())) return true; @@ -468,8 +471,8 @@ public synchronized Topology updateTopology() Topology nextTopology = new Topology(current.epoch + 1, newShards); - Map nextAdditions = getAdditions(current, nextTopology); - for (Map.Entry entry : nextAdditions.entrySet()) + Map nextAdditions = getAdditions(current, nextTopology); + for (Map.Entry entry : nextAdditions.entrySet()) { Ranges previous = previouslyReplicated.getOrDefault(entry.getKey(), Ranges.EMPTY); Ranges added = entry.getValue(); @@ -483,7 +486,7 @@ public synchronized Topology updateTopology() if (nodeLookup != null) { - List nodes = new ArrayList<>(nextTopology.nodes()); + List nodes = new ArrayList<>(nextTopology.nodes()); int originatorIdx = random.nextInt(nodes.size()); topologyUpdates.notify(nodeLookup.apply(nodes.get(originatorIdx)), current, nextTopology); } @@ -506,13 +509,13 @@ private boolean everyShardHasOverlaps(long prevEpoch, Shard[] in, Shard[] out) return true; } - public void onStale(Node.Id id, Timestamp sinceAtLeast, Ranges ranges) + public void onStale(Id id, Timestamp sinceAtLeast, Ranges ranges) { int epoch = (int) sinceAtLeast.epoch(); - Invariants.checkState(epochs.get(epoch).nodeLookup.get(id).ranges.containsAll(ranges)); + Invariants.checkState(epochs.get(epoch).nodeLookup.get(id.id).ranges.containsAll(ranges)); while (++epoch < epochs.size()) { - ranges = ranges.slice(epochs.get(epoch).nodeLookup.get(id).ranges, Routables.Slice.Minimal); + ranges = ranges.slice(epochs.get(epoch).nodeLookup.get(id.id).ranges, Routables.Slice.Minimal); if (ranges.isEmpty()) return; } diff --git a/accord-core/src/test/java/accord/topology/TopologyUtils.java b/accord-core/src/test/java/accord/topology/TopologyUtils.java index f24e604f1..ec49f33a1 100644 --- a/accord-core/src/test/java/accord/topology/TopologyUtils.java +++ b/accord-core/src/test/java/accord/topology/TopologyUtils.java @@ -29,6 +29,7 @@ import accord.primitives.Unseekables.UnseekablesKind; import accord.utils.Gens; import accord.utils.RandomSource; +import accord.utils.SortedArrays.SortedArrayList; import accord.utils.Utils; import accord.utils.WrapAroundList; import accord.utils.WrapAroundSet; @@ -56,7 +57,7 @@ public static Ranges initialRanges(int num, int maxKey) public static Topology withEpoch(Topology topology, long epoch) { - return new Topology(topology.global == null ? null : withEpoch(topology.global, epoch), epoch, topology.shards, topology.ranges, topology.nodeLookup, topology.subsetOfRanges, topology.supersetIndexes); + return new Topology(topology.global == null ? null : withEpoch(topology.global, epoch), epoch, topology.shards, topology.ranges, topology.nodeIds, topology.nodeLookup, topology.subsetOfRanges, topology.supersetIndexes); } public static Topology topology(long epoch, List cluster, Ranges ranges, int rf) @@ -85,7 +86,8 @@ public static Topology topology(long epoch, Node.Id[] cluster, Ranges ranges, in Set noShard = new HashSet<>(Arrays.asList(cluster)); for (int i = 0; i < ranges.size() ; ++i) { - shards.add(new Shard(ranges.get(i), electorates.get(i % electorates.size()), fastPathElectorates.get(i % fastPathElectorates.size()))); + SortedArrayList sortedNodes = SortedArrayList.copyUnsorted(electorates.get(i % electorates.size()), Node.Id[]::new); + shards.add(new Shard(ranges.get(i), sortedNodes, fastPathElectorates.get(i % fastPathElectorates.size()))); noShard.removeAll(electorates.get(i % electorates.size())); } if (!noShard.isEmpty()) diff --git a/accord-core/src/test/java/accord/utils/AccordGens.java b/accord-core/src/test/java/accord/utils/AccordGens.java index e6724dd51..6c32f2aa9 100644 --- a/accord-core/src/test/java/accord/utils/AccordGens.java +++ b/accord-core/src/test/java/accord/utils/AccordGens.java @@ -51,6 +51,7 @@ import accord.primitives.TxnId; import accord.topology.Shard; import accord.topology.Topology; +import accord.utils.SortedArrays.SortedArrayList; import org.agrona.collections.IntHashSet; import static accord.utils.Utils.toArray; @@ -298,11 +299,11 @@ public static Gen keyDeps(Gen keyGen, Gen idGen) }; } - public static Gen shards(Gen rangeGen, Gen> nodesGen) + public static Gen shards(Gen rangeGen, Gen> nodesGen) { return rs -> { Range range = rangeGen.next(rs); - List nodes = nodesGen.next(rs); + SortedArrayList nodes = nodesGen.next(rs); int maxFailures = (nodes.size() - 1) / 2; Set fastPath = new HashSet<>(); if (maxFailures == 0) @@ -367,8 +368,9 @@ public static Gen topologys(Gen.LongGen epochGen, Gen nodeGen for (int i = 0; i < ranges.size() ; ++i) { WrapAroundList replicas = electorates.get(i % electorates.size()); + SortedArrayList sortedIds = SortedArrayList.copyUnsorted(replicas, Node.Id[]::new); Range range = ranges.get(i); - shards.add(shards(ignore -> range, ignore -> replicas).next(rs)); + shards.add(shards(ignore -> range, ignore -> sortedIds).next(rs)); replicas.forEach(noShard::remove); } if (!noShard.isEmpty()) diff --git a/accord-core/src/test/java/accord/utils/SearchableRangeListTest.java b/accord-core/src/test/java/accord/utils/SearchableRangeListTest.java index d31c95c93..9a5f70656 100644 --- a/accord-core/src/test/java/accord/utils/SearchableRangeListTest.java +++ b/accord-core/src/test/java/accord/utils/SearchableRangeListTest.java @@ -45,7 +45,7 @@ public void fullWorld() class Counter { int value;} BiConsumer test = (rangeStart, rangeEnd) -> { Counter counter = new Counter(); - list.forEach(IntKey.range(rangeStart, rangeEnd), (a, b, c, d, e) -> { + list.forEachRange(IntKey.range(rangeStart, rangeEnd), (a, b, c, d, e) -> { counter.value++; }, (a, b, c, d, start, end) -> { counter.value += (end - start + 1); @@ -102,7 +102,7 @@ public void random() expected.add(r); } List actual = new ArrayList<>(expected.size()); - list.forEach(range, (a, b, c, d, idx) -> { + list.forEachRange(range, (a, b, c, d, idx) -> { actual.add(list.ranges[idx]); }, (a, b, c, d, start, end) -> { for (int j = start; j < end; j++) diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromRead.java b/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromRead.java index 9bb16f113..afd0a0b19 100644 --- a/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromRead.java +++ b/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromRead.java @@ -24,6 +24,7 @@ import accord.api.Read; import accord.local.SafeCommandStore; import accord.primitives.Keys; +import accord.primitives.Participants; import accord.primitives.Ranges; import accord.primitives.Seekable; import accord.primitives.Timestamp; @@ -62,6 +63,12 @@ public Read slice(Ranges ranges) return new MaelstromRead(readKeys.slice(ranges), keys.slice(ranges)); } + @Override + public Read intersecting(Participants participants) + { + return new MaelstromRead(readKeys.intersecting(participants), keys.intersecting(participants)); + } + @Override public Read merge(Read other) { diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromUpdate.java b/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromUpdate.java index a89eacae6..cf0ba59dc 100644 --- a/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromUpdate.java +++ b/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromUpdate.java @@ -24,8 +24,10 @@ import accord.api.Key; import accord.api.Data; import accord.api.Update; +import accord.primitives.Participants; import accord.primitives.Ranges; import accord.primitives.Keys; +import accord.primitives.Routables; import accord.primitives.Timestamp; public class MaelstromUpdate extends TreeMap implements Update @@ -48,11 +50,22 @@ public MaelstromWrite apply(Timestamp executeAt, Data read) @Override public Update slice(Ranges ranges) + { + return intersecting((Routables) ranges); + } + + @Override + public Update intersecting(Participants participants) + { + return intersecting((Routables) participants); + } + + public Update intersecting(Routables routables) { MaelstromUpdate result = new MaelstromUpdate(); for (Map.Entry e : entrySet()) { - if (ranges.contains(e.getKey())) + if (routables.contains(e.getKey())) result.put(e.getKey(), e.getValue()); } return result; diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/TopologyFactory.java b/accord-maelstrom/src/main/java/accord/maelstrom/TopologyFactory.java index 71bef1d99..94ae12abe 100644 --- a/accord-maelstrom/src/main/java/accord/maelstrom/TopologyFactory.java +++ b/accord-maelstrom/src/main/java/accord/maelstrom/TopologyFactory.java @@ -32,6 +32,7 @@ import accord.utils.WrapAroundList; import accord.utils.WrapAroundSet; +import static accord.utils.SortedArrays.SortedArrayList.copyUnsorted; import static accord.utils.Utils.toArray; public class TopologyFactory @@ -82,7 +83,7 @@ public Topology toTopology(Id[] cluster) for (int j = 0 ; j < kinds.length ; ++j) { for (int i = 0 ; i < this.shards ; ++i) - shards.add(new Shard(ranges[j][i], electorates.get(i % electorates.size()), fastPathElectorates.get(i % fastPathElectorates.size()))); + shards.add(new Shard(ranges[j][i], copyUnsorted(electorates.get(i % electorates.size()), Id[]::new), fastPathElectorates.get(i % fastPathElectorates.size()))); } return new Topology(1, toArray(shards, Shard[]::new)); }