diff --git a/accord-core/src/main/java/accord/api/AsyncExecutor.java b/accord-core/src/main/java/accord/api/AsyncExecutor.java index 731a2fef5e..f303f1418d 100644 --- a/accord-core/src/main/java/accord/api/AsyncExecutor.java +++ b/accord-core/src/main/java/accord/api/AsyncExecutor.java @@ -26,6 +26,7 @@ import accord.utils.async.AsyncChain; import accord.utils.async.Cancellable; +// TODO (required): consistent RejectedExecutionException handling public interface AsyncExecutor extends Executor { // unlike execute, throws no exceptions, nor will not wrap the runnable @@ -38,6 +39,7 @@ default Cancellable execute(RunOrFail run) // Depending on this implementation this method may queue-jump, i.e. task submission order is not guaranteed. // Make sure this is semantically safe at all call-sites. + // TODO (required): RejectedExecutionException? default void executeMaybeImmediately(Runnable run) { if (!tryExecuteImmediately(run)) diff --git a/accord-core/src/main/java/accord/api/DataStore.java b/accord-core/src/main/java/accord/api/DataStore.java index 45eb231f26..74dd68c6d8 100644 --- a/accord-core/src/main/java/accord/api/DataStore.java +++ b/accord-core/src/main/java/accord/api/DataStore.java @@ -140,10 +140,22 @@ static FetchResult failure(Throwable t) */ FetchResult fetch(Node node, SafeCommandStore safeStore, Ranges ranges, SyncPoint syncPoint, FetchRanges callback, FetchKind kind); + /** + * Logical fsync-like operation: anything within the provided ranges written to the store prior to the + * invocation of this method must be durable once the AsyncResult completes successfully. That is, a restart of the node must + * restore the DataStore to a state on or after the point at which snapshot was invoked. + * + * TODO (desired): clunky to pass integer flags around; is there a neater implementation-agnostic alternative? + */ + default void ensureDurable(CommandStore commandStore, Ranges ranges, RedundantBefore reportOnSuccess, int flags) + { + ensureDurable(commandStore, reportOnSuccess, flags); + } + /** * Logical fsync-like operation: anything written to the store prior to the invocation of this method * must be durable once the AsyncResult completes successfully. That is, a restart of the node must * restore the DataStore to a state on or after the point at which snapshot was invoked. */ - void ensureDurable(CommandStore commandStore, Ranges ranges, RedundantBefore reportOnSuccess); + void ensureDurable(CommandStore commandStore, RedundantBefore reportOnSuccess, int flags); } diff --git a/accord-core/src/main/java/accord/api/Journal.java b/accord-core/src/main/java/accord/api/Journal.java index 381f1be2d6..3b7c5dbe0e 100644 --- a/accord-core/src/main/java/accord/api/Journal.java +++ b/accord-core/src/main/java/accord/api/Journal.java @@ -53,6 +53,7 @@ enum Load MINIMAL_WITH_DEPS } + void open(Node node); void start(Node node); Command loadCommand(int store, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore); @@ -61,26 +62,25 @@ enum Load Command.MinimalWithDeps loadMinimalWithDeps(int store, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore); void saveCommand(int store, CommandUpdate value, Runnable onFlush); - List replayTopologies(); + List loadTopologies(); void saveTopology(TopologyUpdate topologyUpdate, Runnable onFlush); - void purge(CommandStores commandStores, EpochSupplier minEpoch); - - /** - * Replays all messages from journal to rehydrate CommandStores state. Returns whether it has seen (and ignored) - * any exceptions during replay. - */ - boolean replay(CommandStores commandStores); - RedundantBefore loadRedundantBefore(int store); NavigableMap loadBootstrapBeganAt(int store); NavigableMap loadSafeToRead(int store); CommandStores.RangesForEpoch loadRangesForEpoch(int store); + void saveStoreState(int store, FieldUpdates fieldUpdates, Runnable onFlush); Persister durableBeforePersister(); - void saveStoreState(int store, FieldUpdates fieldUpdates, Runnable onFlush); + void purge(CommandStores commandStores, EpochSupplier minEpoch); + /** + * Replays all messages from journal to rehydrate CommandStores state. Returns whether it has seen (and ignored) + * any exceptions during replay. + */ + boolean replay(CommandStores commandStores, Object param); + class TopologyUpdate { public final Int2ObjectHashMap commandStores; diff --git a/accord-core/src/main/java/accord/api/LocalListeners.java b/accord-core/src/main/java/accord/api/LocalListeners.java index caa51ec722..38cf4967b9 100644 --- a/accord-core/src/main/java/accord/api/LocalListeners.java +++ b/accord-core/src/main/java/accord/api/LocalListeners.java @@ -105,6 +105,23 @@ public TxnListener(TxnId waiter, TxnId waitingOn, SaveStatus awaitingStatus) this.waitingOn = waitingOn; this.awaitingStatus = awaitingStatus; } + + @Override + public boolean equals(Object that) + { + return that instanceof TxnListener && equals((TxnListener) that); + } + + public boolean equals(TxnListener that) + { + return this.waiter.equals(that.waiter) && this.waitingOn.equals(that.waitingOn) && this.awaitingStatus == that.awaitingStatus; + } + + @Override + public int hashCode() + { + throw new UnsupportedOperationException(); + } } Iterable txnListeners(); diff --git a/accord-core/src/main/java/accord/api/ProtocolModifiers.java b/accord-core/src/main/java/accord/api/ProtocolModifiers.java index 64e2260107..03f6fb1577 100644 --- a/accord-core/src/main/java/accord/api/ProtocolModifiers.java +++ b/accord-core/src/main/java/accord/api/ProtocolModifiers.java @@ -127,7 +127,7 @@ static Spec parse(String description) case "accept": accept = include; Invariants.require(cfp == DoNotChase, "Invalid to specify ChaseFixedPoint.Chase for accept"); break; case "commit": commit = include; Invariants.require(cfp == DoNotChase, "Invalid to specify ChaseFixedPoint.Chase for commit"); break; case "stable": stable = include; Invariants.require(cfp == DoNotChase, "Invalid to specify ChaseFixedPoint.Chase for stable"); break; - case "recover": recover = include; Invariants.require(cfp == DoNotChase, "Invalid to specify ChaseFixedPoint.Chase for stable"); break; + case "recover": recover = include; Invariants.require(cfp == DoNotChase, "Invalid to specify ChaseFixedPoint.Chase for recover"); break; } } @@ -194,6 +194,32 @@ public static boolean discardPreAcceptDeps(TxnId txnId) } } + public static class RangeSpecParam + { + // Implementations may set this prior to creating any Range object to specify whether the start or end bound of its ranges are inclusive. + public static boolean END_INCLUSIVE = true; + } + public static class RangeSpec + { + public static final boolean END_INCLUSIVE = RangeSpecParam.END_INCLUSIVE; + public static boolean isEndInclusive() + { + return END_INCLUSIVE; + } + public static boolean isEndExclusive() + { + return !END_INCLUSIVE; + } + public static boolean isStartInclusive() + { + return !END_INCLUSIVE; + } + public static boolean isStartExclusive() + { + return END_INCLUSIVE; + } + } + public static class Toggles { private static FastPaths permittedFastPaths = new FastPaths(FastPath.values()); @@ -205,6 +231,18 @@ public static class Toggles public static MediumPath defaultMediumPath() { return defaultMediumPath; } public static void setDefaultMediumPath(MediumPath newDefaultMediumPath) { defaultMediumPath = newDefaultMediumPath; } + private static boolean recoveryAwaitsSupersedingSyncPoints = true; + public static boolean recoveryAwaitsSupersedingSyncPoints() { return recoveryAwaitsSupersedingSyncPoints; } + public static void setRecoveryAwaitsSupersedingSyncPoints(boolean newRecoveryAwaitsSupersedingSyncPoints) { recoveryAwaitsSupersedingSyncPoints = newRecoveryAwaitsSupersedingSyncPoints; } + + private static boolean syncPointsTrackUnstableMediumPathDependencies = false; + public static boolean syncPointsTrackUnstableMediumPathDependencies() { return syncPointsTrackUnstableMediumPathDependencies; } + public static void setSyncPointsTrackUnstableMediumPathDependencies(boolean newSyncPointsTrackUnstableMediumPathDependencies) { syncPointsTrackUnstableMediumPathDependencies = newSyncPointsTrackUnstableMediumPathDependencies; } + + private static boolean recoverPartialAcceptPhaseIfNoFastPath = false; + public static boolean recoverPartialAcceptPhaseIfNoFastPath() { return recoverPartialAcceptPhaseIfNoFastPath; } + public static void setRecoverPartialAcceptPhaseIfNoFastPath(boolean newSyncPointsRecoverPartialAcceptPhase) {recoverPartialAcceptPhaseIfNoFastPath = newSyncPointsRecoverPartialAcceptPhase; } + private static boolean filterDuplicateDependenciesFromAcceptReply = true; public static boolean filterDuplicateDependenciesFromAcceptReply() { return filterDuplicateDependenciesFromAcceptReply; } public static void setFilterDuplicateDependenciesFromAcceptReply(boolean newFilterDuplicateDependenciesFromAcceptReply) { filterDuplicateDependenciesFromAcceptReply = newFilterDuplicateDependenciesFromAcceptReply; } diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java b/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java index 37cb7ca663..8a86375933 100644 --- a/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java +++ b/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java @@ -54,7 +54,6 @@ import static accord.api.ProtocolModifiers.Toggles.permitLocalExecution; import static accord.coordinate.ExecutePath.EPHEMERAL; -import static accord.coordinate.ExecutePath.FAST; import static accord.coordinate.ReadCoordinator.Action.Approve; import static accord.coordinate.ReadCoordinator.Action.ApprovePartial; import static accord.primitives.Status.Phase.Execute; diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java b/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java index 5b06000a94..d38abb6478 100644 --- a/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java +++ b/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java @@ -399,20 +399,23 @@ protected void onDone(Success success, Throwable failure) } else { - stable.informStableOnceQuorum(); - if (sendOnlyReadStableMessages()) + if (!isPrivilegedVoteCommitting) { - // send additional stable messages to record the transaction outcome - Commit.Kind kind = commitKind(); - if (!candidates.isEmpty()) + stable.informStableOnceQuorum(); + if (sendOnlyReadStableMessages()) { - for (int i = 0, size = candidates.size() ; i < size ; ++i) - sendStableOnly(candidates.get(i), kind); - } - if (unstableFastReads != null) - { - for (Node.Id to : unstableFastReads) - sendStableOnly(to, kind); + // send additional stable messages to record the transaction outcome + Commit.Kind kind = commitKind(); + if (!candidates.isEmpty()) + { + for (int i = 0, size = candidates.size() ; i < size ; ++i) + sendStableOnly(candidates.get(i), kind); + } + if (unstableFastReads != null) + { + for (Node.Id to : unstableFastReads) + sendStableOnly(to, kind); + } } } invokeCallback(null, failure); @@ -430,9 +433,8 @@ public void onSlowResponse(Id from) @Override public void onFailure(Id from, Throwable failure) { - super.onFailure(from, failure); - if (isPrivilegedVoteCommitting && from.id == node.id().id) - tryFinishOnFailure(); + if (isPrivilegedVoteCommitting && from.id == node.id().id) finishWithFailure(failure); + else super.onFailure(from, failure); } protected CoordinationAdapter adapter() diff --git a/accord-core/src/main/java/accord/coordinate/KeyBarriers.java b/accord-core/src/main/java/accord/coordinate/KeyBarriers.java index 68beb49d47..e7a28d744a 100644 --- a/accord-core/src/main/java/accord/coordinate/KeyBarriers.java +++ b/accord-core/src/main/java/accord/coordinate/KeyBarriers.java @@ -24,6 +24,7 @@ import org.slf4j.LoggerFactory; import accord.api.RoutingKey; +import accord.api.VisibleForImplementation; import accord.local.CommandSummaries; import accord.local.MapReduceConsumeCommandStores; import accord.local.Node; @@ -48,8 +49,6 @@ import static accord.local.CommandSummaries.SummaryStatus.APPLIED; import static accord.local.CommandSummaries.SummaryStatus.COMMITTED; import static accord.local.CommandSummaries.SummaryStatus.INVALIDATED; -import static accord.local.CommandSummaries.ComputeIsDep.IGNORE; -import static accord.local.CommandSummaries.TestStartedAt.STARTED_AFTER; import static accord.local.durability.DurabilityService.SyncLocal.NoLocal; import static accord.local.durability.DurabilityService.SyncLocal.Self; import static accord.local.durability.DurabilityService.SyncRemote.NoRemote; @@ -63,6 +62,7 @@ * Facility for finding existing key transactions that can serve as a barrier transaction, * ensuring all reads/writes after some point in the txn log have been executed. */ +@VisibleForImplementation public class KeyBarriers { @SuppressWarnings("unused") @@ -84,7 +84,7 @@ public Found(TxnId txnId, RoutingKey key, SyncLocal knownLocal, SyncRemote known } } - public static AsyncResult find(Node node, Timestamp min, RoutingKey key, SyncLocal syncLocal, SyncRemote syncRemote) + public static AsyncResult find(Node node, TxnId min, RoutingKey key, SyncLocal syncLocal, SyncRemote syncRemote) { Find find = new Find(min, key, syncLocal, syncRemote); node.commandStores().mapReduceConsume(min.epoch(), Long.MAX_VALUE, find); @@ -98,16 +98,16 @@ public static AsyncResult find(Node node, Timestamp min, RoutingKey key, * For Applied we can return success immediately with the executeAt epoch. For PreApplied we can add * a listener for when it transitions to Applied and then return success. */ - static class Find extends MapReduceConsumeCommandStores implements CommandSummaries.AllCommandVisitor + static class Find extends MapReduceConsumeCommandStores implements CommandSummaries.SupersedingCommandVisitor { final AsyncResults.SettableByCallback result = new AsyncResults.SettableByCallback<>(); - final Timestamp min; + final TxnId min; final RoutingKey find; final SyncLocal syncLocal; final SyncRemote syncRemote; Found found; - Find(Timestamp min, RoutingKey find, SyncLocal syncLocal, SyncRemote syncRemote) + Find(TxnId min, RoutingKey find, SyncLocal syncLocal, SyncRemote syncRemote) { super(RoutingKeys.of(find)); this.min = min; @@ -122,7 +122,7 @@ public Found applyInternal(SafeCommandStore safeStore) // Barriers are trying to establish that committed transactions are applied before the barrier (or in this case just minEpoch) // so all existing transaction types should ensure that at this point. An earlier txnid may have an executeAt that is after // this barrier or the transaction we listen on and that is fine - safeStore.visit(scope, TxnId.NONE, Ws, STARTED_AFTER, min, IGNORE, this); + safeStore.visit(scope, min, Ws, this); return found; } diff --git a/accord-core/src/main/java/accord/coordinate/Propose.java b/accord-core/src/main/java/accord/coordinate/Propose.java index 80ba9f152b..41e8573cd6 100644 --- a/accord-core/src/main/java/accord/coordinate/Propose.java +++ b/accord-core/src/main/java/accord/coordinate/Propose.java @@ -31,6 +31,7 @@ import accord.local.Node.Id; import accord.local.SequentialAsyncExecutor; import accord.messages.Accept; +import accord.messages.Accept.AcceptFlags; import accord.messages.Accept.AcceptReply; import accord.messages.Callback; import accord.primitives.Ballot; @@ -47,9 +48,9 @@ import accord.utils.SortedListMap; import accord.utils.Rethrowable; -import static accord.api.ProtocolModifiers.Toggles.filterDuplicateDependenciesFromAcceptReply; import static accord.coordinate.tracking.RequestStatus.Failed; import static accord.coordinate.tracking.RequestStatus.Success; +import static accord.messages.Accept.AcceptFlags.filterDeps; import static accord.messages.Commit.Invalidate.commitInvalidate; import static accord.primitives.Routables.Slice.Minimal; import static accord.primitives.Status.AcceptedInvalidate; @@ -66,6 +67,7 @@ abstract class Propose extends AbstractCoordination, R, AcceptRe final Timestamp executeAt; final QuorumTracker tracker; + final int acceptFlags; Propose(Node node, SequentialAsyncExecutor executor, Topologies topologies, Accept.Kind kind, Ballot ballot, TxnId txnId, Txn txn, Route require, FullRoute route, Timestamp executeAt, Deps deps, BiConsumer callback) { @@ -77,6 +79,7 @@ abstract class Propose extends AbstractCoordination, R, AcceptRe this.deps = deps; this.executeAt = executeAt; this.tracker = new QuorumTracker(topologies); + this.acceptFlags = AcceptFlags.encode(txnId, require != scope); Invariants.require(txnId.isSyncPoint() || deps.maxTxnId(txnId).compareTo(executeAt) <= 0, "Attempted to propose %s with an earlier executeAt than a conflicting transaction it witnessed: %s vs executeAt: %s", txnId, deps, executeAt); Invariants.require(topologies.currentEpoch() == executeAt.epoch()); @@ -86,7 +89,7 @@ abstract class Propose extends AbstractCoordination, R, AcceptRe void start() { super.start(); - contact(to -> new Accept(to, tracker.topologies(), kind, ballot, txnId, scope, executeAt, deps, require != scope)); + contact(to -> new Accept(to, tracker.topologies(), kind, ballot, txnId, scope, executeAt, deps, acceptFlags)); } @Override @@ -163,6 +166,9 @@ Deps mergeDeps(Deps newDeps) Deps mergeNewDeps() { SortedListMap oks = finishOks(); + if (!AcceptFlags.calculateDeps(acceptFlags)) + return Deps.NONE; + Deps deps = Deps.merge(oks, oks.domainSize(), SortedListMap::getValue, ok -> ok.deps); if (Faults.discardPreAcceptDeps(txnId)) return deps; @@ -170,8 +176,12 @@ Deps mergeNewDeps() if (txnId.is(TrackStable)) { // we must not propose as stable any dep < txnId that we did not propose as part of this phase - if (filterDuplicateDependenciesFromAcceptReply()) + if (!filterDeps(acceptFlags)) + { + // if the replicas haven't filtered their responses, we need to remove + // anything we sent to ensure we don't mark them as unstable deps = deps.without(this.deps); + } deps = deps.markUnstableBefore(txnId); } @@ -181,7 +191,6 @@ Deps mergeNewDeps() abstract CoordinationAdapter adapter(); - @Override public CoordinationKind kind() { diff --git a/accord-core/src/main/java/accord/coordinate/Recover.java b/accord-core/src/main/java/accord/coordinate/Recover.java index 4b54a43d6d..01f0a50d04 100644 --- a/accord-core/src/main/java/accord/coordinate/Recover.java +++ b/accord-core/src/main/java/accord/coordinate/Recover.java @@ -70,6 +70,7 @@ import accord.utils.async.AsyncChains; import static accord.api.ProtocolModifiers.QuorumEpochIntersections; +import static accord.api.ProtocolModifiers.Toggles.recoverPartialAcceptPhaseIfNoFastPath; import static accord.coordinate.CoordinationAdapter.Factory.Kind.Recovery; import static accord.coordinate.ExecutePath.RECOVER; import static accord.coordinate.Infer.InvalidateAndCallback.locallyInvalidateAndCallback; @@ -86,6 +87,10 @@ import static accord.messages.BeginRecovery.RecoverOk.maxAcceptedNotTruncated; import static accord.messages.BeginRecovery.RecoveryFlags.FAST_PATH_DECIDED; import static accord.messages.BeginRecovery.RecoveryFlags.FORCE_RECOVER_FAST_PATH; +import static accord.messages.BeginRecovery.RecoveryFlags.NO_CALCULATE_DEPS; +import static accord.messages.BeginRecovery.RecoveryFlags.calculateDeps; +import static accord.messages.BeginRecovery.RecoveryFlags.forceRecoverFastPath; +import static accord.messages.BeginRecovery.RecoveryFlags.isFastPathDecided; import static accord.primitives.ProgressToken.TRUNCATED_DURABLE_OR_INVALIDATED; import static accord.primitives.Routables.Slice.Minimal; import static accord.primitives.Status.AcceptedMedium; @@ -124,7 +129,7 @@ private Recover(Node node, SequentialAsyncExecutor executor, Topologies topologi BiConsumer callback) { super(node, executor, txnId, route, topologies.nodes(), callback); - this.flags = computeFlags(isFastPathDecided, topologies); + this.flags = computeFlags(txnId, isFastPathDecided, topologies); Invariants.require(txnId.isVisible()); this.adapter = node.coordinationAdapter(txnId, Recovery); this.ballot = ballot; @@ -134,9 +139,11 @@ private Recover(Node node, SequentialAsyncExecutor executor, Topologies topologi this.tracker = new RecoveryTracker(topologies); } - private static int computeFlags(boolean isFastPathDecided, Topologies topologies) + private static int computeFlags(TxnId txnId, boolean isFastPathDecided, Topologies topologies) { - int flags = (isFastPathDecided ? TinyEnumSet.encode(FAST_PATH_DECIDED) : 0); + int flags = (isFastPathDecided ? TinyEnumSet.encode(FAST_PATH_DECIDED) : 0) + | (!txnId.hasFastPath() && !recoverPartialAcceptPhaseIfNoFastPath() ? TinyEnumSet.encode(NO_CALCULATE_DEPS) : 0); + for (int i = 0 ; i < topologies.size() ; ++i) { Topology topology = topologies.get(i); @@ -284,16 +291,16 @@ private void recover() { Timestamp executeAt = acceptOrCommitNotTruncated.executeAt; Status status; { - Status tmp = acceptOrCommitNotTruncated.status; - if (committedExecuteAt != null) - { - Invariants.require(acceptOrCommitNotTruncated.status.compareTo(Status.PreCommitted) < 0 || executeAt.equals(committedExecuteAt)); - // if we know from a prior Accept attempt that this is committed we can go straight to the commit phase - if (tmp == AcceptedMedium || tmp == AcceptedSlow) - tmp = Status.Committed; - } - status = tmp; + Status tmp = acceptOrCommitNotTruncated.status; + if (committedExecuteAt != null) + { + Invariants.require(acceptOrCommitNotTruncated.status.compareTo(Status.PreCommitted) < 0 || executeAt.equals(committedExecuteAt)); + // if we know from a prior Accept attempt that this is committed we can go straight to the commit phase + if (tmp == AcceptedMedium || tmp == AcceptedSlow) + tmp = Status.Committed; } + status = tmp; + } switch (status) { @@ -312,7 +319,7 @@ private void recover() if (tracing != null) tracing.trace(null, "found AcceptedInvalidate: continuing Invalidate."); - invalidate(oks); + finishAndInvalidate(oks); return; } @@ -323,7 +330,23 @@ private void recover() LatestDeps.Merge merge = mergeDeps(okList); Participants await = merge.notAccepted(scope); - awaitPartialEarlier(okList, await, () -> { + + if (!calculateDeps(flags)) + { + Invariants.require(!txnId.hasFastPath()); + switch (status) + { + case AcceptedMedium: + case AcceptedSlow: + if (!await.isEmpty()) + { + finishAndInvalidate(oks); + return; + } + } + } + + awaitPartialSimple(okList, await, () -> { BiConsumer callback = finishAndTakeResultCallback(); switch (status) { @@ -411,19 +434,19 @@ private void recover() } Invariants.require(committedExecuteAt == null || committedExecuteAt.equals(txnId)); - Invariants.require(!TinyEnumSet.contains(flags, FAST_PATH_DECIDED) || TinyEnumSet.contains(flags, FORCE_RECOVER_FAST_PATH)); + Invariants.require(!isFastPathDecided(flags) || forceRecoverFastPath(flags)); boolean coordinatorInRecoveryQuorum = oks.get(txnId.node) != null; Participants extraCoordVotes = extraCoordinatorVotes(txnId, coordinatorInRecoveryQuorum, okList); - Participants extraRejects = Deps.merge(okList, okList.size(), List::get, ok -> ok.laterCoordRejects) + Participants extraRejects = Deps.merge(okList, okList.size(), List::get, ok -> ok.supersedingCoordRejects) .intersecting(scope, id -> !oks.containsKey(id.node)); InferredFastPath fastPath; if (txnId.hasPrivilegedCoordinator() && coordinatorInRecoveryQuorum) fastPath = Reject; else if (txnId.isSyncPoint()) fastPath = Reject; else fastPath = merge( - supersedingRejects(okList) ? Reject : Unknown, - tracker.inferFastPathDecision(txnId, extraCoordVotes, extraRejects) - ); + supersedingRejects(okList) ? Reject : Unknown, + tracker.inferFastPathDecision(txnId, extraCoordVotes, extraRejects) + ); switch (fastPath) { @@ -432,7 +455,7 @@ private void recover() if (tracing != null) tracing.trace(null, "found fast path rejection; invoking Invalidate."); - invalidate(oks); + finishAndInvalidate(oks); return; } case Accept: @@ -444,8 +467,8 @@ private void recover() case Unknown: { // should all be PreAccept - Deps earlierWait = Deps.merge(okList, okList.size(), List::get, ok -> ok.earlierWait); - Deps earlierNoWait = Deps.merge(okList, okList.size(), List::get, ok -> ok.earlierNoWait); + Deps earlierWait = Deps.merge(okList, okList.size(), List::get, ok -> ok.simpleWait); + Deps earlierNoWait = Deps.merge(okList, okList.size(), List::get, ok -> ok.simpleNoWait); earlierWait = earlierWait.without(earlierNoWait); Deps laterWitnessedCoordRejects = Deps.merge(oks, oks.domainSize(), (map, i) -> selectCoordinatorReplies(map.getKey(i), map.getValue(i)), Function.identity()); @@ -455,37 +478,37 @@ private void recover() // we have to be certain these commands have not successfully committed without witnessing us (thereby // ruling out a fast path decision for us and changing our recovery decision). // So, we wait for these commands to commit and recompute supersedingRejects for them. - awaitToFinish(AsyncChains.reduce(awaitEarlier(node, earlierWait, HasCommittedDeps), - awaitLater(node, laterWitnessedCoordRejects, CommittedOrNotFastPathCommit, extraCoordVotes), - InferredFastPath::merge) - .invokeIfSuccess((inferred) -> { - switch (inferred) - { - default: throw new UnhandledEnum(inferred); - case Accept: - { - if (tracing != null) - tracing.trace(null, "found accepted fast path; proposing."); - propose(SLOW, txnId, okList); - break; - } - case Unknown: - { - if (tracing != null) - tracing.trace(null, "found unknown fast path decision; retrying."); - retry(committedExecuteAt, finishAndUnwrapCallback()); - break; - } - case Reject: - { - if (tracing != null) - tracing.trace(null, "found fast path rejection; invoking Invalidate."); - - invalidate(oks); - break; - } - } - })); + awaitToFinish(AsyncChains.reduce(awaitSimple(node, earlierWait, HasCommittedDeps), + awaitSupersedingCoord(node, laterWitnessedCoordRejects, CommittedOrNotFastPathCommit, extraCoordVotes), + InferredFastPath::merge) + .invokeIfSuccess((inferred) -> { + switch (inferred) + { + default: throw new UnhandledEnum(inferred); + case Accept: + { + if (tracing != null) + tracing.trace(null, "found accepted fast path; proposing."); + propose(SLOW, txnId, okList); + break; + } + case Unknown: + { + if (tracing != null) + tracing.trace(null, "found unknown fast path decision; retrying."); + retry(committedExecuteAt, finishAndUnwrapCallback()); + break; + } + case Reject: + { + if (tracing != null) + tracing.trace(null, "found fast path rejection; invoking Invalidate."); + + finishAndInvalidate(oks); + break; + } + } + })); } else { @@ -502,13 +525,13 @@ private static LatestDeps.Merge mergeDeps(List nullableRecoverOkList) return LatestDeps.merge(nullableRecoverOkList, ok -> ok == null ? null : ok.deps); } - private void awaitPartialEarlier(List nullableRecoverOkList, Participants participants, Runnable whenReady) + private void awaitPartialSimple(List nullableRecoverOkList, Participants participants, Runnable whenReady) { - Deps earlierWait = Deps.merge(nullableRecoverOkList, nullableRecoverOkList.size(), List::get, ok -> ok.earlierWait); - Deps earlierNoWait = Deps.merge(nullableRecoverOkList, nullableRecoverOkList.size(), List::get, ok -> ok.earlierNoWait); + Deps earlierWait = Deps.merge(nullableRecoverOkList, nullableRecoverOkList.size(), List::get, ok -> ok.simpleWait); + Deps earlierNoWait = Deps.merge(nullableRecoverOkList, nullableRecoverOkList.size(), List::get, ok -> ok.simpleNoWait); earlierWait = earlierWait.without(earlierNoWait); earlierWait = earlierWait.intersecting(participants); - awaitToFinish(awaitEarlier(node, earlierWait, HasDecidedExecuteAt).invokeIfSuccess(whenReady)); + awaitToFinish(awaitSimple(node, earlierWait, HasDecidedExecuteAt).invokeIfSuccess(whenReady)); } private static boolean supersedingRejects(List oks) @@ -555,7 +578,7 @@ private static Deps selectCoordinatorReplies(Id from, RecoverOk ok) if (ok == null) return null; - return ok.laterCoordRejects.with(id -> from.equals(id.node)); + return ok.supersedingCoordRejects.with(id -> from.equals(id.node)); } private void withCommittedDeps(LatestDeps.Merge merge, Timestamp executeAt, BiConsumer failureCallback, Consumer withDeps) @@ -568,10 +591,14 @@ private void withStableDeps(LatestDeps.Merge merge, Timestamp executeAt, BiConsu LatestDeps.withStable(adapter, node, executor, merge, Deps.NONE, scope, null, null, scope, ballot, txnId, executeAt, txn, failureCallback, withDeps); } - private void invalidate(SortedListMap recoverOks) + private void finishAndInvalidate(SortedListMap recoverOks) + { + invalidate(recoverOks, finishAndTakeCallback()); + } + + private void invalidate(SortedListMap recoverOks, BiConsumer callback) { Timestamp invalidateUntil = invalidateUntil(recoverOks); - BiConsumer callback = finishAndTakeCallback(); proposeInvalidate(node, executor, ballot, txnId, scope.homeKey(), (success, fail) -> { if (fail != null) callback.accept(null, fail); else commitInvalidate(invalidateUntil, callback); @@ -618,13 +645,13 @@ private void propose(Accept.Kind kind, Timestamp executeAt, LatestDeps.Merge mer private void retry(Timestamp executeAt, BiConsumer callback) { Ballot ballot = node.uniqueTimestamp(Ballot::fromValues); - Recover.recover(node, tracker.topologies(), ballot, txnId, txn, scope, executeAt, TinyEnumSet.contains(flags, FAST_PATH_DECIDED), reportTo, callback); + Recover.recover(node, tracker.topologies(), ballot, txnId, txn, scope, executeAt, isFastPathDecided(flags), reportTo, callback); } - AsyncChain awaitEarlier(Node node, Deps waitOn, Await.Until awaitUntil) + AsyncChain awaitSimple(Node node, Deps waitOn, Await.Until awaitUntil) { if (tracing != null) - tracing.trace(null, "awaiting earlier decisions: " + waitOn.txnIds()); + tracing.trace(null, "awaiting decisions: " + waitOn.txnIds()); long requireEpoch = waitOn.maxTxnId(txnId).epoch(); return node.withEpochAtLeast(requireEpoch, executor, () -> { @@ -633,7 +660,7 @@ AsyncChain awaitEarlier(Node node, Deps waitOn, Await.Until aw for (int i = 0 ; i < waitOn.txnIdCount() ; ++i) { TxnId awaitId = waitOn.txnId(i); - Invariants.require(awaitId.compareTo(recoverId) < 0); + Invariants.require(awaitId.compareTo(recoverId) < 0 || !awaitId.hasFastPath()); Participants participants = waitOn.participants(awaitId); Topologies topologies; @@ -651,7 +678,7 @@ AsyncChain awaitEarlier(Node node, Deps waitOn, Await.Until aw }); } - AsyncChain awaitLater(Node node, Deps waitOn, Await.Until awaitUntil, @Nullable Participants selfCoordVotes) + AsyncChain awaitSupersedingCoord(Node node, Deps waitOn, Await.Until awaitUntil, @Nullable Participants selfCoordVotes) { if (tracing != null) tracing.trace(null, "awaiting later decisions or recoveries: " + waitOn.txnIds()); diff --git a/accord-core/src/main/java/accord/coordinate/tracking/DurabilityTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/DurabilityTracker.java index 89eaab8911..95efbcecdd 100644 --- a/accord-core/src/main/java/accord/coordinate/tracking/DurabilityTracker.java +++ b/accord-core/src/main/java/accord/coordinate/tracking/DurabilityTracker.java @@ -26,7 +26,6 @@ import accord.local.durability.DurabilityService.SyncRemote; import accord.topology.Shard; import accord.topology.Topologies; -import accord.topology.Topology; import accord.utils.Invariants; import accord.utils.ReducingRangeMap; import accord.utils.SortedArrays.SortedArrayList; @@ -214,19 +213,8 @@ public Set excluding() public ReducingRangeMap results(Node.Id self) { - boolean endInclusive = false; - for (int i = 0 ; i < topologies.size() ; ++i) - { - Topology topology = topologies.get(i); - if (topology.ranges().isEmpty()) - continue; - - endInclusive = topology.ranges().get(0).endInclusive(); - break; - } - ReducingRangeMap result = null; - ReducingRangeMap.Builder builder = new ReducingRangeMap.Builder<>(endInclusive, trackers.length); + ReducingRangeMap.Builder builder = new ReducingRangeMap.Builder<>(trackers.length); for (int topologyIndex = 0 ; topologyIndex < topologies.size() ; ++topologyIndex) { for (int i = topologyOffset(topologyIndex), max = topologyOffset(topologyIndex + 1); i < max ; ++i) diff --git a/accord-core/src/main/java/accord/impl/AbstractReplayer.java b/accord-core/src/main/java/accord/impl/AbstractReplayer.java index 8c5c0f908e..5e97681ee3 100644 --- a/accord-core/src/main/java/accord/impl/AbstractReplayer.java +++ b/accord-core/src/main/java/accord/impl/AbstractReplayer.java @@ -34,7 +34,12 @@ import accord.primitives.SaveStatus; import accord.primitives.TxnId; import accord.utils.Invariants; +import accord.utils.UnhandledEnum; +import static accord.impl.AbstractReplayer.Replay.NONE; +import static accord.impl.AbstractReplayer.Replay.TO_BOTH; +import static accord.impl.AbstractReplayer.Replay.TO_COMMAND_STORE; +import static accord.impl.AbstractReplayer.Replay.TO_DATA_STORE; import static accord.local.RedundantStatus.Property.LOCALLY_DURABLE_TO_COMMAND_STORE; import static accord.local.RedundantStatus.Property.LOCALLY_DURABLE_TO_DATA_STORE; import static accord.primitives.SaveStatus.Applying; @@ -45,14 +50,41 @@ public abstract class AbstractReplayer implements Journal.Replayer { + // TODO (required): NON_DURABLE does not properly account for things like pre-bootstrap + public enum Mode { ALL, PART_NON_DURABLE, NON_DURABLE } + public enum Replay + { + // warning: behaviour depends on bit pattern of ordinals, so change with care + NONE, TO_COMMAND_STORE, TO_DATA_STORE, TO_BOTH; + + private static final Replay[] lookup = values(); + + public boolean includes(Replay replay) + { + return (ordinal() & replay.ordinal()) == replay.ordinal(); + } + public Replay atLeast(Replay that) + { + return lookup[ordinal() | that.ordinal()]; + } + public Replay atMost(Replay that) + { + return lookup[ordinal() & that.ordinal()]; + } + } + public final RedundantBefore redundantBefore; + public final Mode mode; public final TxnId minReplay; - protected AbstractReplayer(CommandStore commandStore, @Nullable TxnId minReplay) + protected AbstractReplayer(CommandStore commandStore, Mode mode, @Nullable TxnId minReplay) { this.redundantBefore = commandStore.unsafeGetRedundantBefore(); + this.mode = mode; Invariants.require(redundantBefore.ranges(Objects::nonNull).containsAll(commandStore.unsafeGetRangesForEpoch().all())); - this.minReplay = TxnId.noneIfNull(redundantBefore.foldl((b, v) -> TxnId.nonNullOrMin(v, b.maxBoundBoth(LOCALLY_DURABLE_TO_DATA_STORE, LOCALLY_DURABLE_TO_COMMAND_STORE)), minReplay, ignore -> false)); + if (mode != Mode.ALL) + minReplay = redundantBefore.foldl((b, v) -> TxnId.nonNullOrMin(v, replayBound(b)), minReplay, ignore -> false); + this.minReplay = TxnId.noneIfNull(minReplay); } protected boolean maybeShouldReplay(TxnId txnId) @@ -60,25 +92,47 @@ protected boolean maybeShouldReplay(TxnId txnId) return txnId.compareTo(minReplay) >= 0; } - protected boolean shouldReplay(TxnId txnId, StoreParticipants participants) + protected Replay shouldReplay(TxnId txnId, StoreParticipants participants) { Participants search = participants.route(); if (search == null) search = participants.hasTouched(); - return redundantBefore.foldl(search, (b, v, id) -> v || b.maxBoundBoth(LOCALLY_DURABLE_TO_COMMAND_STORE, LOCALLY_DURABLE_TO_DATA_STORE).compareTo(id) <= 0, false, txnId, i -> i); + switch (mode) + { + default: throw new UnhandledEnum(mode); + case ALL: return TO_BOTH; + case NON_DURABLE: return redundantBefore.foldl(search, (b, v, id) -> v.atMost(replay(b, id)), TO_BOTH, txnId, i -> false); + case PART_NON_DURABLE: return redundantBefore.foldl(search, (b, v, id) -> v.atLeast(replay(b, id)), NONE, txnId, i -> false); + } + } + + private static TxnId replayBound(RedundantBefore.Bounds bounds) + { + return bounds.maxBoundBoth(LOCALLY_DURABLE_TO_COMMAND_STORE, LOCALLY_DURABLE_TO_DATA_STORE); } - protected void initialiseState(SafeCommandStore safeStore, TxnId txnId) + private static Replay replay(RedundantBefore.Bounds bounds, TxnId txnId) + { + Replay replay = NONE; + if (bounds.maxBound(LOCALLY_DURABLE_TO_COMMAND_STORE).compareTo(txnId) <= 0) + replay = TO_COMMAND_STORE; + if (bounds.maxBound(LOCALLY_DURABLE_TO_DATA_STORE).compareTo(txnId) <= 0) + replay = replay.atLeast(TO_DATA_STORE); + return replay; + } + + protected void replay(SafeCommandStore safeStore, TxnId txnId, Replay replay) { SafeCommand safeCommand = safeStore.unsafeGet(txnId); { Command command = safeCommand.current(); if (command.saveStatus().compareTo(SaveStatus.Stable) >= 0 && command.saveStatus().compareTo(PreApplied) <= 0) { - Commands.maybeExecute(safeStore, safeCommand, command, false, true); + if (replay.includes(TO_COMMAND_STORE)) + Commands.maybeExecute(safeStore, safeCommand, command, false, true); } else if (command.saveStatus().compareTo(Applying) >= 0 && command.saveStatus().compareTo(TruncatedApplyWithOutcome) <= 0) { - if (command.txnId().is(Write)) + if (command.txnId().is(Write) && replay.includes(TO_DATA_STORE)) { Commands.applyChain(safeStore, command) .begin(safeStore.agent()); @@ -86,7 +140,10 @@ else if (command.saveStatus().compareTo(Applying) >= 0 && command.saveStatus().c else Invariants.expect(command.hasBeen(Applied), "%s is Applying but is not a Write transaction", txnId); } } - safeCommand.update(safeStore, safeCommand.current(), true); - safeStore.notifyListeners(safeCommand, null); + if (replay.includes(Replay.TO_COMMAND_STORE)) + { + safeCommand.update(safeStore, safeCommand.current(), true); + safeStore.notifyListeners(safeCommand, null); + } } } diff --git a/accord-core/src/main/java/accord/impl/DefaultLocalListeners.java b/accord-core/src/main/java/accord/impl/DefaultLocalListeners.java index 9081b6b907..d83a550222 100644 --- a/accord-core/src/main/java/accord/impl/DefaultLocalListeners.java +++ b/accord-core/src/main/java/accord/impl/DefaultLocalListeners.java @@ -18,10 +18,13 @@ package accord.impl; +import java.util.ArrayList; import java.util.Arrays; import java.util.EnumMap; import java.util.Enumeration; import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentHashMap; import java.util.function.BiFunction; @@ -43,6 +46,8 @@ import accord.utils.Invariants; import accord.utils.btree.BTree; import accord.utils.btree.BTreeRemoval; +import accord.utils.btree.BulkIterator; +import accord.utils.btree.UpdateFunction; import static accord.utils.ArrayBuffers.cachedAny; import static accord.utils.ArrayBuffers.cachedTxnIds; @@ -659,7 +664,10 @@ public boolean hasNext() @Override public TxnListener next() { - return new TxnListener(buffer[0], new TxnId(cur), cur.await); + if (!hasNext()) + throw new NoSuchElementException(); + + return new TxnListener(buffer[bufferIndex++], new TxnId(cur), cur.await); } }; }; @@ -726,9 +734,48 @@ public boolean hasNext() @Override public Registered next() { - return (Registered) buffer[0]; + if (!hasNext()) + throw new NoSuchElementException(); + return (Registered) buffer[bufferIndex++]; } }; }; } + + public void restore(List listeners) + { + if (listeners.isEmpty()) + return; + + if (!BTree.isEmpty(txnListeners)) + throw new IllegalStateException("Restore only supported if uninitialised"); + + listeners.sort((a, b) -> { + int c = a.waitingOn.compareTo(b.waitingOn); + if (c == 0) c = a.awaitingStatus.compareTo(b.awaitingStatus); + if (c == 0) c = a.waiter.compareTo(b.waiter); + return c; + }); + + List build = new ArrayList<>(); + int li = 0; + while (li < listeners.size()) + { + TxnListener l = listeners.get(li); + TxnListeners ls = new TxnListeners(l.waitingOn, l.awaitingStatus); + build.add(ls); + ls.add(l.waiter); + while (++li < listeners.size() && (l = listeners.get(li)).waitingOn.equals(ls) && l.awaitingStatus == ls.await) + ls.add(l.waiter); + } + txnListeners = BTree.build(BulkIterator.of(build.iterator()), build.size(), UpdateFunction.noOp()); + } + + public List snapshot() + { + List snapshot = new ArrayList<>(BTree.size(txnListeners)); + for (TxnListener listener : txnListeners()) + snapshot.add(listener); + return snapshot; + } } diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java index 0e3b7956b6..f0eb6611a8 100644 --- a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java +++ b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java @@ -39,7 +39,9 @@ import java.util.function.Function; import java.util.function.Predicate; import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import accord.primitives.*; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,15 +50,22 @@ import accord.api.DataStore; import accord.api.Journal; import accord.api.LocalListeners; +import accord.api.LocalListeners.TxnListener; import accord.api.ProgressLog; import accord.api.RoutingKey; +import accord.impl.cfr.IdEntry; +import accord.impl.cfr.InMemoryRangeSummaryIndex; +import accord.impl.cfr.LoadListener; import accord.impl.progresslog.DefaultProgressLog; +import accord.impl.progresslog.TxnState; import accord.local.Cleanup; import accord.local.Command; import accord.local.CommandStore; -import accord.local.CommandStores.RangesForEpoch; import accord.local.CommandSummaries; +import accord.local.CommandSummaries.Summary; +import accord.local.CommandSummaries.SummaryLoader; import accord.local.Commands; +import accord.local.LoadKeysFor; import accord.local.MaxDecidedRX; import accord.local.NodeCommandStoreService; import accord.local.PreLoadContext; @@ -69,20 +78,8 @@ import accord.local.cfk.CommandsForKey; import accord.local.cfk.SafeCommandsForKey; import accord.local.cfk.Serialize; -import accord.primitives.AbstractRanges; -import accord.primitives.AbstractUnseekableKeys; -import accord.primitives.PartialDeps; -import accord.primitives.Participants; -import accord.primitives.Ranges; -import accord.primitives.Routable.Domain; -import accord.primitives.RoutableKey; -import accord.primitives.Route; -import accord.primitives.Status; import accord.primitives.Status.Durability.HasOutcome; -import accord.primitives.Timestamp; import accord.primitives.Txn.Kind.Kinds; -import accord.primitives.TxnId; -import accord.primitives.Unseekables; import accord.utils.Invariants; import accord.utils.async.AsyncChain; import accord.utils.async.AsyncChains; @@ -91,8 +88,11 @@ import accord.utils.async.Cancellable; import org.agrona.collections.ObjectHashSet; +import static accord.api.ProtocolModifiers.RangeSpec.isEndInclusive; +import static accord.api.ProtocolModifiers.RangeSpec.isStartInclusive; import static accord.local.Cleanup.Input.FULL; import static accord.local.LoadKeys.NONE; +import static accord.local.LoadKeysFor.RECOVERY; import static accord.local.LoadKeysFor.WRITE; import static accord.local.RedundantStatus.Coverage.ALL; import static accord.local.StoreParticipants.Filter.LOAD; @@ -100,14 +100,11 @@ import static accord.primitives.Routable.Domain.Range; import static accord.primitives.Routables.Slice.Minimal; import static accord.primitives.SaveStatus.Applying; -import static accord.primitives.SaveStatus.Erased; -import static accord.primitives.SaveStatus.NotDefined; +import static accord.primitives.SaveStatus.Invalidated; import static accord.primitives.SaveStatus.ReadyToExecute; -import static accord.primitives.SaveStatus.Vestigial; import static accord.primitives.Status.Applied; import static accord.primitives.Status.Committed; import static accord.primitives.Status.Durability.HasOutcome.Universal; -import static accord.primitives.Status.Durability.NotDurable; import static accord.primitives.Status.Stable; import static accord.primitives.Status.Truncated; import static accord.primitives.Txn.Kind.EphemeralRead; @@ -125,7 +122,9 @@ public static class Snapshot extends AsyncResults.SettableResult private final long id = nextId.incrementAndGet(); private final NavigableMap commandsForKey = new TreeMap<>(); - private final TreeMap rangeCommands = new TreeMap<>(); + private List commandsForRanges; + private List listeners; + private List progressLog; private int waitingForCfk; private Snapshot(){} @@ -135,11 +134,11 @@ public void restore(InMemoryCommandStore commandStore) for (Map.Entry e : commandsForKey.entrySet()) commandStore.commandsForKey.computeIfAbsent(e.getKey(), GlobalCommandsForKey::new).value(Serialize.fromBytes(e.getKey(), e.getValue())); - for (Map.Entry e : rangeCommands.entrySet()) - { - RangeCommand rangeCommand = commandStore.rangeCommands.computeIfAbsent(e.getKey(), RangeCommand::new); - rangeCommand.ranges = e.getValue(); - } + commandStore.progressLog.clear(); + ((DefaultProgressLog)commandStore.progressLog).restore(null, progressLog); + ((DefaultLocalListeners)commandStore.listeners).restore(listeners); + commandStore.commandsForRanges.restore(commandsForRanges); + commandStore.commandsForRanges.prune(commandStore); } void saveCallback(CommandsForKey cfk) @@ -164,9 +163,10 @@ private void save(CommandsForKey cfk) public static AsyncResult snapshot(InMemoryCommandStore commandStore) { Snapshot snapshot = new Snapshot(); - for (Map.Entry e : commandStore.rangeCommands.entrySet()) - snapshot.rangeCommands.put(e.getKey(), e.getValue().ranges); + snapshot.listeners = ((DefaultLocalListeners)commandStore.listeners).snapshot(); + snapshot.progressLog = ((DefaultProgressLog)commandStore.progressLog).snapshot(); + snapshot.commandsForRanges = commandStore.commandsForRanges.snapshot(); for (Map.Entry e : commandStore.commandsForKey.entrySet()) { GlobalCommandsForKey global = e.getValue(); @@ -187,12 +187,31 @@ public static AsyncResult snapshot(InMemoryCommandStore commandStore) } } - final NavigableMap commands = new TreeMap<>(); + public static class CommandsForRangeLoad implements Cancellable + { + public final SummaryLoader loader; + public final TreeMap loaded; + public final Cancellable unregister; + + public CommandsForRangeLoad(SummaryLoader loader, TreeMap loaded, Cancellable unregister) + { + this.loader = loader; + this.loaded = loaded; + this.unregister = unregister; + } + + @Override + public void cancel() + { + unregister.cancel(); + } + } + + protected final NavigableMap commands = new TreeMap<>(); final NavigableMap commandsByExecuteAt = new TreeMap<>(); private final NavigableMap commandsForKey = new TreeMap<>(); - private final TreeMap rangeCommands = new TreeMap<>(); - protected Timestamp maxRedundant = Timestamp.NONE; + protected final InMemoryRangeSummaryIndex commandsForRanges; private InMemorySafeStore current; private final Journal journal; @@ -201,6 +220,7 @@ public InMemoryCommandStore(int id, NodeCommandStoreService node, Agent agent, D { super(id, node, agent, store, progressLogFactory, listenersFactory, epochUpdateHolder); this.journal = journal; + this.commandsForRanges = new InMemoryRangeSummaryIndex(); progressLog.unsafeStart(); } @@ -351,14 +371,14 @@ protected void ensureDurable(Ranges ranges, RedundantBefore onCommandStoreDurabl } @Override - protected void updatedRedundantBefore(SafeCommandStore safeStore, RedundantBefore added) + protected void upsertedRedundantBefore(SafeCommandStore safeStore, RedundantBefore added) { InMemorySafeStore inMemorySafeStore = (InMemorySafeStore) safeStore; for (int i = 0 ; i < added.size() ; ++i) { if (added.valueAt(i) != null) { - commandsForKey.subMap(added.startAt(i), !added.inclusiveEnds(), added.startAt(i + 1), added.inclusiveEnds()).forEach((forKey, forValue) -> { + commandsForKey.subMap(added.startAt(i), isStartInclusive(), added.startAt(i + 1), isEndInclusive()).forEach((forKey, forValue) -> { if (!forValue.isEmpty()) { InMemorySafeCommandsForKey safeCfk = forValue.createSafeReference(); @@ -375,7 +395,8 @@ protected void updatedRedundantBefore(SafeCommandStore safeStore, RedundantBefor for (TxnId txnId : clearing) { GlobalCommand globalCommand = commands.get(txnId); - Invariants.require(globalCommand != null && !globalCommand.isEmpty()); + if (globalCommand == null) + continue; // now we restore contents from snapshot, we might repopulate older items Command command = globalCommand.value(); StoreParticipants participants = command.participants().filter(LOAD, safeStore, txnId, command.executeAtIfKnown()); Cleanup cleanup = Cleanup.shouldCleanup(FULL, txnId, command.executeAtIfKnown(), command.saveStatus(), command.durability(), participants, unsafeGetRedundantBefore(), durableBefore()); @@ -386,7 +407,7 @@ protected void updatedRedundantBefore(SafeCommandStore safeStore, RedundantBefor || !Route.isFullRoute(command.route())))); } } - super.updatedRedundantBefore(safeStore, added); + super.upsertedRedundantBefore(safeStore, added); } @Override @@ -394,41 +415,28 @@ public void markShardDurable(SafeCommandStore safeStore, TxnId syncId, Ranges ra { super.markShardDurable(safeStore, syncId, ranges, level); if (level == Universal) - markShardDurable(syncId, ranges); + commandsForRanges.prune(syncId, ranges, safeStore.redundantBefore()); } - private void markShardDurable(TxnId syncId, Ranges ranges) + @Override + protected void markExclusiveSyncPointLocallyApplied(SafeCommandStore safeStore, TxnId syncId, Ranges ranges, SaveStatus prevStatus) { - rangeCommands.computeIfAbsent(syncId, RangeCommand::new).add(ranges); - rangeCommands.headMap(syncId, false).entrySet().removeIf(tx -> { - Ranges newRanges = tx.getValue().ranges.without(ranges); - if (!newRanges.isEmpty()) - { - tx.getValue().ranges = newRanges; - return false; - } - else - { - GlobalCommand global = commands.get(tx.getKey()); - if (global != null) - maxRedundant = Timestamp.nonNullOrMax(maxRedundant, global.value().executeAt()); - return true; - } - }); + super.markExclusiveSyncPointLocallyApplied(safeStore, syncId, ranges, prevStatus); + commandsForRanges.prune(syncId, ranges, safeStore.redundantBefore()); } - protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges, + protected InMemorySafeStore createSafeStore(PreLoadContext context, CommandsForRangeLoad cfrLoad, Map commands, Map commandsForKeys) { - return new InMemorySafeStore(this, ranges, context, commands, commandsForKeys); + return new InMemorySafeStore(this, context, cfrLoad, commands, commandsForKeys); } protected void onRead(Command current) {} protected void onWrite(Command current) {} protected void onRead(CommandsForKey current) {} - protected final InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges) + protected final InMemorySafeStore createSafeStore(PreLoadContext context, CommandsForRangeLoad cfrLoad) { Map commands = new HashMap<>(); Map commandsForKey = new HashMap<>(); @@ -442,9 +450,9 @@ protected final InMemorySafeStore createSafeStore(PreLoadContext context, Ranges for (RoutingKey key : (AbstractUnseekableKeys)unseekables) commandsForKey.put(key, commandsForKey(key).createSafeReference()); } - else + else if (context.loadKeysFor() != WRITE) { - CommandSummaries.SummaryLoader loader = CommandSummaries.SummaryLoader.loader(unsafeGetRedundantBefore(), unsafeGetMaxDecidedRX(), context); + SummaryLoader loader = SummaryLoader.loader(unsafeGetRedundantBefore(), unsafeGetMaxDecidedRX(), context); for (GlobalCommandsForKey global : this.commandsForKey.values()) { if (!unseekables.contains(global.key)) @@ -460,14 +468,14 @@ protected final InMemorySafeStore createSafeStore(PreLoadContext context, Ranges } } - return createSafeStore(context, ranges, commands, commandsForKey); + return createSafeStore(context, cfrLoad, commands, commandsForKey); } - public SafeCommandStore beginOperation(PreLoadContext context) + public SafeCommandStore beginOperation(PreLoadContext context, @Nullable CommandsForRangeLoad cfrLoad) { if (current != null) throw illegalState("Another operation is in progress or it's store was not cleared"); - current = createSafeStore(context, rangesForEpoch); + current = createSafeStore(context, cfrLoad); updateRangesForEpoch(current); return current; } @@ -492,9 +500,9 @@ public void completeOperation(SafeCommandStore store) } } - protected T executeInContext(InMemoryCommandStore commandStore, PreLoadContext preLoadContext, Function function) + protected T executeInContext(InMemoryCommandStore commandStore, PreLoadContext preLoadContext, @Nullable CommandsForRangeLoad cfrLoad, Function function) { - SafeCommandStore safeStore = commandStore.beginOperation(preLoadContext); + SafeCommandStore safeStore = commandStore.beginOperation(preLoadContext, cfrLoad); try { return function.apply(safeStore); @@ -506,11 +514,11 @@ protected T executeInContext(InMemoryCommandStore commandStore, PreLoadConte } } - protected void executeInContext(InMemoryCommandStore commandStore, PreLoadContext context, Function function, BiConsumer callback) + protected void executeInContext(InMemoryCommandStore commandStore, PreLoadContext context, @Nullable CommandsForRangeLoad cfrLoad, Function function, BiConsumer callback) { try { - T result = executeInContext(commandStore, context, function); + T result = executeInContext(commandStore, context, cfrLoad, function); callback.accept(result, null); } catch (Throwable t) @@ -647,12 +655,13 @@ public static class InMemorySafeStore extends AbstractSafeCommandStore commands; private final Map commandsForKey; + private final CommandsForRangeLoad cfrLoad; private final Set hasLoaded = new ObjectHashSet<>(); private ByTxnIdSnapshot commandsForRanges; public InMemorySafeStore(InMemoryCommandStore commandStore, - RangesForEpoch ranges, PreLoadContext context, + CommandsForRangeLoad cfrLoad, Map commands, Map commandsForKey) { @@ -660,6 +669,7 @@ public InMemorySafeStore(InMemoryCommandStore commandStore, this.commands = commands; this.commandsForKey = commandsForKey; + this.cfrLoad = cfrLoad; for (InMemorySafeCommand cmd : commands.values()) { if (cmd.isUnset()) cmd.uninitialised(); @@ -668,6 +678,8 @@ public InMemorySafeStore(InMemoryCommandStore commandStore, { if (cfk.isUnset()) cfk.initialize(); } + if (cfrLoad != null) + cfrLoad.cancel(); } @Override @@ -757,20 +769,9 @@ protected InMemorySafeCommandsForKey add(InMemorySafeCommandsForKey cfk, InMemor } @Override - protected void update(Command prev, Command updated, boolean force) + public void updateCommandsForRanges(Command prev, Command updated, boolean force) { - super.update(prev, updated, force); - - TxnId txnId = updated.txnId(); - if (txnId.domain() != Domain.Range) - return; - - // TODO (testing): consider removing if erased - if (updated.saveStatus() == Erased || updated.saveStatus() == Vestigial) - return; - - commandStore().rangeCommands.computeIfAbsent(txnId, RangeCommand::new) - .update(((AbstractRanges)updated.participants().stillTouches()).toRanges()); + commandStore().commandsForRanges.update(prev, updated, force); } @Override @@ -799,6 +800,7 @@ public NodeCommandStoreService node() public void postExecute() { + commandStore().commandsForRanges.tryDrainPendingEdits(); super.postExecute(); commands.values().forEach(c -> { if (c == null || c.current() == null) @@ -829,27 +831,16 @@ CommandSummaries commandsForRanges() return commandsForRanges; Invariants.require(context.loadKeysFor() != WRITE); + // TODO (now): reuse existing loader MaxDecidedRX maxDecidedRX = commandStore().unsafeGetMaxDecidedRX(); SummaryLoader loader = SummaryLoader.loader(redundantBefore(), maxDecidedRX, context); - TreeMap summaries = new TreeMap<>(); - for (RangeCommand rangeCommand : commandStore().rangeCommands.values()) - { - GlobalCommand global = commandStore().commands.get(rangeCommand.txnId); - Command command = global == null ? null : global.value(); - if (!loader.isMaybeRelevant(rangeCommand.txnId)) - continue; - - Summary summary; - if (command == null) summary = loader.ifRelevant(rangeCommand.txnId, rangeCommand.txnId, NotDefined, NotDurable, rangeCommand.ranges, null); - else summary = loader.ifRelevant(command); - if (summary != null) - { - summaries.put(summary.plainTxnId(), summary); - loader.maybeRecordFutureRx(summary); - } - } - return commandsForRanges = () -> summaries; + TreeMap loaded = new TreeMap<>(); + commandStore().commandsForRanges.populateMinFutureRx(loader); + commandStore().commandsForRanges.search(loader, loaded::put, null); + if (cfrLoad != null) + loaded.putAll(cfrLoad.loaded); + return commandsForRanges = () -> loaded; } private boolean visitForKey(Unseekables keysOrRanges, Predicate forEach) @@ -870,9 +861,9 @@ private void visitForKey(Unseekables keysOrRanges, Timestamp started visitForKey(keysOrRanges, cfk -> { cfk.visit(startedBefore, testKind, visitor, p1, p2); return true; }); } - public boolean visitForKey(Unseekables keysOrRanges, TxnId testTxnId, Kinds testKind, TestStartedAt testStartedAt, Timestamp testStartedAtTimestamp, ComputeIsDep computeIsDep, AllCommandVisitor visit) + public boolean visitForKey(Unseekables keysOrRanges, TxnId testTxnId, Kinds testKind, SupersedingCommandVisitor visit) { - return visitForKey(keysOrRanges, cfk -> cfk.visit(testTxnId, testKind, testStartedAt, testStartedAtTimestamp, computeIsDep, null, visit)); + return visitForKey(keysOrRanges, cfk -> cfk.visit(testTxnId, testKind, visit)); } @Override @@ -883,10 +874,10 @@ public void visit(Unseekables keysOrRanges, Timestamp startedBefore, } @Override - public boolean visit(Unseekables keysOrRanges, TxnId testTxnId, Kinds testKind, TestStartedAt testStartedAt, Timestamp testStartedAtTimestamp, ComputeIsDep computeIsDep, AllCommandVisitor visit) + public boolean visit(Unseekables keysOrRanges, TxnId testTxnId, Kinds testKind, SupersedingCommandVisitor visit) { - return visitForKey(keysOrRanges, testTxnId, testKind, testStartedAt, testStartedAtTimestamp, computeIsDep, visit) - && commandsForRanges().visit(keysOrRanges, testTxnId, testKind, testStartedAt, testStartedAtTimestamp, computeIsDep, visit); + return visitForKey(keysOrRanges, testTxnId, testKind, visit) + && commandsForRanges().visit(keysOrRanges, testTxnId, testKind, visit); } @Override @@ -934,6 +925,25 @@ public void updateExclusiveSyncPoint(Command prev, Command updated, boolean forc } } + protected CommandsForRangeLoad cfrLoad(PreLoadContext context) + { + if (context.loadKeysFor() != LoadKeysFor.RECOVERY) + return null; + + SummaryLoader loader = SummaryLoader.loader(unsafeGetRedundantBefore(), unsafeGetMaxDecidedRX(), context); + commandsForRanges.populateMinFutureRx(loader); + TreeMap loaded = new TreeMap<>(); + commandsForRanges.search(loader, null, txnId -> { + Invariants.require(loader.loadKeysFor() == RECOVERY); + Command command = commands.get(txnId).value(); + Summary summary = loader.ifRelevant(command); + // TODO (expected): prune implied invalidations from index, so no need to special case + if (summary == null) Invariants.require(command.saveStatus() == Invalidated); + else loaded.put(summary.plainTxnId(), summary); + }); + return new CommandsForRangeLoad(loader, loaded, commandsForRanges.registerListener(new LoadListener(loader, loaded))); + } + public static class Synchronized extends InMemoryCommandStore { Runnable active; @@ -961,13 +971,17 @@ private synchronized void maybeRun() activeThread = null; } - private Cancellable enqueueAndRun(Runnable runnable) + private Cancellable enqueueAndRun(Runnable runnable, @Nullable Cancellable ifCancelled) { boolean result = queue.add(runnable); if (!result) throw illegalState("could not add item to queue"); maybeRun(); - return () -> queue.remove(runnable); + return () -> { + queue.remove(runnable); + if (ifCancelled != null) + ifCancelled.cancel(); + }; } @Override @@ -990,7 +1004,8 @@ public AsyncChain chain(PreLoadContext context, Function callback) { - return enqueueAndRun(() -> executeInContext(InMemoryCommandStore.Synchronized.this, context, function, callback)); + CommandsForRangeLoad cfrLoad = cfrLoad(context); + return enqueueAndRun(() -> executeInContext(InMemoryCommandStore.Synchronized.this, context, cfrLoad, function, callback), cfrLoad); } }; } @@ -1001,7 +1016,7 @@ public void shutdown() {} @Override public void execute(Runnable run) { - enqueueAndRun(run); + enqueueAndRun(run, null); } } @@ -1048,7 +1063,9 @@ public AsyncChain chain(PreLoadContext context, Consumer AsyncChain chain(PreLoadContext context, Function function) { - return chain(() -> executeInContext(SingleThread.this, context, function)); + // TODO (expected): must unregister if chain is cancelled; should also only register when start() called + CommandsForRangeLoad cfrLoad = cfrLoad(context); + return chain(() -> executeInContext(SingleThread.this, context, cfrLoad, function)); } @Override @@ -1069,12 +1086,12 @@ public static class Debug extends SingleThread class DebugSafeStore extends InMemorySafeStore { public DebugSafeStore(InMemoryCommandStore commandStore, - RangesForEpoch ranges, PreLoadContext context, + CommandsForRangeLoad cfrLoad, Map commands, Map commandsForKey) { - super(commandStore, ranges, context, commands, commandsForKey); + super(commandStore, context, cfrLoad, commands, commandsForKey); } @Override @@ -1098,9 +1115,9 @@ public Debug(int id, NodeCommandStoreService time, Agent agent, DataStore store, } @Override - protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges, Map commands, Map commandsForKeys) + protected InMemorySafeStore createSafeStore(PreLoadContext context, CommandsForRangeLoad cfrLoad, Map commands, Map commandsForKeys) { - return new DebugSafeStore(this, ranges, context, commands, commandsForKeys); + return new DebugSafeStore(this, context, cfrLoad, commands, commandsForKeys); } } @@ -1211,9 +1228,9 @@ public void unsafeClearForTesting() commands.clear(); commandsByExecuteAt.clear(); commandsForKey.clear(); - rangeCommands.clear(); + commandsForRanges.clear(); progressLog.clear(); - unsafeSetRejectBefore(new RejectBefore()); + unsafeSetRejectBefore(RejectBefore.EMPTY); hasResumedBootstraps = false; } @@ -1229,16 +1246,17 @@ private static class CommandReplayer extends AbstractReplayer private CommandReplayer(InMemoryCommandStore commandStore) { // TODO (required): we shouldn't be providing TxnId.NONE here, we need to standardise on querying journal for data missing from InMemoryCommandStore - super(commandStore, TxnId.NONE); + super(commandStore, Mode.PART_NON_DURABLE, TxnId.NONE); this.commandStore = commandStore; } - private AsyncChain apply(Command command) + private AsyncChain apply(Command command, Replay replay) { return AsyncChains.success(commandStore.executeInContext(commandStore, PreLoadContext.contextFor(command.txnId(), "Replay"), + null, (SafeCommandStore safeStore) -> { - initialiseState(safeStore, command.txnId()); + super.replay(safeStore, command.txnId(), replay); return null; })); } @@ -1271,10 +1289,14 @@ public AsyncChain replay(TxnId txnId) } } - if (command == null || !maybeShouldReplay(txnId) || !shouldReplay(txnId, command.participants())) + if (command == null || !maybeShouldReplay(txnId)) + return AsyncChains.success(null); + + Replay replay = shouldReplay(txnId, command.participants()); + if (replay == Replay.NONE) return AsyncChains.success(null); - return apply(command); + return apply(command, replay); } } } diff --git a/accord-core/src/main/java/accord/impl/RangeIntervalComparators.java b/accord-core/src/main/java/accord/impl/RangeIntervalComparators.java new file mode 100644 index 0000000000..5bb124e915 --- /dev/null +++ b/accord-core/src/main/java/accord/impl/RangeIntervalComparators.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl; + +import java.util.Comparator; +import java.util.function.Function; + +import accord.api.RoutingKey; +import accord.primitives.Range; +import accord.utils.AsymmetricComparator; +import accord.utils.SymmetricComparator; +import accord.utils.btree.IntervalBTree; + +import static accord.utils.btree.IntervalBTree.InclusiveEndHelper.endWithStart; +import static accord.utils.btree.IntervalBTree.InclusiveEndHelper.keyEndWithStart; +import static accord.utils.btree.IntervalBTree.InclusiveEndHelper.keyStartWithEnd; +import static accord.utils.btree.IntervalBTree.InclusiveEndHelper.keyStartWithStart; +import static accord.utils.btree.IntervalBTree.InclusiveEndHelper.startWithEnd; +import static accord.utils.btree.IntervalBTree.InclusiveEndHelper.startWithStart; + +public class RangeIntervalComparators +{ + public static class InclusiveEndEntryComparators implements IntervalBTree.IntervalComparators + { + final Function get; + final Comparator compareId; + + public InclusiveEndEntryComparators(Function get, Comparator compareId) + { + this.get = get; + this.compareId = compareId; + } + + @Override public Comparator totalOrder() + { + return (a, b) -> { + int c = get.apply(a).compare(get.apply(b)); + if (c == 0) c = compareId.compare(a, b); + return c; + }; + } + @Override public Comparator endWithEndSorter() { return (a, b) -> get.apply(a).end().compareTo(get.apply(b).end()); } + @Override public SymmetricComparator startWithStartSeeker() { return (a, b) -> startWithStart(get.apply(a).start().compareTo(get.apply(b).start())); } + @Override public SymmetricComparator startWithEndSeeker() { return (a, b) -> startWithEnd(get.apply(a).start().compareTo(get.apply(b).end())); } + @Override public SymmetricComparator endWithStartSeeker() { return (a, b) -> endWithStart(get.apply(a).end().compareTo(get.apply(b).start())); } + } + + public static class InclusiveEndWithKeyComparators implements IntervalBTree.WithIntervalComparators + { + final Function get; + + public InclusiveEndWithKeyComparators(Function get) + { + this.get = get; + } + + @Override public AsymmetricComparator startWithStartSeeker() { return (a, b) -> keyStartWithStart(a.compareTo(get.apply(b).start())); } + @Override public AsymmetricComparator startWithEndSeeker() { return (a, b) -> keyStartWithEnd(a.compareTo(get.apply(b).end())); } + @Override public AsymmetricComparator endWithStartSeeker() { return (a, b) -> keyEndWithStart(a.compareTo(get.apply(b).start())); } + } + + public static class InclusiveEndWithRangeComparators implements IntervalBTree.WithIntervalComparators + { + final Function get; + + public InclusiveEndWithRangeComparators(Function get) + { + this.get = get; + } + + @Override public AsymmetricComparator startWithStartSeeker() { return (a, b) -> startWithStart(a.start().compareTo(get.apply(b).start())); } + @Override public AsymmetricComparator startWithEndSeeker() { return (a, b) -> startWithEnd(a.start().compareTo(get.apply(b).end())); } + @Override public AsymmetricComparator endWithStartSeeker() { return (a, b) -> endWithStart(a.end().compareTo(get.apply(b).start())); } + } + +} diff --git a/accord-core/src/main/java/accord/impl/cfr/IdEntry.java b/accord-core/src/main/java/accord/impl/cfr/IdEntry.java new file mode 100644 index 0000000000..7a0ca474be --- /dev/null +++ b/accord-core/src/main/java/accord/impl/cfr/IdEntry.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.cfr; + +import accord.primitives.Range; +import accord.primitives.Ranges; +import accord.primitives.SaveStatus; +import accord.primitives.Status; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.utils.Invariants; + +public abstract class IdEntry extends TxnId +{ + public static class SerializerSupport + { + public static IdSingleEntry create(TxnId txnId, int encoded, Range range) + { + IdSingleEntry result = new IdSingleEntry(txnId, range); + result.encoded = encoded; + return result; + } + + public static IdMultiEntry create(TxnId txnId, int encoded, Ranges ranges) + { + IdMultiEntry result = new IdMultiEntry(txnId, ranges); + result.encoded = encoded; + return result; + } + } + + static final int SAVE_SATUS_SHIFT = Status.Durability.ENCODING_BITS; + static final int EXECUTE_AT_BIT = 1 << (SaveStatus.ENCODING_BITS + SAVE_SATUS_SHIFT); + + int encoded; + + TxnId plainTxnId() + { + return new TxnId(this); + } + + Status.Durability durability() + { + return Status.Durability.forEncoded(encoded & Status.Durability.ENCODING_MASK); + } + + SaveStatus saveStatus() + { + return SaveStatus.forOrdinal(saveStatusOrdinal()); + } + + int saveStatusOrdinal() + { + return (encoded >>> SAVE_SATUS_SHIFT) & SaveStatus.ENCODING_MASK; + } + + Timestamp maybeExecuteAt() + { + if ((encoded & EXECUTE_AT_BIT) != 0) + return plainTxnId(); + return null; + } + + public IdEntry(TxnId txnId) + { + super(txnId); + } + + abstract IdEntry copy(); + + abstract Ranges ranges(); + + public boolean update(SaveStatus saveStatus, Status.Durability durability, Timestamp executeAt) + { + int newEncoded = (saveStatus.ordinal() << SAVE_SATUS_SHIFT) | durability.encoded() | (super.equals(executeAt) ? EXECUTE_AT_BIT : 0); + if (encoded == newEncoded) + return false; + encoded = newEncoded; + Invariants.require(durability() == durability); + Invariants.require(saveStatus() == saveStatus); + return true; + } + + public final int encoded() + { + return encoded; + } +} diff --git a/accord-core/src/main/java/accord/impl/cfr/IdMultiEntry.java b/accord-core/src/main/java/accord/impl/cfr/IdMultiEntry.java new file mode 100644 index 0000000000..e5bd952c75 --- /dev/null +++ b/accord-core/src/main/java/accord/impl/cfr/IdMultiEntry.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.cfr; + +import accord.primitives.Ranges; +import accord.primitives.TxnId; + +public class IdMultiEntry extends IdEntry +{ + public final Ranges ranges; + + public IdMultiEntry(TxnId txnId, Ranges ranges) + { + super(txnId); + this.ranges = ranges; + } + + @Override + IdEntry copy() + { + IdMultiEntry copy = new IdMultiEntry(this, ranges); + copy.encoded = encoded; + return copy; + } + + @Override + Ranges ranges() + { + return ranges; + } +} diff --git a/accord-core/src/main/java/accord/impl/cfr/IdSingleEntry.java b/accord-core/src/main/java/accord/impl/cfr/IdSingleEntry.java new file mode 100644 index 0000000000..4ec8f33128 --- /dev/null +++ b/accord-core/src/main/java/accord/impl/cfr/IdSingleEntry.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.cfr; + +import accord.primitives.Range; +import accord.primitives.Ranges; +import accord.primitives.TxnId; + +public class IdSingleEntry extends IdEntry implements RangeEntry +{ + public final Range range; + + public IdSingleEntry(TxnId txnId, Range range) + { + super(txnId); + this.range = range; + } + + @Override + public Range range() + { + return range; + } + + @Override + public IdEntry id() + { + return this; + } + + @Override + IdEntry copy() + { + accord.impl.cfr.IdSingleEntry copy = new accord.impl.cfr.IdSingleEntry(this, range); + copy.encoded = encoded; + return copy; + } + + @Override + Ranges ranges() + { + // TODO (desired): introduce an interface for iterating ranges for providing to e.g. foldl, so we can avoid wrapping each time + return Ranges.of(range); + } +} diff --git a/accord-core/src/main/java/accord/impl/cfr/InMemoryRangeSummaryIndex.java b/accord-core/src/main/java/accord/impl/cfr/InMemoryRangeSummaryIndex.java new file mode 100644 index 0000000000..693b880f8a --- /dev/null +++ b/accord-core/src/main/java/accord/impl/cfr/InMemoryRangeSummaryIndex.java @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.cfr; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.Consumer; + +import javax.annotation.Nullable; + +import com.google.common.collect.ImmutableList; + +import accord.api.RoutingKey; +import accord.local.Command; +import accord.local.CommandStore; +import accord.local.CommandSummaries.Summary; +import accord.local.CommandSummaries.SummaryLoader; +import accord.local.CommandSummaries.Relevance; +import accord.local.RedundantBefore; +import accord.primitives.AbstractRanges; +import accord.primitives.AbstractUnseekableKeys; +import accord.primitives.Participants; +import accord.primitives.Range; +import accord.primitives.Ranges; +import accord.primitives.SaveStatus; +import accord.primitives.TxnId; +import accord.primitives.Unseekable; +import accord.primitives.Unseekables; +import accord.utils.Invariants; +import accord.utils.SemiSyncIntervalTree; +import accord.utils.UnhandledEnum; +import accord.utils.async.Cancellable; +import accord.utils.btree.BTree; +import accord.utils.btree.IntervalBTree; +import accord.utils.btree.IntervalBTree.FastIntervalTreeBuilder; + +import static accord.api.ProtocolModifiers.RangeSpec.isEndInclusive; +import static accord.impl.cfr.ListenerEntry.LISTENER_ENTRIES; +import static accord.impl.cfr.ListenerEntry.LISTENER_WITH_KEYS; +import static accord.impl.cfr.ListenerEntry.LISTENER_WITH_RANGES; +import static accord.impl.cfr.RangeEntry.ENTRIES; +import static accord.impl.cfr.RangeEntry.WITH_KEY; +import static accord.impl.cfr.RangeEntry.WITH_RANGE; +import static accord.local.RedundantStatus.Property.GC_BEFORE; +import static accord.local.RedundantStatus.Property.LOCALLY_APPLIED; +import static accord.local.RedundantStatus.Property.SHARD_APPLIED; +import static accord.local.RedundantStatus.Property.UNREADY; + +/** + * An implementation filling the same niche as CommandsForKey, only we do not retain + * complete information, and may need to load a command to decide if it is relevant. + * We also don't consume this structure directly, instead building a set of summary records to work with. + */ +public class InMemoryRangeSummaryIndex extends SemiSyncIntervalTree +{ + private final Map byId = new HashMap<>(); + private Object[] listeners = IntervalBTree.empty(); + + public InMemoryRangeSummaryIndex() + { + super(ENTRIES); + Invariants.require(isEndInclusive(), "Need to implement range-exclusive IntervalComparators"); + } + + private boolean update(Command command) + { + TxnId txnId = command.txnId(); + SaveStatus saveStatus = command.saveStatus(); + AbstractRanges participants = (AbstractRanges) command.participants().stillTouches(); + if (saveStatus.compareTo(SaveStatus.TruncatedUnapplied) >= 0 || participants.isEmpty()) + { + IdEntry entry = byId.remove(txnId); + if (entry != null) + pushEdit(entry, null, entry); + return false; + } + + IdEntry cur = byId.get(txnId); + IdEntry next = cur; + if (participants.size() == 1) + { + Range range = participants.get(0).asRange(); + if (cur == null || cur.getClass() != IdSingleEntry.class || !((IdSingleEntry) cur).range.equals(range)) + next = new IdSingleEntry(txnId, range); + } + else + { + if (cur == null || cur.getClass() != IdMultiEntry.class || !((IdMultiEntry) cur).ranges.hasSameRanges(participants)) + next = new IdMultiEntry(txnId, participants.toRanges()); + } + if (next != cur) + { + if (cur != null) + byId.remove(cur, cur); + byId.put(next, next); + pushEdit(next, next, cur); + } + return next.update(saveStatus, command.durability(), command.executeAtIfKnown()) || next != cur; + } + + private static Object[] toMultiTree(IdMultiEntry entry) + { + try (FastIntervalTreeBuilder builder = IntervalBTree.fastBuilder(ENTRIES)) + { + for (Range range : entry.ranges) + builder.add(new RangeMultiEntry(range, entry)); + return builder.build(); + } + } + + public A foldl(Unseekables participants, BiFunction f, A accumulator) + { + Object[] tree = get(); + switch (participants.domain()) + { + case Key: + for (RoutingKey key : (AbstractUnseekableKeys)participants) + accumulator = IntervalBTree.accumulate(tree, WITH_KEY, key, InMemoryRangeSummaryIndex::foldl, f, participants, accumulator); + break; + case Range: + for (Range range : (AbstractRanges)participants) + accumulator = IntervalBTree.accumulate(tree, WITH_RANGE, range, InMemoryRangeSummaryIndex::foldl, f, participants, accumulator); + break; + } + return accumulator; + } + + public void populateMinFutureRx(SummaryLoader loader) + { + foldl(loader.participants(), (e, l) -> { + if (e.isSyncPoint() && l.shouldRecordFutureRx(e, e.saveStatus().summary)) + l.recordFutureRx(e.plainTxnId(), e.ranges()); + return l; + }, loader); + } + + /** + * If we have enough information, simply directly build the Summary object and return a map of these objects; + * otherwise invoke the provided Consumer so that the implementation may build the summary + */ + public void search(SummaryLoader loader, @Nullable BiConsumer found, @Nullable Consumer mustLoad) + { + foldl(loader.participants(), (e, l) -> { + Ranges ranges = e.ranges(); + Relevance relevance = l.relevance(e, e.saveStatus(), e.durability(), e.maybeExecuteAt(), ranges); + if (relevance == Relevance.IRRELEVANT) + return l; + + TxnId txnId = e.plainTxnId(); + switch (relevance) + { + default: throw new UnhandledEnum(relevance); + case ACTIVE: + if (found != null) + { + Summary summary = l.ifRelevant(e.plainTxnId(), e.maybeExecuteAt(), e.saveStatus(), e.durability(), ranges, null); + Invariants.nonNull(summary); + // TODO (expected): we can post-filter collected txnId after recording future rx to remove those that are no longer needed + // (or, we can retain summary information to permit evicting them as we collect) + found.accept(txnId, summary); + } + break; + case SUPERSEDING: + case BOTH: + if (mustLoad != null) + mustLoad.accept(txnId); + } + return l; + }, loader); + } + + static < A> A foldl(BiFunction f, Unseekables find, RangeEntry e, A accumulator) + { + if (e.getClass() == IdSingleEntry.class) + { + return f.apply((IdSingleEntry)e, accumulator); + } + else + { + IdEntry id = e.id(); + IdMultiEntry mid = (IdMultiEntry) id; + if (mid.ranges.size() > 1) + { + int i = (int) (find.findFirstIntersection(mid.ranges) >>> 32); + if (mid.ranges.get(i) != e.range()) + return accumulator; + } + return f.apply(id, accumulator); + } + } + + public void update(Command prev, Command updated, boolean force) + { + if (!force + && updated.saveStatus() == prev.saveStatus() + && (updated.saveStatus().known.executeAt().isDecided() && !prev.saveStatus().known.executeAt().isDecided()) + && updated.participants().stillTouches().equals(prev.participants().touches())) + return; + + if (update(updated)) + { + // invoke listeners + Participants stillTouches = updated.participants().stillTouches(); + switch (stillTouches.domain()) + { + default: throw new UnhandledEnum(stillTouches.domain()); + case Key: + for (RoutingKey key : (AbstractUnseekableKeys)stillTouches) + IntervalBTree.accumulate(listeners, LISTENER_WITH_KEYS, key, InMemoryRangeSummaryIndex::visitListeners, stillTouches, updated, null); + break; + case Range: + for (Range range : (AbstractRanges)stillTouches) + IntervalBTree.accumulate(listeners, LISTENER_WITH_RANGES, range, InMemoryRangeSummaryIndex::visitListeners, stillTouches, updated, null); + break; + } + } + } + + public Cancellable registerListener(Listener listener) + { + Object[] tree; + try (IntervalBTree.FastIntervalTreeBuilder builder = IntervalBTree.fastBuilder(LISTENER_ENTRIES)) + { + for (Unseekable u : listener.participants()) + builder.add(new ListenerEntry(u.asRange(), listener)); + tree = builder.build(); + } + listeners = IntervalBTree.update(listeners, tree, LISTENER_ENTRIES); + return () -> listeners = IntervalBTree.subtract(listeners, tree, LISTENER_ENTRIES); + } + + private static Object visitListeners(Participants participants, Command command, ListenerEntry entry, Object v) + { + Unseekables searchingFor = entry.listener.participants(); + if (searchingFor.size() > 1) + { + int i = (int) searchingFor.findNextIntersection(0, participants, 0); + if (!searchingFor.get(i).asRange().equals(entry.range)) + return v; + } + entry.listener.accept(command); + return v; + } + + public void prune(CommandStore commandStore) + { + prune(commandStore.unsafeGetRangesForEpoch().all(), commandStore.unsafeGetRedundantBefore()); + } + + public void prune(Ranges ranges, RedundantBefore redundantBefore) + { + prune(TxnId.MAX, ranges, redundantBefore); + } + + public void prune(TxnId lessThan, Ranges ranges, RedundantBefore redundantBefore) + { + drainPendingEdits(); + List remove = new ArrayList<>(); + foldl(ranges, (id, es) -> { + if (id.compareTo(lessThan) < 0 && redundantBefore.foldl(id.ranges(), InMemoryRangeSummaryIndex::prune, true, id)) + es.add(id); + return es; + }, remove); + for (IdEntry e : remove) + { + byId.remove(e); + value = applyMultiple(value, null, tree(e)); + } + } + + private static Boolean prune(RedundantBefore.Bounds bounds, Boolean prune, TxnId txnId) + { + // TODO (expected): prune implied invalidations + if (!prune) return false; + if (bounds.maxBound(LOCALLY_APPLIED).compareTo(txnId) <= 0) return false; + if (!txnId.isSyncPoint()) return bounds.maxBound(GC_BEFORE).compareTo(txnId) > 0; + if (bounds.maxBound(SHARD_APPLIED).compareTo(txnId) > 0) return true; + return bounds.maxBound(UNREADY).compareTo(txnId) < 0; + } + + @Override + protected Object[] tree(IdEntry edit) + { + if (edit.getClass() == IdMultiEntry.class) + return toMultiTree((IdMultiEntry) edit); + return BTree.singleton(edit); + } + + public void restore(List idEntries) + { + for (IdEntry original : idEntries) + { + IdEntry copy = original.copy(); + Invariants.require(null == byId.put(copy, copy)); + value = applyMultiple(value, tree(copy), null); + } + } + + public List snapshot() + { + drainPendingEdits(); + ImmutableList.Builder builder = ImmutableList.builder(); + for (IdEntry e : byId.values()) + builder.add(e.copy()); + return builder.build(); + } + + public void clear() + { + drainPendingEdits(); + byId.clear(); + value = IntervalBTree.empty(); + } + + @Override + public boolean tryDrainPendingEdits() + { + return super.tryDrainPendingEdits(); + } + + @Override + public Object[] drainPendingEdits() + { + return super.drainPendingEdits(); + } +} diff --git a/accord-core/src/main/java/accord/impl/cfr/Listener.java b/accord-core/src/main/java/accord/impl/cfr/Listener.java new file mode 100644 index 0000000000..c0ebb8091b --- /dev/null +++ b/accord-core/src/main/java/accord/impl/cfr/Listener.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.cfr; + +import java.util.concurrent.atomic.AtomicLong; + +import accord.local.Command; +import accord.primitives.Unseekables; + +public abstract class Listener +{ + private static final AtomicLong nextId = new AtomicLong(); + + final long id = nextId.incrementAndGet(); + + protected abstract Unseekables participants(); + protected abstract void accept(Command command); +} diff --git a/accord-core/src/main/java/accord/impl/cfr/ListenerEntry.java b/accord-core/src/main/java/accord/impl/cfr/ListenerEntry.java new file mode 100644 index 0000000000..9f366ae7a2 --- /dev/null +++ b/accord-core/src/main/java/accord/impl/cfr/ListenerEntry.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.cfr; + +import java.util.Comparator; + +import accord.api.RoutingKey; +import accord.impl.RangeIntervalComparators; +import accord.primitives.Range; +import accord.utils.btree.IntervalBTree; + +class ListenerEntry +{ + static final IntervalBTree.IntervalComparators LISTENER_ENTRIES = new RangeIntervalComparators.InclusiveEndEntryComparators<>(e -> e.range, Comparator.comparingLong(a -> a.listener.id)); + static final IntervalBTree.WithIntervalComparators LISTENER_WITH_KEYS = new RangeIntervalComparators.InclusiveEndWithKeyComparators<>(e -> e.range); + static final IntervalBTree.WithIntervalComparators LISTENER_WITH_RANGES = new RangeIntervalComparators.InclusiveEndWithRangeComparators<>(e -> e.range); + + final Range range; + final Listener listener; + + ListenerEntry(Range range, Listener listener) + { + this.range = range; + this.listener = listener; + } +} diff --git a/accord-core/src/main/java/accord/impl/cfr/LoadListener.java b/accord-core/src/main/java/accord/impl/cfr/LoadListener.java new file mode 100644 index 0000000000..117fa98f2a --- /dev/null +++ b/accord-core/src/main/java/accord/impl/cfr/LoadListener.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.cfr; + +import java.util.Map; + +import accord.local.Command; +import accord.local.CommandSummaries; +import accord.local.CommandSummaries.Summary; +import accord.local.CommandSummaries.SummaryLoader; +import accord.primitives.Timestamp; +import accord.primitives.Unseekables; + +import static accord.local.CommandSummaries.Relevance.SUPERSEDING; + +public class LoadListener extends Listener +{ + final SummaryLoader loader; + final Map into; + + public LoadListener(SummaryLoader loader, Map into) + { + this.loader = loader; + this.into = into; + } + + public SummaryLoader loader() + { + return loader; + } + + public Map into() + { + return into; + } + + @Override + protected Unseekables participants() + { + return loader.participants(); + } + + @Override + protected void accept(Command command) + { + CommandSummaries.Relevance relevance = loader.relevance(command); + if (relevance.is(SUPERSEDING)) + { + Summary summary = loader.get(relevance, command); + if (summary != null) + into.put(command.txnId(), summary); + } + } +} diff --git a/accord-core/src/main/java/accord/impl/cfr/RangeEntry.java b/accord-core/src/main/java/accord/impl/cfr/RangeEntry.java new file mode 100644 index 0000000000..f2f0a30fca --- /dev/null +++ b/accord-core/src/main/java/accord/impl/cfr/RangeEntry.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.cfr; + +import accord.api.RoutingKey; +import accord.impl.RangeIntervalComparators.InclusiveEndEntryComparators; +import accord.impl.RangeIntervalComparators.InclusiveEndWithKeyComparators; +import accord.impl.RangeIntervalComparators.InclusiveEndWithRangeComparators; +import accord.primitives.Range; +import accord.utils.btree.IntervalBTree.IntervalComparators; +import accord.utils.btree.IntervalBTree.WithIntervalComparators; + +public interface RangeEntry +{ + Range range(); + IdEntry id(); + + IntervalComparators ENTRIES = new InclusiveEndEntryComparators<>(RangeEntry::range, (a, b) -> a.id().compareTo(b.id())); + WithIntervalComparators WITH_KEY = new InclusiveEndWithKeyComparators<>(RangeEntry::range); + WithIntervalComparators WITH_RANGE = new InclusiveEndWithRangeComparators<>(RangeEntry::range); +} diff --git a/accord-core/src/main/java/accord/impl/cfr/RangeMultiEntry.java b/accord-core/src/main/java/accord/impl/cfr/RangeMultiEntry.java new file mode 100644 index 0000000000..67eb7bdad9 --- /dev/null +++ b/accord-core/src/main/java/accord/impl/cfr/RangeMultiEntry.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.impl.cfr; + +import accord.primitives.Range; + +public class RangeMultiEntry implements RangeEntry +{ + public final Range range; + public final IdMultiEntry id; + + RangeMultiEntry(Range range, IdMultiEntry id) + { + this.range = range; + this.id = id; + } + + @Override + public Range range() + { + return range; + } + + @Override + public IdEntry id() + { + return id; + } +} diff --git a/accord-core/src/main/java/accord/impl/progresslog/BaseTxnState.java b/accord-core/src/main/java/accord/impl/progresslog/BaseTxnState.java index 57d6d80385..e98d6f86c3 100644 --- a/accord-core/src/main/java/accord/impl/progresslog/BaseTxnState.java +++ b/accord-core/src/main/java/accord/impl/progresslog/BaseTxnState.java @@ -46,7 +46,10 @@ */ abstract class BaseTxnState extends LogGroupTimers.Timer implements Comparable { - private static final int CONTACT_ALL_SHIFT = 63; // TODO (desired): have separate contact all flags for recovery and waiting states + private static final int RESTORED_SHIFT = 63; + static final long RESTORED_BIT = 1L << RESTORED_SHIFT; + private static final int CONTACT_ALL_SHIFT = RESTORED_SHIFT - 1; + private static final long CONTACT_ALL_BIT = 1L << CONTACT_ALL_SHIFT; // TODO (desired): have separate contact all flags for recovery and waiting states private static final int SCHEDULED_TIMER_SHIFT = CONTACT_ALL_SHIFT - 1; private static final int INVALID_IF_UNCOMMITTED_SHIFT = SCHEDULED_TIMER_SHIFT - 1; private static final int PENDING_TIMER_BITS = 9; @@ -59,22 +62,25 @@ abstract class BaseTxnState extends LogGroupTimers.Timer implements Comparable - * bits [43..51) encode HomeState + * bits [8..51) encode WaitingState * 2 bits for Progress - * 3 bits for CoordinatePhase + * 2 bits for BlockedUntil target + * 2 bits for BlockedUntil querying (<= target) + * 2 bits for BlockedUntil that home shard can satisfy + * 1 bit to query non-home shards (as home shard has been truncated) + * 24+5 bits for remote key progress tracking [note: if we need to in future we can safely and easily reclaim bits here] * 3 bits for retry counter *

- * bits [52..61) for pending timer delay - * bit 61 indicates if this transaction can be inferred invalid if a later quorum finds it not committed on any shard - * bit 62 for which kind of timer is scheduled - * bit 63 for whether we should contact all candidate replicas (rather than just our preferred group) + * bits [51..60) for pending timer delay + * bit 60 indicates if this transaction can be inferred invalid if a later quorum finds it not committed on any shard + * bit 61 for which kind of timer is scheduled + * bit 62 for whether we should contact all candidate replicas (rather than just our preferred group) + * bit 63 for whether the item was restored from a snapshot (so can modify certain invariants) */ long encodedState; @@ -89,9 +95,14 @@ public int compareTo(BaseTxnState that) return this.txnId.compareTo(that.txnId); } + boolean isRestored() + { + return (encodedState & RESTORED_BIT) != 0L; + } + boolean contactEveryone() { - return ((encodedState >>> CONTACT_ALL_SHIFT) & 1L) == 1L; + return (encodedState & CONTACT_ALL_BIT) != 0L; } void setContactEveryone(boolean newContactEveryone) diff --git a/accord-core/src/main/java/accord/impl/progresslog/DefaultProgressLog.java b/accord-core/src/main/java/accord/impl/progresslog/DefaultProgressLog.java index 33aa3021ba..fa0f94665e 100644 --- a/accord-core/src/main/java/accord/impl/progresslog/DefaultProgressLog.java +++ b/accord-core/src/main/java/accord/impl/progresslog/DefaultProgressLog.java @@ -21,6 +21,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; +import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -55,6 +56,8 @@ import accord.utils.TinyEnumSet; import accord.utils.btree.BTree; import accord.utils.btree.BTreeRemoval; +import accord.utils.btree.BulkIterator; +import accord.utils.btree.UpdateFunction; import org.agrona.collections.Long2ObjectHashMap; import org.agrona.collections.Object2ObjectHashMap; @@ -462,7 +465,7 @@ public void waiting(BlockedUntil blockedUntil, SafeCommandStore safeStore, SafeC if (command == null) command = uninitialised(blockedBy.txnId()); SaveStatus saveStatus = command.saveStatus(); - Invariants.require(saveStatus.compareTo(blockedUntil.unblockedFrom) < 0); + Invariants.expect(saveStatus.compareTo(blockedUntil.unblockedFrom) < 0); StoreParticipants blockedOnStoreParticipants2 = null; if (blockedOnParticipants != null || blockedOnRoute != null) @@ -883,7 +886,6 @@ void clearPendingAndActive(TxnStateKind kind, TxnId txnId) } } - @VisibleForImplementation public void requeue(SafeCommandStore safeStore, TxnStateKind kind, TxnId txnId) { clearPendingAndActive(kind, txnId); @@ -1072,7 +1074,7 @@ public boolean contactEveryone() public boolean isWaitingUninitialised() { - return current.isUninitialised(); + return current.isWaitingUninitialised(); } @Nonnull @@ -1122,4 +1124,36 @@ public int homeRetryCounter() return current.homeRunCounter(); } } + + public void restore(SafeCommandStore safeStore, List states) + { + if (!BTree.isEmpty(stateMap)) + throw new IllegalStateException("Restore only supported if uninitialised"); + + { + List snapshot = new ArrayList<>(states.size()); + for (TxnState state : states) + snapshot.add(state.snapshot()); + states = snapshot; + } + + states.sort(Comparator.naturalOrder()); + stateMap = BTree.build(BulkIterator.of(states.iterator()), states.size(), UpdateFunction.noOp()); + + for (TxnState state : states) + { + if (!state.isHomeDoneOrUninitialised() && state.homeProgress() != NoneExpected) + state.updateScheduling(safeStore, this, Home, null, Queued); + if (!state.isWaitingDoneOrUninitialised() && state.waitingProgress() != NoneExpected) + state.updateScheduling(safeStore, this, Waiting, null, Queued); + } + } + + public List snapshot() + { + List snapshot = new ArrayList<>(BTree.size(stateMap)); + for (TxnState state : BTree.iterable(stateMap)) + snapshot.add(state.snapshot()); + return snapshot; + } } diff --git a/accord-core/src/main/java/accord/impl/progresslog/HomeState.java b/accord-core/src/main/java/accord/impl/progresslog/HomeState.java index 54dee971d7..8262dce5e4 100644 --- a/accord-core/src/main/java/accord/impl/progresslog/HomeState.java +++ b/accord-core/src/main/java/accord/impl/progresslog/HomeState.java @@ -67,6 +67,7 @@ abstract class HomeState extends BaseTxnState private static final long SET_MASK = ~((PROGRESS_MASK << PROGRESS_SHIFT) | (STATUS_MASK << STATUS_SHIFT)); static final int HOME_STATE_END_SHIFT = RUN_COUNTER_SHIFT + 3; + static final long SNAPSHOT_HOME_MASK = ~SET_MASK; static { @@ -80,13 +81,13 @@ abstract class HomeState extends BaseTxnState void set(SafeCommandStore safeStore, DefaultProgressLog owner, HomePhase newHomePhase, Progress newProgress) { - setNoScheduling(newHomePhase, newProgress); + setWithoutScheduling(newHomePhase, newProgress); if (newProgress == NoneExpected) owner.clearProgressToken(txnId); updateScheduling(safeStore, owner, Home, null, newProgress); } - void setNoScheduling(HomePhase newHomePhase, Progress newProgress) + void setWithoutScheduling(HomePhase newHomePhase, Progress newProgress) { encodedState &= SET_MASK; encodedState |= ((long) newHomePhase.ordinal() << STATUS_SHIFT) @@ -156,7 +157,7 @@ void maybeUpdatePhase(SafeCommandStore safeStore, DefaultProgressLog owner, Comm HomePhase shouldUpdatePhase(DefaultProgressLog owner, Command command) { - if (command.saveStatus() == SaveStatus.Erased) + if (command.durability().isDurableOrInvalidated() || command.saveStatus() == SaveStatus.Erased) return Done; HomePhase phase = homePhase(); @@ -197,7 +198,7 @@ final void runHome(DefaultProgressLog owner, SafeCommandStore safeStore, SafeCom set(safeStore, owner, updatePhase, NoneExpected); return; } - setNoScheduling(updatePhase, Queued); + setWithoutScheduling(updatePhase, Queued); } } diff --git a/accord-core/src/main/java/accord/impl/progresslog/TxnState.java b/accord-core/src/main/java/accord/impl/progresslog/TxnState.java index 1292061e30..800b53d365 100644 --- a/accord-core/src/main/java/accord/impl/progresslog/TxnState.java +++ b/accord-core/src/main/java/accord/impl/progresslog/TxnState.java @@ -34,6 +34,16 @@ public final class TxnState extends WaitingState implements PreLoadContext { + public static class SerializationSupport + { + public static TxnState create(TxnId txnId, long encoded) + { + TxnState result = new TxnState(txnId); + result.encodedState = encoded; + return result; + } + } + TxnState(TxnId txnId) { super(txnId); @@ -185,4 +195,26 @@ public String reason() { return "Progress"; } + + public TxnState snapshot() + { + TxnState copy = new TxnState(txnId); + copy.encodedState = (encodedState & (SNAPSHOT_HOME_MASK | SNAPSHOT_WAITING_MASK)) | RESTORED_BIT; + return copy; + } + + public long encodedState() + { + return encodedState; + } + + public boolean equals(Object that) + { + return that instanceof TxnState && equals((TxnState) that); + } + + public boolean equals(TxnState that) + { + return this.txnId.equals(that.txnId) && this.encodedState == that.encodedState; + } } diff --git a/accord-core/src/main/java/accord/impl/progresslog/WaitingState.java b/accord-core/src/main/java/accord/impl/progresslog/WaitingState.java index d6c9065ca9..8dfc765c5d 100644 --- a/accord-core/src/main/java/accord/impl/progresslog/WaitingState.java +++ b/accord-core/src/main/java/accord/impl/progresslog/WaitingState.java @@ -104,7 +104,7 @@ abstract class WaitingState extends HomeState private static final int AWAIT_STARTED_SHIFT = QUERY_SHARDS_NOT_HOME_SHIFT + 1; private static final int AWAIT_STARTED_BIT = 1 << AWAIT_STARTED_SHIFT; private static final int AWAIT_SHIFT = AWAIT_STARTED_SHIFT + 1; - private static final int AWAIT_BITS = 26; + private static final int AWAIT_BITS = 24; private static final long AWAIT_MASK = (1L << AWAIT_BITS) - 1; private static final int AWAIT_EPOCH_SHIFT = AWAIT_SHIFT + AWAIT_BITS; private static final int AWAIT_EPOCH_BITS = 4; @@ -114,6 +114,8 @@ abstract class WaitingState extends HomeState private static final int RETRY_COUNTER_SHIFT = AWAIT_EPOCH_SHIFT + AWAIT_EPOCH_BITS; private static final long RETRY_COUNTER_MASK = 0x7; static final int WAITING_STATE_END_SHIFT = RETRY_COUNTER_SHIFT + 3; + static long SNAPSHOT_WAITING_MASK = INITIALISED_MASK | ~SET_MASK | QUERY_SHARDS_NOT_HOME_BIT; + static { Invariants.require(BLOCKED_UNTIL_SHIFT == PROGRESS_SHIFT + Long.bitCount(PROGRESS_MASK)); @@ -153,7 +155,7 @@ private void setHomeSatisfies(BlockedUntil homeStatus) encodedState |= (long) homeStatus.ordinal() << HOME_SATISFIES_SHIFT; } - boolean isUninitialised() + boolean isWaitingUninitialised() { return 0 == (encodedState & INITIALISED_MASK); } @@ -450,13 +452,16 @@ private void runInternal(SafeCommandStore safeStore, SafeCommand safeCommand, De incrementWaitingRunCounter(); BlockedUntil blockedUntil = blockedUntil(); Command command = safeCommand.current(); - if (command.saveStatus().compareTo(SaveStatus.Erased) >= 0 // TODO (expected): improve progress log clearing to guarantee we don't encounter this status - || !Invariants.expect(command.saveStatus().compareTo(blockedUntil.unblockedFrom) < 0, - "Command has met desired criteria (%s) but progress log entry has not been cancelled: %s", blockedUntil.unblockedFrom, command)) + if (command.saveStatus().compareTo(blockedUntil.unblockedFrom) >= 0) { + // TODO (expected): improve progress log clearing to guarantee we don't encounter Erased + Invariants.expect(command.saveStatus() == SaveStatus.Erased || isRestored(), + "Command has met desired criteria (%s) but progress log entry has not been cancelled: %s", + blockedUntil.unblockedFrom, command); setWaitingDone(owner); return; } + TxnId txnId = safeCommand.txnId(); // first make sure we have enough information to obtain the command locally Timestamp executeAt = command.executeAtIfKnown(); @@ -832,6 +837,13 @@ void asynchronousAwaitCallback(DefaultProgressLog owner, SafeCommandStore safeSt return; } + if (safeCommand != null && safeCommand.current().saveStatus().compareTo(querying.unblockedFrom) >= 0) + { + if (tracing != null) + tracing.trace(owner.commandStore, "Received async callback %d with %s; local command already exceeds wait status"); + return; + } + callbackId >>= 1; Invariants.nonNull(safeCommand); Route route = Route.castToRoute(safeCommand.current().maxParticipants()); @@ -924,7 +936,7 @@ void awaitHomeKey(DefaultProgressLog owner, BlockedUntil blockedUntil, TxnId txn void awaitSlice(DefaultProgressLog owner, BlockedUntil blockedUntil, TxnId txnId, Timestamp executeAt, Route route, int callbackId, @Nullable Tracing tracing) { - Invariants.require(blockedUntil.waitsOn == SHARD); + Invariants.require(blockedUntil.waitsOn == SHARD || queryShardsNotHome()); // TODO (expected): special-case when this shard is home key to avoid remote messages await(owner, blockedUntil, txnId, executeAt, route, callbackId, WaitingState::synchronousAwaitSliceCallback, tracing); } @@ -982,6 +994,11 @@ boolean isWaitingDone() return waitingProgress() == NoneExpected && blockedUntil() == CanApply; } + boolean isWaitingDoneOrUninitialised() + { + return isWaitingDone() || isWaitingUninitialised(); + } + enum CallbackKind { Fetch, FetchRoute, AwaitHome, AwaitSlice diff --git a/accord-core/src/main/java/accord/local/Bootstrap.java b/accord-core/src/main/java/accord/local/Bootstrap.java index 27aba94cb8..891bd1d287 100644 --- a/accord-core/src/main/java/accord/local/Bootstrap.java +++ b/accord-core/src/main/java/accord/local/Bootstrap.java @@ -144,11 +144,13 @@ TxnId start(SafeCommandStore safeStore) CommandStore commandStore = safeStore.commandStore(); CoordinateSyncPoint.exclusive(node, globalSyncId, commitRanges) .flatMap(success -> commandStore.chain((PreLoadContext.Empty) () -> "Mark Bootstrapping", safeStore0 -> { + // we submit a separate execution so that we know markBootstrapping is durable before we initiate the fetch - Bootstrap.this.commandStore.markBootstrapping(safeStore0, globalSyncId, commitRanges); + if (!valid.isEmpty()) + commandStore.markBootstrapping(safeStore0, globalSyncId, valid); return success; })) - .flatMap(syncPoint -> node.withEpochAtLeast(epoch, null, () -> Bootstrap.this.commandStore.chain((PreLoadContext.Empty) () -> "Start Bootstrap Fetch", safeStore1 -> { + .flatMap(syncPoint -> node.withEpochAtLeast(epoch, null, () -> commandStore.chain((PreLoadContext.Empty) () -> "Start Bootstrap Fetch", safeStore1 -> { if (valid.isEmpty()) // we've lost ownership of the range return AsyncResults.success(Ranges.EMPTY); return fetch = safeStore1.dataStore().fetch(node, safeStore1, valid, syncPoint, this, Image); diff --git a/accord-core/src/main/java/accord/local/Catchup.java b/accord-core/src/main/java/accord/local/Catchup.java index edf1314f0f..658321d749 100644 --- a/accord-core/src/main/java/accord/local/Catchup.java +++ b/accord-core/src/main/java/accord/local/Catchup.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Objects; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Consumer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,6 +32,7 @@ import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.SaveStatus; +import accord.primitives.Status; import accord.primitives.TxnId; import accord.utils.Reduce; import accord.utils.async.AsyncChain; @@ -39,6 +41,7 @@ import accord.utils.async.AsyncResults; import accord.utils.async.AsyncResults.SettableResult; +import static accord.api.ProgressLog.BlockedUntil.CanApply; import static accord.local.RedundantStatus.Property.LOCALLY_APPLIED; import static accord.primitives.Routables.Slice.Minimal; @@ -55,13 +58,59 @@ static class CommandStoreListener extends SettableResult implements SyncPo this.durableBefore = durableBefore; } - void register(SafeCommandStore safeStore) + boolean register(SafeCommandStore safeStore) { - waitingOn = safeStore.ranges().all().slice(durableBefore.ranges(Objects::nonNull), Minimal); + waitingOn = safeStore.ranges().all().slice(durableBefore.ranges(Objects::nonNull), Minimal).mergeTouching(); + logger.debug("{}: Registering listener on {}, filtering by {}", safeStore.commandStore(), waitingOn, safeStore.redundantBefore().map(b -> b == null ? null : b.maxBound(LOCALLY_APPLIED), TxnId[]::new)); updateWaitingOn(safeStore); - if (waitingOn.isEmpty()) setSuccess(null); - else safeStore.register(this); + if (!waitingOn.isEmpty()) + { + logger.info("{}: catching-up {}", safeStore.commandStore(), durableBefore.foldlWithBounds(waitingOn, (entry, sb, start, end) -> { + if (sb.length() > 0) + sb.append(", "); + if (start == null || end == null || entry == null) + { + sb.append("??(").append(start).append(',').append(end).append(',').append(entry).append(')'); + } + else + { + TxnId txnId = entry.quorumBefore.withoutNonIdentityFlags(); + Range range = start.rangeFactory().newRange(start, end); + markWaiting(safeStore, txnId, range); + sb.append(range).append(": ").append(entry.quorumBefore); + } + return sb; + }, new StringBuilder(), ignore -> false)); + safeStore.register(this); + return true; + } + else + { + done(safeStore); + return false; + } + } + + private static void markWaiting(SafeCommandStore safeStore, TxnId txnId, Range range) + { + //noinspection DataFlowIssue + safeStore = safeStore; + PreLoadContext ctx = PreLoadContext.contextFor(txnId, "Catchup"); + if (safeStore.canExecuteWith(ctx)) markWaiting(safeStore, safeStore.get(txnId), range); + else safeStore.commandStore().execute(ctx, (Consumer) safeStore0 -> markWaiting(safeStore0, safeStore0.get(txnId), range), safeStore.agent()); + } + + private static void markWaiting(SafeCommandStore safeStore, SafeCommand safeCommand, Range range) + { + if (!safeCommand.current().hasBeen(Status.PreApplied)) + safeStore.progressLog().waiting(CanApply, safeStore, safeCommand, null, Ranges.of(range), null); + } + + private void done(SafeCommandStore safeStore) + { + setSuccess(null); + logger.info("{}: fully caught-up with quorums", safeStore.commandStore()); } private void updateWaitingOn(SafeCommandStore safeStore) @@ -106,13 +155,11 @@ public void update(SafeCommandStore safeStore, Command command) return; updateWaitingOn(safeStore); - - if (!waitingOn.isEmpty()) - return; - - logger.info("{}: fully caught-up with quorums", safeStore.commandStore()); - setSuccess(null); - safeStore.unregister(this); + if (waitingOn.isEmpty()) + { + done(safeStore); + safeStore.unregister(this); + } } } @@ -121,9 +168,9 @@ public static AsyncChain catchup(Node node) return FetchDurableBefore.catchup(node).flatMap(durableBefore -> { List> results = new CopyOnWriteArrayList<>(); return node.commandStores().forAll("Catchup", safeStore -> { - CommandStoreListener commandStoreListener = new CommandStoreListener(durableBefore); - commandStoreListener.register(safeStore); - results.add(commandStoreListener); + CommandStoreListener listener = new CommandStoreListener(durableBefore); + if (listener.register(safeStore)) + results.add(listener); }).flatMap(ignore -> { List> list = new ArrayList<>(results); if (list.isEmpty()) diff --git a/accord-core/src/main/java/accord/local/CommandStore.java b/accord-core/src/main/java/accord/local/CommandStore.java index d321bc6794..94f0742b36 100644 --- a/accord-core/src/main/java/accord/local/CommandStore.java +++ b/accord-core/src/main/java/accord/local/CommandStore.java @@ -35,6 +35,7 @@ import java.util.function.Supplier; import javax.annotation.Nullable; +import accord.primitives.*; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSortedMap; import org.slf4j.Logger; @@ -55,14 +56,7 @@ import accord.local.PreLoadContext.Empty; import accord.local.RedundantBefore.Bounds; import accord.local.RedundantStatus.SomeStatus; -import accord.primitives.Ranges; -import accord.primitives.Routables; -import accord.primitives.SaveStatus; -import accord.primitives.Status; import accord.primitives.Status.Durability.HasOutcome; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.primitives.Unseekables; import accord.utils.DeterministicIdentitySet; import accord.utils.Invariants; import accord.utils.Reduce; @@ -77,6 +71,8 @@ import accord.utils.async.AsyncResults.SettableResult; import org.agrona.collections.LongHashSet; +import static accord.local.RedundantStatus.Property.LOCALLY_APPLIED; +import static accord.local.RedundantStatus.Property.UNREADY; import static accord.topology.EpochReady.DONE; import static accord.topology.EpochReady.done; import static accord.api.DataStore.FetchKind.Image; @@ -188,7 +184,7 @@ CommandStore create(int id, */ private NavigableMap safeToRead = emptySafeToRead(); private final Set bootstraps = Collections.synchronizedSet(new DeterministicIdentitySet<>()); - @Nullable private RejectBefore rejectBefore; + private RejectBefore rejectBefore = RejectBefore.EMPTY; static class WaitingOnVisibility { @@ -226,8 +222,6 @@ public final int id() return id; } - public void restore() {}; - public abstract Journal.Replayer replayer(); // expected to invoke safeStore.upsertRedundantBefore at some future point, when the commandStore state is durably persisted protected abstract void ensureDurable(Ranges ranges, RedundantBefore onCommandStoreDurable); @@ -439,7 +433,7 @@ protected synchronized void loadBootstrapBeganAt(NavigableMap new /** * To be overridden by implementations, to ensure the new state is persisted. */ - protected void setMaxConflicts(MaxConflicts maxConflicts) + protected void unsafeSetMaxConflicts(MaxConflicts maxConflicts) { this.maxConflicts = maxConflicts; } @@ -510,15 +504,14 @@ else if (prunedSize != initialSize) maxConflictsUpdates = 0; } - setMaxConflicts(updatedMaxConflicts); + unsafeSetMaxConflicts(updatedMaxConflicts); } - final void markExclusiveSyncPoint(SafeCommandStore safeStore, TxnId txnId, Ranges ranges) + final void upsertRejectBefore(SafeCommandStore safeStore, TxnId txnId, Ranges ranges) { // TODO (desired): narrow ranges to those that are owned Invariants.requireArgument(txnId.isSyncPoint()); - RejectBefore newRejectBefore = rejectBefore != null ? rejectBefore : new RejectBefore(); - newRejectBefore = RejectBefore.add(newRejectBefore, ranges, txnId); + RejectBefore newRejectBefore = RejectBefore.add(rejectBefore, ranges, txnId); unsafeSetRejectBefore(newRejectBefore); } @@ -527,15 +520,25 @@ final void markExclusiveSyncPointDecided(SafeCommandStore safeStore, TxnId txnId unsafeSetMaxDecidedRX(maxDecidedRX.update(ranges, txnId)); } - final void markExclusiveSyncPointLocallyApplied(SafeCommandStore safeStore, TxnId txnId, Ranges ranges) + protected void markExclusiveSyncPointLocallyApplied(SafeCommandStore safeStore, TxnId txnId, Ranges ranges, SaveStatus prevStatus) { // TODO (desired): narrow ranges to those that are owned + if (prevStatus.compareTo(SaveStatus.Applied) < 0) + { + String alreadyApplied = redundantBefore.foldl(ranges, (b, m) -> { + if (b.maxBound(LOCALLY_APPLIED).compareTo(txnId) > 0 && b.maxBound(UNREADY).compareTo(txnId) <= 0 && !b.isLocallyRetired()) + return m + (m.isEmpty() ? "" : ", ") + b.range + ": " + b; + return m; + }, ""); + Invariants.expect(alreadyApplied.isEmpty(), "%s should already have been applied: %s", txnId, alreadyApplied); + } + Invariants.requireArgument(txnId.isSyncPoint()); RedundantBefore addNow = RedundantBefore.create(ranges, txnId, LOCALLY_APPLIED_ONLY); safeStore.upsertRedundantBefore(addNow); RedundantBefore addOnDataStoreDurable = RedundantBefore.create(ranges, txnId, LOCALLY_DURABLE_TO_DATA_STORE_ONLY); RedundantBefore addOnCommandStoreDurable = RedundantBefore.create(ranges, txnId, LOCALLY_DURABLE_TO_COMMAND_STORE_ONLY); - dataStore.ensureDurable(this, ranges, addOnDataStoreDurable); + dataStore.ensureDurable(this, ranges, addOnDataStoreDurable, 0); ensureDurable(ranges, addOnCommandStoreDurable); } @@ -736,7 +739,7 @@ private void rebootstrap(Node node, Ranges ranges, long epoch, int attempt, Epoc // If rebootstrap can grab a later timestamp for subsequent attempts, but this timestamp is enough for us // to establish which transactions, for which ranges the node can safely participate in). TxnId unreadyBefore = bootstrap.start(safeStore); - safeStore.unsafeUpsertRedundantBefore(RedundantBefore.create(ranges, unreadyBefore, LOG_UNAVAILABLE_ONLY)); + safeStore.upsertRedundantBefore(RedundantBefore.create(ranges, unreadyBefore, LOG_UNAVAILABLE_ONLY)); updateMaxConflicts(ranges, unreadyBefore); // TODO (desired): we could start accepting non-dep requests here bootstrap.data.invoke((SettableByCallback)ready.data); @@ -744,7 +747,7 @@ private void rebootstrap(Node node, Ranges ranges, long epoch, int attempt, Epoc ready.coordinate.invokeIfSuccess(() -> { execute((Empty)() -> "Accept Dependency Requests", safeStore0 -> { unsafeAcceptRequests(remaining); - }); + }, agent); }); return null; })).begin(agent); @@ -799,7 +802,7 @@ private void ensureReadyToCoordinate(long epoch, Ranges ranges) logger.info("Failed to close epoch {} for ranges {} on store {}, but some are retired; marking these as synced.", epoch, ranges, id, fail); execute((Empty)() -> "Mark Retired Ranges Synced", safeStore -> { markVisibleInternal(safeStore, epoch, retired, "(Retired)"); - }); + }, agent); } else if (remaining.isEmpty()) { @@ -807,7 +810,7 @@ else if (remaining.isEmpty()) } if (!remaining.isEmpty()) { - logger.error("Failed to close epoch {} for ranges {} on store {}. Retrying.", epoch, remaining, id, fail); + logger.warn("Failed to close epoch {} for ranges {} on store {}. Retrying.", epoch, remaining, id, fail); node.someExecutor().execute(() -> ensureReadyToCoordinate(epoch, remaining)); } } @@ -875,11 +878,11 @@ public void markShardDurable(SafeCommandStore safeStore, TxnId globalSyncId, Ran if (globalSyncId.is(HLC_BOUND) || !requiresUniqueHlcs()) { RedundantBefore addOnDataStoreDurable = RedundantBefore.create(slicedRanges, globalSyncId, GC_BEFORE_AND_LOCALLY_DURABLE); - dataStore.ensureDurable(this, slicedRanges, addOnDataStoreDurable); + dataStore.ensureDurable(this, slicedRanges, addOnDataStoreDurable, 0); } } - protected void updatedRedundantBefore(SafeCommandStore safeStore, RedundantBefore added) + protected void upsertedRedundantBefore(SafeCommandStore safeStore, RedundantBefore added) { TxnId clearWaitingBefore = redundantBefore.minShardAndLocallyAppliedBefore(); TxnId clearAllBefore = TxnId.min(clearWaitingBefore, durableBefore().min.quorumBefore); @@ -1064,7 +1067,7 @@ public final AsyncResult operatorTryToExecuteListeningTxns() SettableResult done = new SettableResult<>(); execute((Empty)() -> "Try Execute Listening", safeStore -> { tryExecuteListening(safeStore, listeners.txnsWaitingOn(SaveStatus.Applied).iterator(), done); - }); + }, agent); return done; } @@ -1084,7 +1087,7 @@ private void tryExecuteListening(SafeCommandStore safeStore, Iterator ite { //noinspection DataFlowIssue safeStore = safeStore; - execute(context, safeStore0 -> tryExecuteListening(safeStore0, waitingOn, iterator, done)); + execute(context, safeStore0 -> { tryExecuteListening(safeStore0, waitingOn, iterator, done); }, agent); } else { @@ -1128,9 +1131,6 @@ private void tryExecuteListening(SafeCommandStore safeStore, TxnId waitingOn, It public final boolean isRejectedIfNotPreAccepted(TxnId txnId, Unseekables participants) { - if (rejectBefore == null) - return false; - return rejectBefore.rejects(txnId, participants); } @@ -1255,7 +1255,7 @@ public void updateMinHlc(long minHlc) { Timestamp timestamp = Timestamp.fromValues(rangesForEpoch.epochs[rangesForEpoch.epochs.length - 1], minHlc, 0, node.id()); MaxConflicts updated = maxConflicts.update(rangesForEpoch.all(), timestamp); - setMaxConflicts(updated); + unsafeSetMaxConflicts(updated); } public static NavigableMap emptyBootstrapBeganAt() diff --git a/accord-core/src/main/java/accord/local/CommandStores.java b/accord-core/src/main/java/accord/local/CommandStores.java index 9a6e2d29af..29bc2867e7 100644 --- a/accord-core/src/main/java/accord/local/CommandStores.java +++ b/accord-core/src/main/java/accord/local/CommandStores.java @@ -567,6 +567,7 @@ public Ranges removed(long presentIn, long removedByInclusive) protected void loadSnapshot(Snapshot toLoad) { + Invariants.require(!shuttingDown); current = toLoad; } @@ -657,6 +658,7 @@ public Iterator iterator() final ShardDistributor shardDistributor; final Journal journal; volatile Snapshot current; + boolean shuttingDown; int nextId; private CommandStores(StoreSupplier supplier, ShardDistributor shardDistributor, Journal journal) @@ -962,10 +964,13 @@ public synchronized void initializeTopologyUnsafe(Journal.TopologyUpdate update) for (Map.Entry e : update.commandStores.entrySet()) { Invariants.require(e.getValue() != null); - EpochUpdateHolder holder = new EpochUpdateHolder(); - holder.add(1, e.getValue(), e.getValue().all()); - shards[i++] = new ShardHolder(supplier.create(e.getKey(), holder), e.getValue()); + EpochUpdateHolder epochUpdates = new EpochUpdateHolder(); + ShardHolder shard = new ShardHolder(supplier.create(e.getKey(), epochUpdates), e.getValue()); + // TODO (required): if the add is necessary (highly unlikely) it needs to be done once journal is writeable so we NEED to move this + if (!shard.ranges.equals(shard.store.rangesForEpoch)) + epochUpdates.add(1, e.getValue(), e.getValue().all()); maxId = Math.max(maxId, e.getKey()); + shards[i++] = shard; } Arrays.sort(shards, Comparator.comparingInt(shard -> shard.store.id)); @@ -975,7 +980,6 @@ public synchronized void initializeTopologyUnsafe(Journal.TopologyUpdate update) public synchronized void resetTopology(Journal.TopologyUpdate update) { - // TODO: assert Snapshot current = this.current; Invariants.require(update.global.epoch() == current.local.epoch()); ShardHolder[] shards = new ShardHolder[current.commandStores.size()]; @@ -1025,6 +1029,9 @@ public void accept(Long epoch, Ranges ranges) public synchronized Supplier updateTopology(Node node, Topology newTopology) { + if (shuttingDown) + throw new IllegalStateException("CommandStores are shutting down"); + TopologyUpdate update = updateTopology(node, current, newTopology); if (update.snapshot != current) { @@ -1044,8 +1051,14 @@ public synchronized Supplier updateTopology(Node node, Topology newT return update.bootstrap; } + protected synchronized void markShuttingDown() + { + shuttingDown = true; + } + public void shutdown() { + markShuttingDown(); for (ShardHolder shard : current.shards) shard.store.shutdown(); } diff --git a/accord-core/src/main/java/accord/local/CommandSummaries.java b/accord-core/src/main/java/accord/local/CommandSummaries.java index 57c026f5a8..870cb66e68 100644 --- a/accord-core/src/main/java/accord/local/CommandSummaries.java +++ b/accord-core/src/main/java/accord/local/CommandSummaries.java @@ -32,6 +32,7 @@ import accord.primitives.Participants; import accord.primitives.Ranges; import accord.primitives.SaveStatus; +import accord.primitives.Status; import accord.primitives.Status.Durability; import accord.primitives.Timestamp; import accord.primitives.Txn; @@ -44,14 +45,19 @@ import accord.utils.TriPredicate; import accord.utils.UnhandledEnum; +import static accord.local.CommandSummaries.Relevance.ACTIVE; +import static accord.local.CommandSummaries.Relevance.IRRELEVANT; +import static accord.local.CommandSummaries.Relevance.SUPERSEDING; import static accord.local.CommandSummaries.SummaryStatus.ACCEPTED; import static accord.local.CommandSummaries.SummaryStatus.APPLIED; +import static accord.local.CommandSummaries.SummaryStatus.PREACCEPTED; import static accord.local.LoadKeysFor.RECOVERY; +import static accord.local.LoadKeysFor.WRITE; import static accord.local.MaxDecidedRX.forDeps; +import static accord.primitives.Known.KnownDeps.NoDeps; import static accord.primitives.Routables.Slice.Minimal; import static accord.primitives.Status.Durability.NotDurable; import static accord.primitives.Txn.Kind.Nothing; -import static accord.primitives.TxnId.FastPath.PrivilegedCoordinatorWithDeps; public interface CommandSummaries { @@ -77,15 +83,52 @@ enum IsDep private static final IsDep[] IS_DEPS = values(); } + enum Relevance + { + /* No need to visit this command */ + IRRELEVANT(0), + /* May need to visit this command, but can make do with TxnId, SaveStatus and Durability */ + ACTIVE(1), + /* May need to visit this command, and requires SupersedingVisitor information */ + SUPERSEDING(2), + BOTH(3), + ; + + + private static final Relevance[] lookup = values(); + public static final int ENCODED_MASK = 3; + public static final int ENCODED_BITS = 2; + + Relevance(int encoded) + { + Invariants.require(encoded == ordinal()); + } + + public boolean is(Relevance relevance) + { + return (relevance.ordinal() & ordinal()) == relevance.ordinal(); + } + + public Relevance or(Relevance that) + { + return forOrdinal(this.ordinal() | that.ordinal()); + } + + public static Relevance forOrdinal(int ordinal) + { + return lookup[ordinal]; + } + } + class Summary extends TxnId { private static final int SUMMARY_STATUS_MASK = 0x7; private static final int IS_DEP_SHIFT = 3; private static final int IS_DEP_MASK = 0x7; private static final int DURABILITY_SHIFT = 6; - private static final int DURABILITY_MASK = (1 << Durability.TOTAL_ENCODING_BITS) - 1; + private static final int RELEVANCE_SHIFT = DURABILITY_SHIFT + Durability.ENCODING_BITS; - final @Nonnull Timestamp executeAt; + final @Nullable Timestamp executeAt; final int encoded; final Unseekables participants; @@ -95,19 +138,22 @@ public Summary slice(Ranges ranges) } @VisibleForTesting - public Summary(@Nonnull TxnId txnId, @Nonnull Timestamp executeAt, @Nonnull SummaryStatus status, @Nonnull Durability durability, IsDep dep, Unseekables participants) + public Summary(@Nonnull TxnId txnId, @Nullable Timestamp executeAt, @Nonnull SummaryStatus status, @Nonnull Durability durability, IsDep dep, Relevance relevance, Unseekables participants) { super(txnId); this.participants = participants; - this.executeAt = executeAt.equals(txnId) ? this : executeAt; - this.encoded = Invariants.nonNull(status).ordinal() | (dep == null ? Integer.MIN_VALUE : (dep.ordinal() << IS_DEP_SHIFT)) | (Invariants.nonNull(durability).encoded() << DURABILITY_SHIFT); + this.executeAt = txnId.equals(executeAt) ? this : executeAt; + this.encoded = Invariants.nonNull(status).ordinal() + | (dep == null ? Integer.MIN_VALUE : (dep.ordinal() << IS_DEP_SHIFT)) + | (Invariants.nonNull(durability).encoded() << DURABILITY_SHIFT) + | (relevance.ordinal() << RELEVANCE_SHIFT); } - private Summary(@Nonnull TxnId txnId, @Nonnull Timestamp executeAt, int encoded, Unseekables participants) + private Summary(@Nonnull TxnId txnId, @Nullable Timestamp executeAt, int encoded, Unseekables participants) { super(txnId); this.participants = participants; - this.executeAt = executeAt == txnId || executeAt.equals(txnId) ? this : executeAt; + this.executeAt = executeAt == txnId || txnId.equals(executeAt) ? this : executeAt; this.encoded = encoded; } @@ -116,6 +162,11 @@ public boolean is(IsDep isDep) return (encoded >> IS_DEP_SHIFT) == isDep.ordinal(); } + public boolean is(Relevance relevance) + { + return ((encoded >> RELEVANCE_SHIFT) & relevance.ordinal()) == relevance.ordinal(); + } + public IsDep isDep() { if (encoded < 0) @@ -125,7 +176,12 @@ public IsDep isDep() public Durability durability() { - return Durability.forEncoded((encoded >>> DURABILITY_SHIFT) & DURABILITY_MASK); + return Durability.forEncoded((encoded >>> DURABILITY_SHIFT) & Durability.ENCODING_MASK); + } + + public Relevance relevance() + { + return Relevance.forOrdinal((encoded >>> RELEVANCE_SHIFT) & Relevance.ENCODED_MASK); } public boolean is(SummaryStatus summaryStatus) @@ -139,6 +195,11 @@ public SummaryStatus status() return SummaryStatus.SUMMARY_STATUSES[ordinal]; } + public Unseekables participants() + { + return participants; + } + public TxnId plainTxnId() { return new TxnId(this); @@ -157,6 +218,7 @@ public String toString() ", executeAt=" + plainExecuteAt() + ", saveStatus=" + status() + ", isDep=" + isDep() + + ", relevance=" + relevance() + '}'; } } @@ -165,62 +227,76 @@ class SummaryLoader { public interface Factory { - L create(RedundantBefore redundantBefore, @Nullable MaxDecidedRX maxDecidedRX, TxnId primaryTxnId, Unseekables searchKeysOrRanges, Kinds testKind, TxnId minTxnId, Timestamp maxTxnId, @Nullable TxnId findAsDep); + L create(RedundantBefore redundantBefore, @Nullable MaxDecidedRX maxDecidedRX, TxnId primaryTxnId, Unseekables searchKeysOrRanges, Kinds testKind, TxnId minTxnId, Timestamp executeAt, LoadKeysFor loadKeysFor); } - private static final ReducingRangeMap NO_FUTURE_RX = new ReducingRangeMap<>(); + private static final ReducingRangeMap NO_RX = new ReducingRangeMap<>(); protected final RedundantBefore redundantBefore; protected final MaxDecidedRX maxDecidedRX; - protected final Unseekables searchKeysOrRanges; + protected final Unseekables searchFor; // TODO (expected): separate out Kinds we need before/after primaryTxnId/executeAt protected final Kinds testKind; - protected final TxnId primaryTxnId, findAsDep, minTxnId; + protected final LoadKeysFor loadKeysFor; + protected final TxnId primaryTxnId, minTxnId; protected final DecidedRX decidedRx; - protected final Timestamp maxTxnId; + protected final Timestamp primaryExecuteAt; - private ReducingRangeMap minVisitedFutureRX = NO_FUTURE_RX; - private TxnId maxRx = TxnId.MAX; + private ReducingRangeMap minVisitedFutureRX = NO_RX; + private TxnId maxRx = TxnId.MAX; // a cached summary of minVisitedFutureRX to avoid consulting the full collection // TODO (expected): provide executeAt to PreLoadContext so we can more aggressively filter what we load, esp. by Kind public static SummaryLoader loader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, PreLoadContext context) { - return loader(redundantBefore, maxDecidedRX, context.primaryTxnId(), context.loadKeysFor(), context.keys()); + return loader(redundantBefore, maxDecidedRX, context.primaryTxnId(), context.executeAt(), context.loadKeysFor(), context.keys()); } - public static SummaryLoader loader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, TxnId primaryTxnId, LoadKeysFor loadKeysFor, Unseekables keysOrRanges) + public static SummaryLoader loader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, TxnId primaryTxnId, Timestamp executeAt, LoadKeysFor loadKeysFor, Unseekables keysOrRanges) { - return loader(redundantBefore, maxDecidedRX, primaryTxnId, loadKeysFor, keysOrRanges, SummaryLoader::new); + return loader(redundantBefore, maxDecidedRX, primaryTxnId, executeAt, loadKeysFor, keysOrRanges, SummaryLoader::new); } public static L loader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, PreLoadContext context, Factory factory) { - return loader(redundantBefore, maxDecidedRX, context.primaryTxnId(), context.loadKeysFor(), context.keys(), factory); + return loader(redundantBefore, maxDecidedRX, context.primaryTxnId(), context.executeAt(), context.loadKeysFor(), context.keys(), factory); } - public static L loader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, TxnId primaryTxnId, LoadKeysFor loadKeysFor, Unseekables keysOrRanges, Factory factory) + public static L loader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, TxnId primaryTxnId, Timestamp executeAt, LoadKeysFor loadKeysFor, Unseekables keysOrRanges, Factory factory) { Invariants.require(primaryTxnId != null); TxnId minTxnId = redundantBefore.min(keysOrRanges, Bounds::gcBefore); - Timestamp maxTxnId = loadKeysFor == RECOVERY || !primaryTxnId.isSyncPoint() ? Timestamp.MAX : primaryTxnId; - TxnId findAsDep = loadKeysFor == RECOVERY ? primaryTxnId : null; Kinds kinds = primaryTxnId.witnesses().or(loadKeysFor == RECOVERY ? primaryTxnId.witnessedBy() : Nothing); if (!primaryTxnId.is(Txn.Kind.ExclusiveSyncPoint)) // the main distinction between RX and RV is that RV doesn't filter out decided transactions maxDecidedRX = null; - return factory.create(redundantBefore, maxDecidedRX, primaryTxnId, keysOrRanges, kinds, minTxnId, maxTxnId, findAsDep); + return factory.create(redundantBefore, maxDecidedRX, primaryTxnId, keysOrRanges, kinds, minTxnId, executeAt, loadKeysFor); } - public SummaryLoader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, TxnId primaryTxnId, Unseekables searchKeysOrRanges, Kinds testKind, TxnId minTxnId, Timestamp maxTxnId, @Nullable TxnId findAsDep) + public SummaryLoader(RedundantBefore redundantBefore, MaxDecidedRX maxDecidedRX, TxnId primaryTxnId, Unseekables searchFor, Kinds testKind, TxnId minTxnId, Timestamp primaryExecuteAt, LoadKeysFor loadKeysFor) { this.redundantBefore = redundantBefore; this.maxDecidedRX = maxDecidedRX; this.primaryTxnId = primaryTxnId; - this.searchKeysOrRanges = searchKeysOrRanges; + this.searchFor = searchFor; this.testKind = testKind; this.minTxnId = minTxnId; - this.maxTxnId = maxTxnId; - this.findAsDep = findAsDep; - this.decidedRx = forDeps(maxDecidedRX, searchKeysOrRanges, primaryTxnId); + this.primaryExecuteAt = primaryExecuteAt; + this.loadKeysFor = loadKeysFor; + this.decidedRx = forDeps(maxDecidedRX, searchFor, primaryTxnId); + } + + public Unseekables participants() + { + return searchFor; + } + + public LoadKeysFor loadKeysFor() + { + return loadKeysFor; + } + + public TxnId primaryTxnId() + { + return primaryTxnId; } public boolean isRelevant(CommandsForKey cfk) @@ -254,14 +330,16 @@ public boolean isRelevant(RoutingKey key, TxnId last, TxnId minUndecided) return decidedRx == null || decidedRx.includeDecided(last); } + public boolean shouldRecordFutureRx(TxnId txnId, SummaryStatus status) + { + return txnId.isSyncPoint() && txnId.compareTo(primaryTxnId) > 0 && (status == SummaryStatus.STABLE || status == APPLIED); + } + // the caller must manage mutual exclusion for this method, but not to any others - public void maybeRecordFutureRx(Summary summary) + public void recordFutureRx(TxnId txnId, Unseekables participants) { - if (summary.isSyncPoint() && summary.compareTo(primaryTxnId) > 0 && (summary.is(SummaryStatus.STABLE) || summary.is(APPLIED))) - { - minVisitedFutureRX = ReducingRangeMap.merge(minVisitedFutureRX, ReducingRangeMap.create(summary.participants.toRanges(), summary.plainTxnId()), TxnId::min); - maxRx = minVisitedFutureRX.foldlWithDefault(searchKeysOrRanges, TxnId::max, TxnId.MAX, TxnId.NONE); - } + minVisitedFutureRX = ReducingRangeMap.merge(minVisitedFutureRX, ReducingRangeMap.create(participants.toRanges(), txnId), TxnId::min); + maxRx = minVisitedFutureRX.foldlWithDefault(searchFor, TxnId::max, TxnId.MAX, TxnId.NONE); } public final Summary ifRelevant(Command cmd) @@ -284,39 +362,91 @@ public final Summary ifRelevant(Command.MinimalWithDeps cmd) public final boolean isMaybeRelevant(TxnId txnId) { - return isMaybeRelevant(txnId, null, null); + return relevance(txnId, null, null, null, null) != IRRELEVANT; } - // durability is used as a proxy for durably *decided* - public final boolean isMaybeRelevant(TxnId txnId, @Nullable Durability durability, @Nullable Unseekables participants) + public final Relevance relevance(Command cmd) { - if (!txnId.is(testKind)) - return false; + return relevance(cmd.txnId(), cmd.saveStatus(), cmd.durability(), cmd.executeAtIfKnown(), cmd.participants().stillTouches()); + } - if (txnId.compareTo(minTxnId) < 0 || txnId.compareTo(maxTxnId) > 0) - return false; + public final Relevance relevance(TxnId txnId, @Nullable SaveStatus saveStatus, @Nullable Durability durability, @Nullable Timestamp executeAt, @Nullable Unseekables participants) + { + if (loadKeysFor == WRITE || !txnId.is(testKind) || (saveStatus != null && saveStatus.compareTo(SaveStatus.TruncatedUnapplied) >= 0)) + return IRRELEVANT; - if (txnId.isSyncPoint()) + if (txnId.compareTo(minTxnId) < 0) + return IRRELEVANT; + + Relevance atLeast = IRRELEVANT; + if (loadKeysFor == RECOVERY && txnId.witnesses(primaryTxnId)) { - if (txnId.compareTo(maxRx) >= 0) - return false; + if (isIgnorableFutureRx(txnId, participants)) + return IRRELEVANT; - if (participants != null && txnId.compareTo(minVisitedFutureRX.foldlWithDefault(participants, TxnId::max, TxnId.MAX, TxnId.NONE)) >= 0) - return false; + atLeast = supersedingRelevance(txnId, saveStatus, executeAt, participants); } + if (primaryExecuteAt.compareTo(txnId) < 0 || !primaryTxnId.witnesses(txnId)) + return atLeast; + boolean mayFilterAsDecided = maxDecidedRX != null && (txnId.isSyncPoint() || (durability != null && durability.isDurablyCommitted())); if (!mayFilterAsDecided) - return true; + return atLeast.or(ACTIVE); - if (decidedRx != null && !decidedRx.includeDecided(txnId)) - return false; + if (decidedRx != null && decidedRx.excludeDecided(txnId)) + return atLeast; if (participants == null) - return true; + return atLeast.or(ACTIVE); DecidedRX decidedRx = forDeps(maxDecidedRX, participants, primaryTxnId); - return decidedRx == null || decidedRx.includeDecided(txnId); + return atLeast.or(decidedRx == null || decidedRx.includeDecided(txnId) ? ACTIVE : IRRELEVANT); + } + + private Relevance supersedingRelevance(TxnId txnId, @Nullable SaveStatus saveStatus, @Nullable Timestamp executeAt, @Nullable Unseekables participants) + { + if (txnId.isSyncPoint()) + executeAt = txnId; + + if (saveStatus != null) + { + switch (saveStatus.known.deps()) + { + default: throw UnhandledEnum.unknown(saveStatus.known.deps()); + case NoDeps: throw UnhandledEnum.invalid(NoDeps); + case DepsUnknown: + // SyncPoint do not collect additional dependencies in their Accept phase; to support this we must + // wait for any undecided future sync point (prior to the latest decided one after us) to decide itself + // This does not risk deadlock as there is no fast path to recover for sync points + if (txnId.isSyncPoint() && saveStatus.is(Status.PreAccepted)) + break; + case DepsErased: + return IRRELEVANT; + case DepsFromCoordinator: + case DepsProposed: + case DepsProposedFixed: + executeAt = txnId; + case DepsCommitted: + case DepsKnown: + } + } + + if ((executeAt == null || executeAt.compareTo(primaryTxnId) > 0) && (participants == null || participants.intersects(searchFor))) + { + // TODO (desired): for pre-filter we can terminate here; only need to continue on construction + return SUPERSEDING; + } + return IRRELEVANT; + } + + public final boolean isIgnorableFutureRx(TxnId txnId, Unseekables participants) + { + return txnId.isSyncPoint() && + (txnId.compareTo(maxRx) > 0 || + (participants != null + && txnId.compareTo(primaryTxnId) > 0 + && txnId.compareTo(minVisitedFutureRX.foldlWithDefault(participants, TxnId::max, TxnId.MAX, TxnId.NONE)) > 0)); } public final Summary ifRelevant(TxnId txnId, Timestamp executeAt, SaveStatus saveStatus, Durability durability, StoreParticipants participants, @Nullable PartialDeps partialDeps) @@ -342,87 +472,37 @@ public final

Summary ifRelevant(TxnId txnId, Timestamp executeAt, SaveStatus public final

Summary ifRelevant(TxnId txnId, Timestamp executeAt, SaveStatus saveStatus, Durability durability, Participants touches, @Nullable P deps, TriPredicate> depTester) { - SummaryStatus summaryStatus = saveStatus.summary; - if (summaryStatus == null) - return null; - - if (!txnId.is(testKind)) - return null; - - if (txnId.compareTo(minTxnId) < 0 || txnId.compareTo(maxTxnId) > 0) - return null; - - boolean mayFilterAsDecided = maxDecidedRX != null && (txnId.isSyncPoint() || durability.isDurablyCommitted()); - if (mayFilterAsDecided && decidedRx != null && decidedRx.excludeDecided(txnId)) - return null; - - // start in search key domain, since this is what we consult to decide if can be recovered - Unseekables intersecting = searchKeysOrRanges.intersecting(touches, Minimal); - if (intersecting.isEmpty()) - return null; - - if (redundantBefore != null) - { - // TODO (expected): consider whether this is necessary (and document it). - Unseekables newIntersecting = redundantBefore.foldlWithBounds(intersecting, (e, accum, start, end) -> { - if (e.gcBefore.compareTo(txnId) <= 0) - return accum; - return accum.without(Ranges.of(start.rangeFactory().newRange(start, end))); - }, intersecting, ignore -> false); - - if (newIntersecting.isEmpty()) - return null; - - intersecting = newIntersecting; - } - - if (mayFilterAsDecided) - { - DecidedRX decidedRx = forDeps(maxDecidedRX, intersecting, primaryTxnId); - if (decidedRx != null && decidedRx.excludeDecided(txnId)) - return null; - } - - IsDep isDep = null; - if (findAsDep != null) - { - if (deps == null || !isEligibleDep(summaryStatus, findAsDep, txnId, executeAt)) - { - isDep = IsDep.NOT_ELIGIBLE; - } - else - { - boolean isCoordDeps = summaryStatus.compareTo(ACCEPTED) < 0; - boolean isAnyDep = depTester.test(deps, findAsDep, intersecting); - isDep = isAnyDep ? (isCoordDeps ? IsDep.IS_COORD_DEP : IsDep.IS_PROPOSED_OR_STABLE_DEP) - : (isCoordDeps ? IsDep.IS_NOT_COORD_DEP : IsDep.IS_NOT_PROPOSED_OR_STABLE_DEP); - } - } + Relevance relevance = relevance(txnId, saveStatus, durability, executeAt, touches); + return get(relevance, txnId, executeAt, saveStatus, durability, touches, deps, depTester); + } - // convert to the domain of the command we're loading - intersecting = touches.intersecting(intersecting, Minimal); - return construct(txnId, executeAt, summaryStatus, durability, isDep, intersecting); + public final

Summary get(Relevance relevance, Command cmd) + { + return get(relevance, cmd.txnId(), cmd.executeAtOrTxnId(), cmd.saveStatus(), cmd.durability(), cmd.participants().stillTouches(), cmd.partialDeps(), SummaryLoader::isDep); } - final boolean isEligibleDep(SummaryStatus status, TxnId findAsDep, TxnId txnId, Timestamp executeAt) + public final

Summary get(Relevance relevance, TxnId txnId, Timestamp executeAt, SaveStatus saveStatus, Durability durability, Participants touches, @Nullable P deps, TriPredicate> depTester) { - switch (status) + touches = touches.intersecting(searchFor, Minimal); + IsDep isDep = loadKeysFor == RECOVERY ? IsDep.NOT_ELIGIBLE : null; + switch (relevance) { - default: throw new UnhandledEnum(status); - case NOT_DIRECTLY_WITNESSED: - case INVALIDATED: - return false; - case NOTACCEPTED: - case PREACCEPTED: - if (!txnId.is(PrivilegedCoordinatorWithDeps)) - return false; - case ACCEPTED: - return txnId.compareTo(findAsDep) > 0; - case COMMITTED: - case APPLIED: - case STABLE: - return executeAt.compareTo(findAsDep) > 0; + default: throw new UnhandledEnum(relevance); + case IRRELEVANT: return null; + case BOTH: + case SUPERSEDING: + if (deps != null) + { + boolean isCoordDeps = saveStatus.summary.compareTo(ACCEPTED) < 0; + boolean isAnyDep = depTester.test(deps, primaryTxnId, touches); + isDep = isAnyDep ? (isCoordDeps ? IsDep.IS_COORD_DEP : IsDep.IS_PROPOSED_OR_STABLE_DEP) + : (isCoordDeps ? IsDep.IS_NOT_COORD_DEP : IsDep.IS_NOT_PROPOSED_OR_STABLE_DEP); + } + else Invariants.require(txnId.isSyncPoint() && saveStatus.summary == PREACCEPTED); + case ACTIVE: } + return construct(txnId, executeAt, saveStatus.summary, durability, isDep, relevance, touches); + } private static boolean isDep(PartialDeps deps, TxnId find, Unseekables intersecting) @@ -434,13 +514,12 @@ private static boolean isDep(PartialDeps deps, TxnId find, Unseekables inters } - protected Summary construct(TxnId txnId, Timestamp executeAt, SummaryStatus summaryStatus, Durability durability, IsDep isDep, Unseekables participants) + protected Summary construct(TxnId txnId, Timestamp executeAt, SummaryStatus summaryStatus, Durability durability, IsDep isDep, Relevance relevance, Unseekables participants) { - return new Summary(txnId, executeAt, summaryStatus, durability, isDep, participants); + return new Summary(txnId, executeAt, summaryStatus, durability, isDep, relevance, participants); } } - enum TestStartedAt { STARTED_BEFORE, STARTED_AFTER, ANY } enum ComputeIsDep { // don't test deps @@ -456,7 +535,7 @@ interface ActiveCommandVisitor default void visitMaxAppliedHlc(long maxAppliedHlc) {} } - interface AllCommandVisitor + interface SupersedingCommandVisitor { /** * Note: Durability is not guaranteed to return anything besides NotDurable; implementation is free to return more information if easily available. @@ -464,7 +543,7 @@ interface AllCommandVisitor boolean visit(Unseekable keyOrRange, TxnId txnId, Timestamp executeAt, SummaryStatus status, @Nullable IsDep dep, Durability minDurability); } - boolean visit(Unseekables keysOrRanges, TxnId testTxnId, Kinds testKind, TestStartedAt testStartedAt, Timestamp testStartAtTimestamp, ComputeIsDep computeIsDep, AllCommandVisitor visit); + boolean visit(Unseekables keysOrRanges, TxnId testTxnId, Kinds testKind, SupersedingCommandVisitor visit); /** * Visits keys first in ascending order, with equal keys visiting TxnId is ascending order. @@ -476,40 +555,28 @@ interface AllCommandVisitor interface ByTxnIdSnapshot extends CommandSummaries { NavigableMap byTxnId(); - class Helper - { - static NavigableMap slice(TestStartedAt testStartedAt, Timestamp testStartedAtTimestamp, NavigableMap map) - { - switch (testStartedAt) - { - default: throw new UnhandledEnum(testStartedAt); - case STARTED_AFTER: return map.tailMap(testStartedAtTimestamp, false); - case STARTED_BEFORE: return map.headMap(testStartedAtTimestamp, false); - case ANY: return map; - } - } - } default boolean visit(Unseekables keysOrRanges, TxnId testTxnId, Kinds testKind, - TestStartedAt testStartedAt, - Timestamp testStartedAtTimestamp, - ComputeIsDep computeIsDep, - AllCommandVisitor visit) + SupersedingCommandVisitor visit) { - NavigableMap map = Helper.slice(testStartedAt, testStartedAtTimestamp, byTxnId()); + NavigableMap map = byTxnId(); for (Summary value : map.values()) { if (!testKind.test(value)) continue; + if (!value.is(SUPERSEDING)) + continue; + Unseekables participants = value.participants; Unseekables intersecting = participants.overlapping(keysOrRanges); if (!intersecting.isEmpty()) { Timestamp executeAt = value.plainExecuteAt(); + Invariants.require(executeAt != null); SummaryStatus status = value.status(); IsDep dep = value.isDep(); for (Unseekable participant : intersecting) @@ -535,6 +602,9 @@ default void visit(Unseekables keysOrRanges, Timestamp startedBefore if (value.is(SummaryStatus.INVALIDATED)) continue; + if (!value.is(ACTIVE)) + continue; + for (Unseekable keyOrRange : value.participants.intersecting(keysOrRanges, Minimal)) visit.visit(p1, p2, value.status(), value.durability(), keyOrRange, value.plainTxnId()); } diff --git a/accord-core/src/main/java/accord/local/Commands.java b/accord-core/src/main/java/accord/local/Commands.java index 3154e7de86..e7be67e66b 100644 --- a/accord-core/src/main/java/accord/local/Commands.java +++ b/accord-core/src/main/java/accord/local/Commands.java @@ -1128,10 +1128,8 @@ private static boolean validateSafeToCleanup(RedundantBefore redundantBefore, Co return false; } - static int counter = 0; public static boolean maybeCleanup(SafeCommandStore safeStore, SafeCommand safeCommand, Command command, @Nonnull StoreParticipants newParticipants) { - ++counter; StoreParticipants cleanupParticipants = newParticipants.filter(LOAD, safeStore, command.txnId(), command.executeAtIfKnown()); Cleanup cleanup = shouldCleanup(FULL, safeStore, command, cleanupParticipants); if (cleanup == NO) @@ -1224,7 +1222,7 @@ void start(SafeCommandStore safeStore) } else { - safeStore.commandStore().execute(this, this); + safeStore.commandStore().execute(this, this, safeStore.agent()); } } @@ -1303,9 +1301,9 @@ boolean acceptInternal(SafeCommandStore safeStore) { // TODO (desired): slightly costly to invert a large partialDeps collection participants = waiting.partialDeps().participants(dep.txnId()); - Participants stillExecutes = participants.intersecting(waiting.participants().stillExecutes(), Minimal); + Participants waitsOn = participants.intersecting(waiting.participants().stillWaitsOn(), Minimal); - depSafe = maybeCleanupRedundantDependency(safeStore, waitingSafe, depSafe, stillExecutes); + depSafe = maybeCleanupRedundantDependency(safeStore, waitingSafe, depSafe, waitsOn); if (depSafe == null) continue; } diff --git a/accord-core/src/main/java/accord/local/DurableBefore.java b/accord-core/src/main/java/accord/local/DurableBefore.java index 45b658e330..e7856416a1 100644 --- a/accord-core/src/main/java/accord/local/DurableBefore.java +++ b/accord-core/src/main/java/accord/local/DurableBefore.java @@ -42,14 +42,11 @@ public class DurableBefore extends ReducingRangeMap { public static class SerializerSupport { - public static DurableBefore create(boolean inclusiveEnds, RoutingKey[] ends, Entry[] values) + public static DurableBefore create(RoutingKey[] ends, Entry[] values) { if (values.length == 0) - { - Invariants.require(ends.length == 1 && ends[0] == null); return DurableBefore.EMPTY; - } - return new DurableBefore(inclusiveEnds, ends, values); + return new DurableBefore(ends, values); } } @@ -135,9 +132,9 @@ private DurableBefore() this.min = new Entry(TxnId.NONE, TxnId.NONE); } - DurableBefore(boolean inclusiveEnds, RoutingKey[] starts, Entry[] values) + DurableBefore(RoutingKey[] starts, Entry[] values) { - super(inclusiveEnds, starts, values); + super(starts, values); if (values.length == 0) { min = new Entry(TxnId.NONE, TxnId.NONE); @@ -171,6 +168,14 @@ public static DurableBefore merge(DurableBefore a, DurableBefore b) return ReducingIntervalMap.merge(a, b, DurableBefore.Entry::max, Builder::new); } + public static DurableBefore mergeIfDifferent(DurableBefore prev, DurableBefore add) + { + DurableBefore next = DurableBefore.merge(prev, add); + if (next.equals(prev)) + return prev; + return next.equals(prev) ? prev : next; + } + public HasOutcome min(TxnId txnId, Unseekables unseekables) { return notDurableIfNull(foldlWithDefault(unseekables, Entry::mergeMin, Entry.NONE, null, txnId, test -> test == None)); @@ -218,15 +223,15 @@ private static HasOutcome notDurableIfNull(HasOutcome status) static class Builder extends AbstractBoundariesBuilder { - protected Builder(boolean inclusiveEnds, int capacity) + protected Builder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override protected DurableBefore buildInternal() { - return new DurableBefore(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new Entry[0])); + return new DurableBefore(starts.toArray(new RoutingKey[0]), values.toArray(new Entry[0])); } } diff --git a/accord-core/src/main/java/accord/local/MaxConflicts.java b/accord-core/src/main/java/accord/local/MaxConflicts.java index 557b2b1287..21dd9e0575 100644 --- a/accord-core/src/main/java/accord/local/MaxConflicts.java +++ b/accord-core/src/main/java/accord/local/MaxConflicts.java @@ -33,9 +33,9 @@ private MaxConflicts() super(); } - private MaxConflicts(boolean inclusiveEnds, Object[] tree) + private MaxConflicts(Object[] tree) { - super(inclusiveEnds, tree); + super(tree); } Timestamp get(Routables keysOrRanges) @@ -51,17 +51,17 @@ public MaxConflicts update(Unseekables keysOrRanges, Timestamp maxConflict) return update(this, keysOrRanges, maxConflict, Timestamp::mergeMax, MaxConflicts::new, Builder::new); } - private static class Builder extends AbstractBoundariesBuilder + public static class Builder extends AbstractBoundariesBuilder { - protected Builder(boolean inclusiveEnds, int capacity) + public Builder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override protected MaxConflicts buildInternal(Object[] tree) { - return new MaxConflicts(inclusiveEnds, tree); + return new MaxConflicts(tree); } } } diff --git a/accord-core/src/main/java/accord/local/MaxDecidedRX.java b/accord-core/src/main/java/accord/local/MaxDecidedRX.java index c04da97513..50bd3f3c37 100644 --- a/accord-core/src/main/java/accord/local/MaxDecidedRX.java +++ b/accord-core/src/main/java/accord/local/MaxDecidedRX.java @@ -49,6 +49,8 @@ public static final class DecidedRX public DecidedRX(TxnId any, TxnId hlcBound) { + Invariants.nonNull(any); + Invariants.nonNull(hlcBound); this.any = any; this.hlcBound = hlcBound; } @@ -56,7 +58,7 @@ public DecidedRX(TxnId any, TxnId hlcBound) @Override public String toString() { - return "{any=" + any + ",hlcBound=" + hlcBound + "}"; + return "{any=" + any + ",hlcBound=" + hlcBound + '}'; } public boolean includeDecided(TxnId txnId) @@ -117,6 +119,31 @@ static DecidedRX selectOrCreate(TxnId any, TxnId hlcBound, DecidedRX a, DecidedR return b; return new DecidedRX(any, hlcBound); } + + @Override + public boolean equals(Object that) + { + return that instanceof DecidedRX && equals((DecidedRX) that); + } + + public boolean equals(DecidedRX that) + { + return this.any.equals(that.any) && this.hlcBound.equals(that.hlcBound); + } + + @Override + public int hashCode() + { + throw new UnsupportedOperationException(); + } + } + + public static final class SerializerSupport + { + public static MaxDecidedRX create(RoutingKey[] starts, DecidedRX[] values) + { + return new MaxDecidedRX(starts, values); + } } public static final MaxDecidedRX EMPTY = new MaxDecidedRX(); @@ -126,9 +153,9 @@ private MaxDecidedRX() super(); } - private MaxDecidedRX(boolean inclusiveEnds, RoutingKey[] starts, DecidedRX[] values) + private MaxDecidedRX(RoutingKey[] starts, DecidedRX[] values) { - super(inclusiveEnds, starts, values); + super(starts, values); } DecidedRX min(Routables keysOrRanges) @@ -196,15 +223,15 @@ public MaxDecidedRX update(Unseekables keysOrRanges, TxnId syncId) static class Builder extends AbstractBoundariesBuilder { - protected Builder(boolean inclusiveEnds, int capacity) + protected Builder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override protected MaxDecidedRX buildInternal() { - return new MaxDecidedRX(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new DecidedRX[0])); + return new MaxDecidedRX(starts.toArray(new RoutingKey[0]), values.toArray(new DecidedRX[0])); } } diff --git a/accord-core/src/main/java/accord/local/Node.java b/accord-core/src/main/java/accord/local/Node.java index 9c4f1620c4..6b843698bd 100644 --- a/accord-core/src/main/java/accord/local/Node.java +++ b/accord-core/src/main/java/accord/local/Node.java @@ -205,14 +205,10 @@ public Node(Id id, MessageSink messageSink, this.agent = agent; this.random = random; this.persistDurableBefore = new PersistentField<>(() -> durableBefore, - (input, prev) -> { - DurableBefore next = DurableBefore.merge(input, prev); - if (next.equals(prev)) - return prev; - return next.equals(prev) ? prev : next; - }, + DurableBefore::merge, DurableBefore::mergeIfDifferent, safeDurableBeforePersister(durableBeforePersister), - this::setPersistedDurableBefore); + this::setPersistedDurableBefore, + run -> someExecutor().execute(run)); this.commandStores = factory.create(this, agent, dataSupplier.get(), random.fork(), journal, shardDistributor, progressLogFactory.apply(this), localListenersFactory.apply(this)); this.topology = new TopologyManager(topologySorter, this, topologyService, time, timeouts); this.durabilityService = new DurabilityService(this); @@ -319,7 +315,7 @@ public AsyncResult markDurable(Ranges ranges, TxnId majorityBefore, TxnId uni public AsyncResult markDurable(DurableBefore addDurableBefore) { - return withEpochExact(addDurableBefore.maxEpoch(), (AsyncExecutor)null, () -> persistDurableBefore.mergeAndUpdate(addDurableBefore).chain()) + return withEpochExact(addDurableBefore.maxEpoch(), (AsyncExecutor)null, () -> persistDurableBefore.save(addDurableBefore).chain()) .beginAsResult(); } diff --git a/accord-core/src/main/java/accord/local/PreLoadContext.java b/accord-core/src/main/java/accord/local/PreLoadContext.java index ba2b717007..68de4a048e 100644 --- a/accord-core/src/main/java/accord/local/PreLoadContext.java +++ b/accord-core/src/main/java/accord/local/PreLoadContext.java @@ -23,6 +23,7 @@ import accord.primitives.AbstractUnseekableKeys; import accord.primitives.Routable; import accord.primitives.RoutingKeys; +import accord.primitives.Timestamp; import accord.primitives.TxnId; import accord.primitives.Unseekables; @@ -41,6 +42,8 @@ /** * Lists txnids and keys of commands and commands for key that will be needed for an operation. Used * to ensure the necessary state is in memory for an operation before it executes. + * + * TODO (desired): rename to simply Context, or LoadContext */ public interface PreLoadContext { @@ -95,8 +98,11 @@ default void forEachId(Consumer consumer) default Unseekables keys() { return RoutingKeys.EMPTY; } default LoadKeys loadKeys() { return NONE; } + default LoadKeysFor loadKeysFor() { return WRITE; } + default Timestamp executeAt() { return primaryTxnId(); } + default boolean isEmpty() { boolean isEmpty = primaryTxnId() == null && keys().isEmpty(); diff --git a/accord-core/src/main/java/accord/local/RedundantBefore.java b/accord-core/src/main/java/accord/local/RedundantBefore.java index 1b54114dc7..76920d13b5 100644 --- a/accord-core/src/main/java/accord/local/RedundantBefore.java +++ b/accord-core/src/main/java/accord/local/RedundantBefore.java @@ -93,9 +93,9 @@ public interface RedundantBeforeSupplier public static class SerializerSupport { - public static RedundantBefore create(boolean inclusiveEnds, RoutingKey[] ends, Bounds[] values) + public static RedundantBefore create(RoutingKey[] ends, Bounds[] values) { - return new RedundantBefore(inclusiveEnds, ends, values); + return new RedundantBefore(ends, values); } } @@ -639,7 +639,7 @@ static Ranges withoutAnyRetired(Bounds bounds, @Nonnull Ranges notRetired) return notRetired.without(Ranges.of(bounds.range)); } - RedundantStatus get(TxnId txnId, @Nullable Timestamp applyAtIfKnown) + public RedundantStatus get(TxnId txnId, @Nullable Timestamp applyAtIfKnown) { if (wasOwned(txnId)) { @@ -820,9 +820,9 @@ private RedundantBefore() minEndEpoch = Long.MAX_VALUE; } - RedundantBefore(boolean inclusiveEnds, RoutingKey[] starts, Bounds[] values) + RedundantBefore(RoutingKey[] starts, Bounds[] values) { - super(inclusiveEnds, starts, values); + super(starts, values); staleRanges = extractRanges(values, b -> b.staleUntilAtLeast != null); lostRanges = extractRanges(values, Bounds::hasLostOwnership); TxnId maxUnready = TxnId.NONE, maxGcBefore = TxnId.NONE, maxShardAppliedBefore = TxnId.NONE; @@ -914,7 +914,7 @@ public static RedundantBefore create(AbstractRanges ranges, long startEpoch, lon return EMPTY; Bounds bounds = new Bounds(null, startEpoch, endEpoch, new TxnId[] { bound }, new int[] { (int) (status.encoded & ONLY_LE_MASK), (int)status.encoded }, staleUntilAtLeast); - Builder builder = new Builder(ranges.get(0).endInclusive(), ranges.size() * 2); + Builder builder = new Builder(ranges.size() * 2); for (int i = 0 ; i < ranges.size() ; ++i) { Range cur = ranges.get(i); @@ -1114,9 +1114,9 @@ public Participants withoutShardAppliedLocallySynced(TxnId txnId, Participant public static class Builder extends AbstractIntervalBuilder { - public Builder(boolean inclusiveEnds, int capacity) + public Builder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override @@ -1158,7 +1158,7 @@ public void append(RoutingKey start, RoutingKey end, @Nonnull Bounds value) @Override protected RedundantBefore buildInternal() { - return new RedundantBefore(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new Bounds[0])); + return new RedundantBefore(starts.toArray(new RoutingKey[0]), values.toArray(new Bounds[0])); } } @@ -1277,17 +1277,26 @@ public final boolean hasLocallyRedundantDependencies(TxnId minimumDependencyId, @Override public String toString() { - return "gc:" + toString(GC_BEFORE) - + "\nlocal:" + toString(LOCALLY_DURABLE_TO_DATA_STORE, LOCALLY_DURABLE_TO_COMMAND_STORE) - + "\nbootstrap:" + toString(UNREADY); + return toString(", "); } - private String toString(Property p1) + public String toString(String delimiter) { - return toString(p1, null); + StringBuilder sb = new StringBuilder(); + append(sb, delimiter, "gc:", GC_BEFORE); + append(sb, delimiter, "applied:", LOCALLY_APPLIED); + append(sb, delimiter, "command_store:", LOCALLY_DURABLE_TO_COMMAND_STORE); + append(sb, delimiter, "data_store:", LOCALLY_DURABLE_TO_DATA_STORE); + append(sb, delimiter, "unready:", UNREADY); + return sb.toString(); } - private String toString(Property p1, Property p2) + private void append(StringBuilder builder, String delimiter, String prefix, Property p1) + { + append(builder, delimiter, prefix, p1, null); + } + + private void append(StringBuilder builder, String delimiter, String prefix, Property p1, Property p2) { TreeMap> map = new TreeMap<>(); foldl((e, m, pp1, pp2) -> { @@ -1297,8 +1306,14 @@ private String toString(Property p1, Property p2) return m; }, map, p1, p2, i -> false); - return map.descendingMap().entrySet().stream() - .map(e -> (e.getKey().equals(TxnId.NONE) ? "none" : e.getKey().toString()) + ":" + Ranges.ofSorted(e.getValue().toArray(new Range[0])).mergeTouching()) - .collect(Collectors.joining(", ", "{", "}")); + if (map.size() == 0 || map.size() == 1 && map.firstKey().equals(TxnId.NONE)) + return; + + if (builder.length() > 0) + builder.append(delimiter); + builder.append(prefix); + builder.append(map.descendingMap().entrySet().stream() + .map(e -> (e.getKey().equals(TxnId.NONE) ? "none" : e.getKey().toString()) + ':' + Ranges.ofSorted(e.getValue().toArray(new Range[0])).mergeTouching()) + .collect(Collectors.joining(", ", "{", "}"))); } } diff --git a/accord-core/src/main/java/accord/local/RedundantStatus.java b/accord-core/src/main/java/accord/local/RedundantStatus.java index 4ac9c8e227..24d3026839 100644 --- a/accord-core/src/main/java/accord/local/RedundantStatus.java +++ b/accord-core/src/main/java/accord/local/RedundantStatus.java @@ -109,13 +109,13 @@ public enum Property * We have applied the preceding transactions durably to the store, so that we can safely truncate the Write * information as we will not need to replay it to the store */ - LOCALLY_DURABLE_TO_DATA_STORE (false, false, LE, LOCALLY_APPLIED), + LOCALLY_DURABLE_TO_DATA_STORE (false, true, LE, LOCALLY_APPLIED), /** * We have applied the preceding transactions durably to all summary structures, so that on restart we do * not need to replay the transaction to restore any internal state. */ - LOCALLY_DURABLE_TO_COMMAND_STORE (false, false, LE, LOCALLY_APPLIED), + LOCALLY_DURABLE_TO_COMMAND_STORE (false, true, LE, LOCALLY_APPLIED), /** * We have fully executed until across all a majority of replicas for the range in question, diff --git a/accord-core/src/main/java/accord/local/RejectBefore.java b/accord-core/src/main/java/accord/local/RejectBefore.java index 67b072dac4..e969a14797 100644 --- a/accord-core/src/main/java/accord/local/RejectBefore.java +++ b/accord-core/src/main/java/accord/local/RejectBefore.java @@ -31,11 +31,13 @@ public class RejectBefore extends ReducingRangeMap { + public static final RejectBefore EMPTY = new RejectBefore(); + public static class SerializerSupport { - public static RejectBefore create(boolean inclusiveEnds, RoutingKey[] ends, Timestamp[] values) + public static RejectBefore create(RoutingKey[] ends, Timestamp[] values) { - return new RejectBefore(inclusiveEnds, ends, values); + return new RejectBefore(ends, values); } } @@ -44,9 +46,9 @@ public RejectBefore() super(); } - protected RejectBefore(boolean inclusiveEnds, RoutingKey[] starts, Timestamp[] values) + protected RejectBefore(RoutingKey[] starts, Timestamp[] values) { - super(inclusiveEnds, starts, values); + super(starts, values); } public static RejectBefore add(RejectBefore existing, Ranges ranges, TxnId value) @@ -76,22 +78,22 @@ public static RejectBefore create(AbstractRanges ranges, Timestamp value) throw new IllegalArgumentException("value is null"); if (ranges.isEmpty()) - return new RejectBefore(); + return EMPTY; return create(ranges, value, Builder::new); } static class Builder extends AbstractBoundariesBuilder { - protected Builder(boolean inclusiveEnds, int capacity) + protected Builder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override protected RejectBefore buildInternal() { - return new RejectBefore(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new Timestamp[0])); + return new RejectBefore(starts.toArray(new RoutingKey[0]), values.toArray(new Timestamp[0])); } } } diff --git a/accord-core/src/main/java/accord/local/SafeCommandStore.java b/accord-core/src/main/java/accord/local/SafeCommandStore.java index 99faad7a0b..ba0ecca6b2 100644 --- a/accord-core/src/main/java/accord/local/SafeCommandStore.java +++ b/accord-core/src/main/java/accord/local/SafeCommandStore.java @@ -21,9 +21,13 @@ import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; +import java.util.function.Consumer; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import accord.api.Agent; import accord.api.DataStore; import accord.api.LocalListeners; @@ -80,6 +84,8 @@ */ public abstract class SafeCommandStore implements RangesForEpochSupplier, RedundantBeforeSupplier, CommandSummaries { + private static final Logger logger = LoggerFactory.getLogger(SafeCommandStore.class); + private static final int MAX_REENTRANCY = 50; private int reentrancyCounter; public boolean tryRecurse() @@ -281,9 +287,11 @@ public SafeCommandsForKey get(RoutingKey key) protected void update(Command prev, Command updated, boolean force) { + updateExclusiveSyncPoint(prev, updated, force); updateMaxConflicts(prev, updated, force); + if (updated.txnId().is(Range)) + updateCommandsForRanges(prev, updated, force); updateCommandsForKey(prev, updated, force); - updateExclusiveSyncPoint(prev, updated, force); } public void updateExclusiveSyncPoint(Command prev, Command updated, boolean force) @@ -291,33 +299,26 @@ public void updateExclusiveSyncPoint(Command prev, Command updated, boolean forc if (!updated.txnId().isSyncPoint() || updated.txnId().domain() != Range) return; if (updated.route() == null) return; - List listeners = commandStore().syncPointListeners; - if (listeners != null) - { - for (SyncPointListener listener : listeners) - listener.update(this, updated); - } - - SaveStatus oldSaveStatus = prev == null ? SaveStatus.Uninitialised : prev.saveStatus(); + SaveStatus prevSaveStatus = prev == null ? SaveStatus.Uninitialised : prev.saveStatus(); SaveStatus newSaveStatus = updated.saveStatus(); - if (newSaveStatus.known.isDefinitionKnown() && (force || !oldSaveStatus.known.isDefinitionKnown())) + if (newSaveStatus.known.isDefinitionKnown() && (force || !prevSaveStatus.known.isDefinitionKnown())) { Ranges ranges = updated.participants().touches().toRanges(); - commandStore().markExclusiveSyncPoint(this, updated.txnId(), ranges); + commandStore().upsertRejectBefore(this, updated.txnId(), ranges); } - if (newSaveStatus.compareTo(Committed) >= 0 && newSaveStatus.compareTo(TruncatedApply) <= 0 && (force || oldSaveStatus.compareTo(Committed) < 0)) + if (newSaveStatus.compareTo(Committed) >= 0 && newSaveStatus.compareTo(TruncatedApply) <= 0 && (force || prevSaveStatus.compareTo(Committed) < 0)) { Ranges ranges = updated.participants().owns().toRanges(); commandStore().markExclusiveSyncPointDecided(this, updated.txnId(), ranges); } - if (newSaveStatus == Applied && (force || oldSaveStatus != Applied)) + if (newSaveStatus == Applied && (force || prevSaveStatus != Applied)) { Ranges ranges = updated.participants().touches().toRanges(); TxnId txnIdWithFlags = (TxnId)updated.executeAt(); - commandStore().markExclusiveSyncPointLocallyApplied(this, txnIdWithFlags, ranges); + commandStore().markExclusiveSyncPointLocallyApplied(this, txnIdWithFlags, ranges, prevSaveStatus); } if (updated.partialDeps() != null) @@ -341,6 +342,15 @@ public void updateExclusiveSyncPoint(Command prev, Command updated, boolean forc if (addRedundantBefore != RedundantBefore.EMPTY) upsertRedundantBefore(addRedundantBefore); } + + // invoke listeners only after updating redundantBefore + List listeners = commandStore().syncPointListeners; + if (listeners != null) + { + logger.debug("Notifying SyncPoint listeners"); + for (SyncPointListener listener : listeners) + listener.update(this, updated); + } } public void updateMaxConflicts(Command prev, Command updated, boolean force) @@ -357,36 +367,8 @@ public void updateMaxConflicts(Command prev, Command updated, boolean force) commandStore().updateMaxConflicts(prev, updated, force); } - /** - * Methods that implementors can use to capture changes to auxiliary collections: - */ - - public abstract void upsertRedundantBefore(RedundantBefore addRedundantBefore); - - protected void unsafeSetRedundantBefore(RedundantBefore newRedundantBefore) + public void updateCommandsForRanges(Command prev, Command updated, boolean force) { - commandStore().unsafeSetRedundantBefore(newRedundantBefore); - } - - protected void unsafeUpsertRedundantBefore(RedundantBefore addRedundantBefore) - { - commandStore().unsafeUpsertRedundantBefore(addRedundantBefore); - commandStore().updatedRedundantBefore(this, addRedundantBefore); - } - - public void setBootstrapBeganAt(NavigableMap newBootstrapBeganAt) - { - commandStore().unsafeSetBootstrapBeganAt(newBootstrapBeganAt); - } - - public void setSafeToRead(NavigableMap newSafeToRead) - { - commandStore().unsafeSetSafeToRead(newSafeToRead); - } - - public void setRangesForEpoch(CommandStores.RangesForEpoch rangesForEpoch) - { - commandStore().unsafeSetRangesForEpoch(rangesForEpoch); } public void updateCommandsForKey(Command prev, Command next, boolean force) @@ -403,8 +385,7 @@ public void updateCommandsForKey(Command prev, Command next, boolean force) // updateUnmanagedCommandsForKey(this, next, REGISTER_DEPS_ONLY); } - abstract protected void persistFieldUpdates(); - + // TODO (expected): should these and related methods live in CommandStore for consistency? private static void updateManagedCommandsForKey(SafeCommandStore safeStore, Command prev, Command next, boolean forceNotify) { StoreParticipants participants = next.participants().supplement(prev.participants()); @@ -516,15 +497,6 @@ private static void updateUnmanagedCommandsForKey(SafeCommandStore safeStore, Co } } - private static TxnId maxTxnId(KeyDeps keyDeps, RoutingKey key) - { - int i = keyDeps.keys().indexOf(key); - if (i < 0) - return TxnId.NONE; - SortedList txnIdsForKey = keyDeps.txnIdsForKeyIndex(i); - return txnIdsForKey.get(txnIdsForKey.size() - 1); - } - private static void updateUnmanagedCommandsForKey(SafeCommandStore safeStore, Unseekables update, TxnId txnId, UpdateUnmanagedMode mode) { SafeCommand safeCommand = safeStore.get(txnId); @@ -560,8 +532,8 @@ private static void registerTransitiveRangeDeps(SafeCommandStore safeStore, TxnI AsyncChains.chain(() -> commandStore.markingVisible(syncId, waitingOn)) .flatMap(ignore -> AsyncChains.reduce(async, Reduce.toNull(), null)) .begin((success, fail) -> { - if (fail == null) commandStore.execute((PreLoadContext.Empty)() -> "Mark Synced", safeStore0 -> commandStore.markVisible(safeStore0, syncId, waitingOn)); - else commandStore.execute((PreLoadContext.Empty)() -> "Unmark Syncing", safeStore0 -> commandStore.cancelMarkingVisible(syncId, waitingOn)); + if (fail == null) commandStore.execute((PreLoadContext.Empty)() -> "Mark Synced", (Consumer) safeStore0 -> commandStore.markVisible(safeStore0, syncId, waitingOn), commandStore.agent()); + else commandStore.execute((PreLoadContext.Empty)() -> "Unmark Syncing", (Consumer) safeStore0 -> commandStore.cancelMarkingVisible(syncId, waitingOn), commandStore.agent); }); } @@ -584,6 +556,44 @@ private static void registerTransitive(SafeCommandStore safeStore, TxnId txnId, safeCommand.updateParticipants(safeStore, safeCommand.current().participants().supplement(null, witnessedBy)); } + /** + * Methods that implementors can use to capture changes to auxiliary collections: + */ + + public abstract void upsertRedundantBefore(RedundantBefore addRedundantBefore); + public void reportDurable(RedundantBefore addRedundantBefore, int flags) + { + upsertRedundantBefore(addRedundantBefore); + } + + protected void unsafeSetRedundantBefore(RedundantBefore newRedundantBefore) + { + commandStore().unsafeSetRedundantBefore(newRedundantBefore); + } + + protected void unsafeUpsertRedundantBefore(RedundantBefore addRedundantBefore) + { + commandStore().unsafeUpsertRedundantBefore(addRedundantBefore); + commandStore().upsertedRedundantBefore(this, addRedundantBefore); + } + + public void setBootstrapBeganAt(NavigableMap newBootstrapBeganAt) + { + commandStore().unsafeSetBootstrapBeganAt(newBootstrapBeganAt); + } + + public void setSafeToRead(NavigableMap newSafeToRead) + { + commandStore().unsafeSetSafeToRead(newSafeToRead); + } + + public void setRangesForEpoch(CommandStores.RangesForEpoch rangesForEpoch) + { + commandStore().unsafeSetRangesForEpoch(rangesForEpoch); + } + + protected abstract void persistFieldUpdates(); + public abstract CommandStore commandStore(); public abstract DataStore dataStore(); public abstract Agent agent(); 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 442b905738..7f49e518e5 100644 --- a/accord-core/src/main/java/accord/local/cfk/CommandsForKey.java +++ b/accord-core/src/main/java/accord/local/cfk/CommandsForKey.java @@ -21,7 +21,6 @@ import java.util.Arrays; import java.util.Iterator; import java.util.Objects; -import java.util.function.Predicate; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -38,11 +37,9 @@ import accord.local.SafeCommand; import accord.local.SafeCommandStore; import accord.local.CommandSummaries.ActiveCommandVisitor; -import accord.local.CommandSummaries.AllCommandVisitor; +import accord.local.CommandSummaries.SupersedingCommandVisitor; import accord.local.CommandSummaries.IsDep; import accord.local.CommandSummaries.SummaryStatus; -import accord.local.CommandSummaries.ComputeIsDep; -import accord.local.CommandSummaries.TestStartedAt; import accord.primitives.Deps; import accord.primitives.Deps.DepRelationList; import accord.primitives.RoutingKeys; @@ -58,7 +55,6 @@ import accord.primitives.TxnId; import accord.utils.Invariants; import accord.utils.SortedArrays; -import accord.utils.UnhandledEnum; import accord.utils.btree.BTree; import static accord.api.ProgressLog.BlockedUntil.CanApply; @@ -97,7 +93,6 @@ import static accord.local.cfk.Pruning.prunedBeforeId; import static accord.local.cfk.UpdateUnmanagedMode.UPDATE; import static accord.local.cfk.Updating.insertOrUpdate; -import static accord.local.CommandSummaries.ComputeIsDep.IGNORE; import static accord.local.cfk.Updating.maybeUpdateMaxAppliedUnreadyWriteById; import static accord.primitives.Known.KnownExecuteAt.ApplyAtKnown; import static accord.primitives.Routable.Domain.Key; @@ -1285,7 +1280,7 @@ private TxnId nextWaitingToApply(Kinds kinds, @Nullable Timestamp untilExecuteAt TxnInfo result = committedByExecuteAt[i]; if (untilExecuteAt != null && result.executeAt.compareTo(untilExecuteAt) >= 0) return null; - return result; + return result.plainTxnId(); } @VisibleForTesting @@ -1316,88 +1311,66 @@ public TxnId blockedOnTxnId(TxnId txnId, @Nullable Timestamp executeAt) * (either executeAt is before, or txnId is before and command is not committed so deps do not extend to executeAt) *

*/ - public boolean visit(TxnId testTxnId, - Kinds testKind, - TestStartedAt testStartedAt, - Timestamp testStartedAtTimestamp, - ComputeIsDep computeIsDep, - Predicate testStatus, - AllCommandVisitor visitor) - { - int start, end, loadingIndex = 0; + public boolean visit(TxnId testTxnId, Kinds testKind, SupersedingCommandVisitor visitor) + { + int loadingIndex = 0; // if this is null the TxnId is known in byId // otherwise, it must be non-null and represents the transactions (if any) that have requested it be loaded due to being pruned TxnId prunedBefore = prunedBefore(); TxnId[] loadingFor = null; + if (Arrays.binarySearch(byId, testTxnId) < 0) { - int insertPos = Arrays.binarySearch(byId, testStartedAtTimestamp); - if (insertPos < 0) - { - loadingFor = NOT_LOADING_PRUNED; - insertPos = -1 - insertPos; - if (computeIsDep != IGNORE && testTxnId.compareTo(prunedBefore) < 0) - loadingFor = loadingPrunedFor(loadingPruned, testTxnId, NOT_LOADING_PRUNED); - } - - switch (testStartedAt) - { - default: throw new UnhandledEnum(testStartedAt); - case STARTED_BEFORE: start = 0; end = insertPos; break; - case STARTED_AFTER: start = insertPos; end = byId.length; break; - case ANY: start = 0; end = byId.length; - } + loadingFor = NOT_LOADING_PRUNED; + if (testTxnId.compareTo(prunedBefore) < 0) + loadingFor = loadingPrunedFor(loadingPruned, testTxnId, NOT_LOADING_PRUNED); } - for (int i = start; i < end ; ++i) + for (int i = 0; i < byId.length ; ++i) { TxnInfo txn = byId[i]; if (!txn.is(testKind)) continue; SummaryStatus summaryStatus = txn.summaryStatus(); - if (summaryStatus == null || (testStatus != null && !testStatus.test(summaryStatus))) continue; + if (summaryStatus == null) continue; Timestamp executeAt = txn.executeAt; - IsDep dep = null; - if (computeIsDep != IGNORE) + IsDep dep; + if (!txn.hasDeps() || (txn.depsKnownUntilExecuteAt() ? executeAt : txn).compareTo(testTxnId) <= 0) + { + dep = NOT_ELIGIBLE; + } + else { - if (!txn.hasDeps() || (summaryStatus == SummaryStatus.ACCEPTED ? txn : executeAt).compareTo(testTxnId) <= 0) + boolean hasAsDep; + if (loadingFor == null) + { + TxnId[] missing = txn.missing(); + hasAsDep = missing == NO_TXNIDS || Arrays.binarySearch(txn.missing(), testTxnId) < 0; + } + else if (loadingFor == NOT_LOADING_PRUNED) { - dep = NOT_ELIGIBLE; + hasAsDep = false; } else { - boolean hasAsDep; - if (loadingFor == null) - { - TxnId[] missing = txn.missing(); - hasAsDep = missing == NO_TXNIDS || Arrays.binarySearch(txn.missing(), testTxnId) < 0; - } - else if (loadingFor == NOT_LOADING_PRUNED) + // we could use expontentialSearch and moving index for improved algorithmic complexity, + // but since should be rarely taken path probably not worth code complexity + loadingIndex = SortedArrays.exponentialSearch(loadingFor, loadingIndex, loadingFor.length, txn); + if (loadingIndex >= 0) { - hasAsDep = false; + hasAsDep = !loadingFor[loadingIndex].is(UNSTABLE); + ++loadingIndex; } else { - // we could use expontentialSearch and moving index for improved algorithmic complexity, - // but since should be rarely taken path probably not worth code complexity - loadingIndex = SortedArrays.exponentialSearch(loadingFor, loadingIndex, loadingFor.length, txn); - if (loadingIndex >= 0) - { - hasAsDep = !loadingFor[loadingIndex].is(UNSTABLE); - ++loadingIndex; - } - else - { - hasAsDep = false; - loadingIndex = -1 - loadingIndex; - } + hasAsDep = false; + loadingIndex = -1 - loadingIndex; } - - if (txn.compareTo(ACCEPTED) >= 0) dep = hasAsDep ? IS_PROPOSED_OR_STABLE_DEP : IS_NOT_PROPOSED_OR_STABLE_DEP; - else if (txn.is(PrivilegedCoordinatorWithDeps)) dep = hasAsDep ? IS_COORD_DEP : IS_NOT_COORD_DEP; - else dep = NOT_ELIGIBLE; } - } + if (txn.compareTo(ACCEPTED) >= 0) dep = hasAsDep ? IS_PROPOSED_OR_STABLE_DEP : IS_NOT_PROPOSED_OR_STABLE_DEP; + else if (txn.is(PrivilegedCoordinatorWithDeps)) dep = hasAsDep ? IS_COORD_DEP : IS_NOT_COORD_DEP; + else dep = NOT_ELIGIBLE; + } if (!visitor.visit(key, txn.plainTxnId(), executeAt, summaryStatus, dep, txn.durability())) return false; } diff --git a/accord-core/src/main/java/accord/local/durability/DurabilityQueue.java b/accord-core/src/main/java/accord/local/durability/DurabilityQueue.java index 450b0157dc..c80d9559d1 100644 --- a/accord-core/src/main/java/accord/local/durability/DurabilityQueue.java +++ b/accord-core/src/main/java/accord/local/durability/DurabilityQueue.java @@ -55,7 +55,9 @@ import accord.utils.async.Cancellable; import accord.utils.btree.BTree; import accord.utils.btree.IntervalBTree; +import accord.utils.btree.IntervalBTree.IntervalComparators; +import static accord.api.ProtocolModifiers.RangeSpec.isEndInclusive; import static accord.coordinate.ExecuteSyncPoint.coordinateIncluding; import static accord.local.durability.DurabilityQueue.Status.ABANDONED; import static accord.local.durability.DurabilityQueue.Status.ACTIVE; @@ -146,7 +148,7 @@ public DurabilityResults execute(PartialSyncPoint syncPoint, int attempt) } } - private static final PendingComparators BY_RANGE = new PendingComparators(); + private static final IntervalComparators BY_RANGE = new InclusiveEndPendingComparators(); private static final Comparator BY_PRIORITY = (a, b) -> { if ((a.status == ACTIVE) != (b.status == ACTIVE)) return a.status == ACTIVE ? -1 : 1; @@ -159,7 +161,7 @@ public DurabilityResults execute(PartialSyncPoint syncPoint, int attempt) return a.syncPoint.syncId.compareTo(b.syncPoint.syncId); }; - private static class PendingComparators implements IntervalBTree.IntervalComparators + private static class InclusiveEndPendingComparators implements IntervalComparators { @Override public Comparator totalOrder() { @@ -310,12 +312,13 @@ static final class PendingQueue extends IntrusivePriorityHeap public DurabilityQueue(Node node) { - this.adapter = new NodeAdapter(node); + this(new NodeAdapter(node)); } public DurabilityQueue(Adapter adapter) { this.adapter = adapter; + Invariants.require(isEndInclusive(), "Need to implement range-exclusive IntervalComparators"); } void submit(SyncPoint syncPoint, @Nullable DurabilityRequest request) diff --git a/accord-core/src/main/java/accord/messages/Accept.java b/accord-core/src/main/java/accord/messages/Accept.java index f6fa2cb507..7f0110bbf9 100644 --- a/accord-core/src/main/java/accord/messages/Accept.java +++ b/accord-core/src/main/java/accord/messages/Accept.java @@ -44,10 +44,12 @@ import accord.primitives.TxnId; import accord.topology.Topologies; import accord.utils.Invariants; +import accord.utils.TinyEnumSet; import accord.utils.UnhandledEnum; import accord.utils.async.Cancellable; import static accord.api.ProtocolModifiers.Toggles.filterDuplicateDependenciesFromAcceptReply; +import static accord.api.ProtocolModifiers.Toggles.syncPointsTrackUnstableMediumPathDependencies; import static accord.local.Commands.AcceptOutcome.Redundant; import static accord.local.Commands.AcceptOutcome.RejectedBallot; import static accord.local.Commands.AcceptOutcome.Success; @@ -63,40 +65,78 @@ public class Accept extends RouteRequest.WithUnsynced { public static class SerializerSupport { - public static Accept create(TxnId txnId, Route scope, long waitForEpoch, long minEpoch, Kind kind, Ballot ballot, Timestamp executeAt, PartialDeps partialDeps, boolean isPartialAccept) + public static Accept create(TxnId txnId, Route scope, long waitForEpoch, long minEpoch, Kind kind, Ballot ballot, Timestamp executeAt, PartialDeps partialDeps, int acceptFlags) { - return new Accept(txnId, scope, waitForEpoch, minEpoch, kind, ballot, executeAt, partialDeps, isPartialAccept); + return new Accept(txnId, scope, waitForEpoch, minEpoch, kind, ballot, executeAt, partialDeps, acceptFlags); } } public enum Kind { SLOW, MEDIUM } + public enum AcceptFlags + { + IS_PARTIAL, + + /** + * SyncPoints don't need to compute any deps in the Accept phase, + * as any dependencies they did not witness in the preaccept phase cannot reach consensus + */ + NO_CALCULATE_DEPS, + + /** + * See {@link accord.api.ProtocolModifiers.Toggles#filterDuplicateDependenciesFromAcceptReply()}. + */ + NO_FILTER_DEPS; + + public static int encode(TxnId txnId, boolean isPartial) + { + return (isPartial ? TinyEnumSet.encode(IS_PARTIAL) : 0) + | (txnId.isSyncPoint() && !syncPointsTrackUnstableMediumPathDependencies() ? TinyEnumSet.encode(NO_CALCULATE_DEPS) : 0) + | (filterDuplicateDependenciesFromAcceptReply() ? 0 : TinyEnumSet.encode(NO_FILTER_DEPS)); + } + + public static boolean isPartial(int encoded) + { + return TinyEnumSet.contains(encoded, IS_PARTIAL); + } + + public static boolean filterDeps(int encoded) + { + return !TinyEnumSet.contains(encoded, NO_FILTER_DEPS); + } + + public static boolean calculateDeps(int encoded) + { + return !TinyEnumSet.contains(encoded, NO_CALCULATE_DEPS); + } + } + public final Kind kind; public final Ballot ballot; public final Timestamp executeAt; private PartialDeps partialDeps; - public final boolean isPartialAccept; + public final int acceptFlags; public PartialDeps partialDeps() { return partialDeps; } - public Accept(Id to, Topologies topologies, Kind kind, Ballot ballot, TxnId txnId, FullRoute route, Timestamp executeAt, Deps deps, boolean isPartialAccept) + public Accept(Id to, Topologies topologies, Kind kind, Ballot ballot, TxnId txnId, FullRoute route, Timestamp executeAt, Deps deps, int acceptFlags) { super(to, topologies, txnId, route); this.kind = kind; this.ballot = ballot; this.executeAt = executeAt; this.partialDeps = deps.intersecting(scope); - this.isPartialAccept = isPartialAccept; + this.acceptFlags = acceptFlags; } - private Accept(TxnId txnId, Route scope, long waitForEpoch, long minEpoch, Kind kind, Ballot ballot, Timestamp executeAt, PartialDeps partialDeps, boolean isPartialAccept) + private Accept(TxnId txnId, Route scope, long waitForEpoch, long minEpoch, Kind kind, Ballot ballot, Timestamp executeAt, PartialDeps partialDeps, int acceptFlags) { super(txnId, scope, waitForEpoch, minEpoch); this.kind = kind; this.ballot = ballot; this.executeAt = executeAt; this.partialDeps = partialDeps; - this.isPartialAccept = isPartialAccept; + this.acceptFlags = acceptFlags; } @Override @@ -121,7 +161,7 @@ public AcceptReply applyInternal(SafeCommandStore safeStore) Participants hasDeps = null; Deps deps = null; - if (command.known().is(DepsKnown) && (isPartialAccept || notOwner)) + if (command.known().is(DepsKnown) && (isPartialAccept() || notOwner)) { deps = command.partialDeps().asFullUnsafe(); hasDeps = command.participants().stillTouches(); @@ -131,12 +171,12 @@ public AcceptReply applyInternal(SafeCommandStore safeStore) if (superseding.compareTo(ballot) <= 0) superseding = null; - boolean calculateDeps = isPartialAccept; + boolean calculateDeps = isPartialAccept() && calculateDeps(); if (command.saveStatus() == SaveStatus.Vestigial) { superseding = null; outcome = Success; - calculateDeps = true; + calculateDeps = calculateDeps(); } if (calculateDeps) @@ -156,9 +196,12 @@ public AcceptReply applyInternal(SafeCommandStore safeStore) hasDeps = participants.touches(); } - Participants successful = isPartialAccept ? hasDeps : null; + Participants successful = isPartialAccept() ? hasDeps : null; if (notOwner && (outcome == Redundant || (hasDeps != null && hasDeps.containsAll(participants.touches())))) outcome = Success; + + if (deps != null && filterDeps()) + deps = deps.without(partialDeps); return new AcceptReply(outcome, superseding, successful, deps, command.executeAtIfKnown()); } @@ -171,23 +214,46 @@ public AcceptReply applyInternal(SafeCommandStore safeStore) case Success: ExecuteFlags flags; Deps deps; - try (DepsCalculator calculator = new DepsCalculator()) + if (calculateDeps()) { - deps = calculator.calculate(safeStore, txnId, participants, minEpoch, executeAt, true); - if (deps == null) - return AcceptReply.inThePast(ballot, participants, safeCommand.current()); - flags = calculator.executeFlags(txnId); - } + try (DepsCalculator calculator = new DepsCalculator()) + { + deps = calculator.calculate(safeStore, txnId, participants, minEpoch, executeAt, true); + if (deps == null) + return AcceptReply.inThePast(ballot, participants, safeCommand.current()); + flags = calculator.executeFlags(txnId); + } - Invariants.require(deps.maxTxnId(txnId).epoch() <= executeAt.epoch()); - if (filterDuplicateDependenciesFromAcceptReply()) - deps = deps.without(partialDeps); + Invariants.require(deps.maxTxnId(txnId).epoch() <= executeAt.epoch()); + if (filterDeps()) + deps = deps.without(partialDeps); + } + else + { + flags = ExecuteFlags.none(); + deps = Deps.NONE; + } - Participants successful = isPartialAccept ? participants.touches() : null; + Participants successful = isPartialAccept() ? participants.touches() : null; return new AcceptReply(successful, deps, flags); } } + private boolean isPartialAccept() + { + return AcceptFlags.isPartial(acceptFlags); + } + + private boolean calculateDeps() + { + return AcceptFlags.calculateDeps(acceptFlags); + } + + private boolean filterDeps() + { + return AcceptFlags.filterDeps(acceptFlags); + } + @Override public AcceptReply reduce(AcceptReply r1, AcceptReply r2) { @@ -218,7 +284,7 @@ public LoadKeys loadKeys() @Override public LoadKeysFor loadKeysFor() { - return LoadKeysFor.READ_WRITE; + return calculateDeps() ? LoadKeysFor.READ_WRITE : LoadKeysFor.WRITE; } @Override @@ -374,6 +440,12 @@ public String toString() } } + @Override + public Timestamp executeAt() + { + return executeAt; + } + public static class NotAccept extends ParticipantsRequest, AcceptReply> { public final Status status; @@ -436,7 +508,7 @@ public String toString() @Override public String reason() { - return status + "{" + txnId + '}'; + return status.toString(); } } } diff --git a/accord-core/src/main/java/accord/messages/Await.java b/accord-core/src/main/java/accord/messages/Await.java index b05e15254a..84027f27a1 100644 --- a/accord-core/src/main/java/accord/messages/Await.java +++ b/accord-core/src/main/java/accord/messages/Await.java @@ -471,19 +471,6 @@ protected Cancellable submit() return node.commandStores().mapReduceConsume(txnId.epoch(), Long.MAX_VALUE, this); } - @Nullable - @Override - public TxnId primaryTxnId() - { - return txnId; - } - - @Override - public String reason() - { - return "AwaitComplete{" + newStatus + ',' + txnId + '}'; - } - @Override protected void acceptInternal(Reply reply, Throwable failure) { diff --git a/accord-core/src/main/java/accord/messages/BeginInvalidation.java b/accord-core/src/main/java/accord/messages/BeginInvalidation.java index e5e94bf2a7..0418ef3356 100644 --- a/accord-core/src/main/java/accord/messages/BeginInvalidation.java +++ b/accord-core/src/main/java/accord/messages/BeginInvalidation.java @@ -108,24 +108,12 @@ public InvalidateReply reduce(InvalidateReply o1, InvalidateReply o2) return new InvalidateReply(supersededBy, maxStatus.accepted, maxStatus.maxStatus, maxKnowledgeStatus.maxKnowledgeStatus, acceptedFastPath, truncated, route, homeKey); } - @Override - public long waitForEpoch() - { - return txnId.epoch(); - } - @Override public MessageType type() { return BEGIN_INVALIDATE_REQ; } - @Override - public String reason() - { - return "Invalidate{" + txnId + '}'; - } - @Override public String toString() { diff --git a/accord-core/src/main/java/accord/messages/BeginRecovery.java b/accord-core/src/main/java/accord/messages/BeginRecovery.java index b44c84968e..7e88c8cb48 100644 --- a/accord-core/src/main/java/accord/messages/BeginRecovery.java +++ b/accord-core/src/main/java/accord/messages/BeginRecovery.java @@ -48,18 +48,16 @@ import accord.utils.UnhandledEnum; import accord.utils.async.Cancellable; -import static accord.local.CommandSummaries.ComputeIsDep.EITHER; import static accord.local.CommandSummaries.SummaryStatus.APPLIED; import static accord.local.CommandSummaries.SummaryStatus.NOT_DIRECTLY_WITNESSED; import static accord.local.CommandSummaries.SummaryStatus.ACCEPTED; import static accord.local.CommandSummaries.SummaryStatus.STABLE; -import static accord.local.CommandSummaries.TestStartedAt.ANY; import static accord.messages.BeginRecovery.RecoverReply.Kind.Ok; import static accord.messages.BeginRecovery.RecoverReply.Kind.Reject; import static accord.messages.BeginRecovery.RecoverReply.Kind.Retired; import static accord.messages.BeginRecovery.RecoverReply.Kind.Truncated; -import static accord.messages.BeginRecovery.RecoveryFlags.FAST_PATH_DECIDED; -import static accord.messages.BeginRecovery.RecoveryFlags.FORCE_RECOVER_FAST_PATH; +import static accord.messages.BeginRecovery.RecoveryFlags.forceRecoverFastPath; +import static accord.messages.BeginRecovery.RecoveryFlags.isFastPathDecided; import static accord.messages.MessageType.StandardMessage.BEGIN_RECOVER_REQ; import static accord.messages.MessageType.StandardMessage.BEGIN_RECOVER_RSP; import static accord.primitives.Known.KnownDeps.DepsUnknown; @@ -82,7 +80,24 @@ public static BeginRecovery create(TxnId txnId, Route scope, long waitForEpoc public enum RecoveryFlags { - FAST_PATH_DECIDED, FORCE_RECOVER_FAST_PATH + FAST_PATH_DECIDED, + FORCE_RECOVER_FAST_PATH, + NO_CALCULATE_DEPS; + + public static boolean isFastPathDecided(int encoded) + { + return TinyEnumSet.contains(encoded, FAST_PATH_DECIDED); + } + + public static boolean forceRecoverFastPath(int encoded) + { + return TinyEnumSet.contains(encoded, FORCE_RECOVER_FAST_PATH); + } + + public static boolean calculateDeps(int encoded) + { + return !TinyEnumSet.contains(encoded, NO_CALCULATE_DEPS); + } } public final PartialTxn partialTxn; @@ -139,7 +154,7 @@ public RecoverReply applyInternal(SafeCommandStore safeStore) LatestDeps deps; { PartialDeps coordinatedDeps = command.partialDeps(); Deps localDeps = null; - if (!command.known().deps().hasCommittedOrDecidedDeps()) + if (!command.known().deps().hasCommittedOrDecidedDeps() && calculateDeps()) { localDeps = DepsCalculator.calculateDeps(safeStore, txnId, participants, minEpoch, txnId, false); } @@ -168,11 +183,11 @@ public RecoverReply applyInternal(SafeCommandStore safeStore) { try (Visitor visitor = new Visitor()) { - safeStore.visit(participants.owns(), txnId, txnId.witnessedBy(), ANY, txnId, EITHER, visitor); + safeStore.visit(participants.owns(), txnId, txnId.witnessedBy(), visitor); supersedingRejects = visitor.supersedingRejects; - earlierNoWait = visitor.earlierNoWait == null ? Deps.NONE : visitor.earlierNoWait.build(); - earlierWait = visitor.earlierWait == null ? Deps.NONE : visitor.earlierWait.build(); - laterCoordRejects = visitor.laterCoordRejects == null ? Deps.NONE : visitor.laterCoordRejects.build(); + earlierNoWait = visitor.simpleNoWait == null ? Deps.NONE : visitor.simpleNoWait.build(); + earlierWait = visitor.simpleWait == null ? Deps.NONE : visitor.simpleWait.build(); + laterCoordRejects = visitor.supersedingCoordRejects == null ? Deps.NONE : visitor.supersedingCoordRejects.build(); } } @@ -188,9 +203,14 @@ public RecoverReply applyInternal(SafeCommandStore safeStore) private boolean recoverFastPath() { - if (TinyEnumSet.contains(flags, FORCE_RECOVER_FAST_PATH)) + if (forceRecoverFastPath(flags)) return true; - return !txnId.isSyncPoint() && !TinyEnumSet.contains(flags, FAST_PATH_DECIDED); + return txnId.hasFastPath() && !isFastPathDecided(flags); + } + + private boolean calculateDeps() + { + return RecoveryFlags.calculateDeps(flags); } static boolean acceptsFastPath(TxnId txnId, StoreParticipants participants, SaveStatus saveStatus, @Nullable Timestamp executeAt) @@ -224,10 +244,10 @@ public RecoverReply reduce(RecoverReply r1, RecoverReply r2) if (!ok1.status.hasBeen(PreAccepted)) throw new IllegalStateException(); LatestDeps deps = LatestDeps.merge(ok1.deps, ok2.deps); - Deps earlierNoWait = ok1.earlierNoWait.with(ok2.earlierNoWait); - Deps earlierWait = ok1.earlierWait.with(ok2.earlierWait) - .without(earlierNoWait); - Deps laterNoVote = ok1.laterCoordRejects.with(ok2.laterCoordRejects); + Deps earlierNoWait = ok1.simpleNoWait.with(ok2.simpleNoWait); + Deps earlierWait = ok1.simpleWait.with(ok2.simpleWait) + .without(earlierNoWait); + Deps laterNoVote = ok1.supersedingCoordRejects.with(ok2.supersedingCoordRejects); Timestamp timestamp = ok1.status == PreAccepted ? Timestamp.max(ok1.executeAt, ok2.executeAt) : ok1.executeAt; return new RecoverOk( @@ -251,7 +271,9 @@ public LoadKeysFor loadKeysFor() { if (recoverFastPath()) return LoadKeysFor.RECOVERY; - return LoadKeysFor.READ_WRITE; + if (calculateDeps()) + return LoadKeysFor.READ_WRITE; + return LoadKeysFor.WRITE; } @Override @@ -260,12 +282,6 @@ public MessageType type() return BEGIN_RECOVER_REQ; } - @Override - public String reason() - { - return "Recover{" + txnId + '}'; - } - @Override public String toString() { @@ -276,10 +292,10 @@ public String toString() '}'; } - class Visitor implements CommandSummaries.AllCommandVisitor, AutoCloseable + class Visitor implements CommandSummaries.SupersedingCommandVisitor, AutoCloseable { - Deps.Builder earlierWait, earlierNoWait; - Deps.Builder laterCoordRejects; + Deps.Builder simpleWait, simpleNoWait; + Deps.Builder supersedingCoordRejects; boolean supersedingRejects; @Override @@ -315,7 +331,7 @@ public boolean visit(Unseekable keyOrRange, TxnId testTxnId, Timestamp testExecu if (status != ACCEPTED) break; case PREACCEPTED: - ensureEarlierWait().add(keyOrRange, testTxnId); + ensureSimpleWait().add(keyOrRange, testTxnId); case NOTACCEPTED: case INVALIDATED: } @@ -326,7 +342,7 @@ public boolean visit(Unseekable keyOrRange, TxnId testTxnId, Timestamp testExecu default: throw new UnhandledEnum(dep); case IS_PROPOSED_OR_STABLE_DEP: if (status == STABLE || status == APPLIED) - ensureEarlierNoWait().add(keyOrRange, testTxnId); + ensureSimpleNoWait().add(keyOrRange, testTxnId); break; case IS_NOT_PROPOSED_OR_STABLE_DEP: @@ -344,12 +360,12 @@ public boolean visit(Unseekable keyOrRange, TxnId testTxnId, Timestamp testExecu { case INVALIDATED: // TODO (desired): optionally exclude these and other normally-unnecessary entries on e.g. first recovery attempt - ensureEarlierNoWait().add(keyOrRange, testTxnId); + ensureSimpleNoWait().add(keyOrRange, testTxnId); break; case ACCEPTED: if (testExecuteAt.compareTo(txnId) > 0) - ensureEarlierWait().add(keyOrRange, testTxnId); + ensureSimpleWait().add(keyOrRange, testTxnId); break; case PREACCEPTED: @@ -359,7 +375,7 @@ public boolean visit(Unseekable keyOrRange, TxnId testTxnId, Timestamp testExecu // (that is, if either transaction use the optimisation, we must wait for the earlier transaction) // TODO (desired): compute against shard whether this is a necessary wait condition - for many quorum configurations it isn't if (testTxnId.hasPrivilegedCoordinator() || txnId.hasPrivilegedCoordinator()) - ensureEarlierWait().add(keyOrRange, testTxnId); + ensureSimpleWait().add(keyOrRange, testTxnId); } } } @@ -389,7 +405,12 @@ public boolean visit(Unseekable keyOrRange, TxnId testTxnId, Timestamp testExecu case IS_NOT_COORD_DEP: Invariants.requireArgument(testTxnId.is(PrivilegedCoordinatorWithDeps)); // TODO (expected): if we are the original coordinator and we know we cannot fast path commit then we should not include this in the reply - ensureLaterCoordRejects().add(keyOrRange, testTxnId); + ensureSupersedingCoordRejects().add(keyOrRange, testTxnId); + } + if (testTxnId.isSyncPoint()) + { + if (status.compareTo(ACCEPTED) < 0) ensureSimpleWait().add(keyOrRange, testTxnId); + else ensureSimpleNoWait().add(keyOrRange, testTxnId); } } @@ -402,44 +423,44 @@ private boolean markSupersedingRejects() return false; } - private Deps.Builder ensureEarlierNoWait() + private Deps.Builder ensureSimpleNoWait() { - if (earlierNoWait == null) - earlierNoWait = new Deps.Builder(true); - return earlierNoWait; + if (simpleNoWait == null) + simpleNoWait = new Deps.Builder(true); + return simpleNoWait; } - private Deps.Builder ensureEarlierWait() + private Deps.Builder ensureSimpleWait() { - if (earlierWait == null) - earlierWait = new Deps.Builder(true); - return earlierWait; + if (simpleWait == null) + simpleWait = new Deps.Builder(true); + return simpleWait; } - private Deps.Builder ensureLaterCoordRejects() + private Deps.Builder ensureSupersedingCoordRejects() { - if (laterCoordRejects == null) - laterCoordRejects = new Deps.Builder(true); - return laterCoordRejects; + if (supersedingCoordRejects == null) + supersedingCoordRejects = new Deps.Builder(true); + return supersedingCoordRejects; } @Override public void close() { - if (earlierNoWait != null) + if (simpleNoWait != null) { - earlierNoWait.close(); - earlierNoWait = null; + simpleNoWait.close(); + simpleNoWait = null; } - if (earlierWait != null) + if (simpleWait != null) { - earlierWait.close(); - earlierWait = null; + simpleWait.close(); + simpleWait = null; } - if (laterCoordRejects != null) + if (supersedingCoordRejects != null) { - laterCoordRejects.close(); - laterCoordRejects = null; + supersedingCoordRejects.close(); + supersedingCoordRejects = null; } } } @@ -465,8 +486,13 @@ public static class RecoverOk extends RecoverReply public final Ballot accepted; public final Timestamp executeAt; public final LatestDeps deps; - public final Deps earlierWait, earlierNoWait; - public final Deps laterCoordRejects; + // either preceding transactions or a potentially-superseding sync point (with no intervening decided sync point); + // these transactions cannot be blocked by our decision, so we can simply wait for their decision, + // and if they execute after us determine if they reject our execution + public final Deps simpleWait, simpleNoWait; + // superseding transactions where the coordinator had not witnessed us, and so they may reject our execution; + // these transactions may await our decision, so we must treat them differently to ensure there is no deadlock + public final Deps supersedingCoordRejects; public final boolean selfAcceptsFastPath; public final @Nullable Participants coordinatorAcceptsFastPath; public final boolean supersedingRejects; @@ -474,7 +500,7 @@ public static class RecoverOk extends RecoverReply public final Result result; public RecoverOk(TxnId txnId, Status status, Ballot accepted, Timestamp executeAt, LatestDeps deps, - Deps earlierWait, Deps earlierNoWait, Deps laterCoordRejects, + Deps simpleWait, Deps simpleNoWait, Deps supersedingCoordRejects, boolean selfAcceptsFastPath, Participants coordinatorAcceptsFastPath, boolean supersedingRejects, Writes writes, Result result) { this.txnId = txnId; @@ -482,9 +508,9 @@ public RecoverOk(TxnId txnId, Status status, Ballot accepted, Timestamp executeA this.executeAt = executeAt; this.status = status; this.deps = deps; - this.earlierWait = earlierWait; - this.earlierNoWait = earlierNoWait; - this.laterCoordRejects = laterCoordRejects; + this.simpleWait = simpleWait; + this.simpleNoWait = simpleNoWait; + this.supersedingCoordRejects = supersedingCoordRejects; this.selfAcceptsFastPath = selfAcceptsFastPath; this.coordinatorAcceptsFastPath = coordinatorAcceptsFastPath; this.supersedingRejects = supersedingRejects; @@ -512,9 +538,9 @@ String toString(String kind) ", accepted:" + accepted + ", executeAt:" + executeAt + ", deps:" + deps + - ", earlierWait:" + earlierWait + - ", earlierNoWait:" + earlierNoWait + - ", laterCoordRejects:" + laterCoordRejects + + ", simpleWait:" + simpleWait + + ", simpleNoWait:" + simpleNoWait + + ", laterCoordRejects:" + supersedingCoordRejects + ", selfAcceptsFastPath:" + selfAcceptsFastPath + (txnId.hasPrivilegedCoordinator() ? ", coordinatorFastPath:" + selfAcceptsFastPath : "") + ", supersedingRejects:" + supersedingRejects + diff --git a/accord-core/src/main/java/accord/messages/Commit.java b/accord-core/src/main/java/accord/messages/Commit.java index 1e07914d08..71f8faaeff 100644 --- a/accord-core/src/main/java/accord/messages/Commit.java +++ b/accord-core/src/main/java/accord/messages/Commit.java @@ -362,22 +362,10 @@ public Reply reduce(Reply o1, Reply o2) return null; } - @Override - public TxnId primaryTxnId() - { - return txnId; - } - @Override public String reason() { - return "Commit Invalidate"; - } - - @Override - public long waitForEpoch() - { - return waitForEpoch; + return "CommitInvalidate"; } @Override diff --git a/accord-core/src/main/java/accord/messages/GetMaxConflict.java b/accord-core/src/main/java/accord/messages/GetMaxConflict.java index ba2eff5e89..1649f33151 100644 --- a/accord-core/src/main/java/accord/messages/GetMaxConflict.java +++ b/accord-core/src/main/java/accord/messages/GetMaxConflict.java @@ -96,12 +96,6 @@ public TxnId primaryTxnId() return null; } - @Override - public String reason() - { - return toString(); - } - public static class GetMaxConflictOk implements Reply { public final Timestamp maxConflict; diff --git a/accord-core/src/main/java/accord/messages/NoWaitRequest.java b/accord-core/src/main/java/accord/messages/NoWaitRequest.java index ea6a252db1..ae388adf58 100644 --- a/accord-core/src/main/java/accord/messages/NoWaitRequest.java +++ b/accord-core/src/main/java/accord/messages/NoWaitRequest.java @@ -221,6 +221,6 @@ private static void cleanup(@Nullable Cancellable cancel) @Override public String reason() { - return getClass().getSimpleName() + (txnId == null ? "" : "{" + txnId + '}'); + return getClass().getSimpleName(); } } diff --git a/accord-core/src/main/java/accord/messages/ParticipantsRequest.java b/accord-core/src/main/java/accord/messages/ParticipantsRequest.java index 65b5b3b0a0..acd6d3d14c 100644 --- a/accord-core/src/main/java/accord/messages/ParticipantsRequest.java +++ b/accord-core/src/main/java/accord/messages/ParticipantsRequest.java @@ -64,7 +64,7 @@ public P scope() * safe to do so. */ @Override - public long waitForEpoch() + public final long waitForEpoch() { return waitForEpoch; } diff --git a/accord-core/src/main/java/accord/messages/ReadData.java b/accord-core/src/main/java/accord/messages/ReadData.java index 9b4d0072a0..96cb61e24e 100644 --- a/accord-core/src/main/java/accord/messages/ReadData.java +++ b/accord-core/src/main/java/accord/messages/ReadData.java @@ -201,6 +201,7 @@ private static boolean requiresListenersDuringExecution(TxnId txnId, ExecuteFlag return partialTxn; } + @Override public final @Nullable Timestamp executeAt() { return executeAt; diff --git a/accord-core/src/main/java/accord/messages/SetShardDurable.java b/accord-core/src/main/java/accord/messages/SetShardDurable.java index 3edc04b6d7..557fa76ef3 100644 --- a/accord-core/src/main/java/accord/messages/SetShardDurable.java +++ b/accord-core/src/main/java/accord/messages/SetShardDurable.java @@ -56,6 +56,7 @@ public Cancellable submit() { Invariants.require(durability.compareTo(Quorum) >= 0); TxnId syncIdWithFlags = syncIdWithFlags(); + // TODO (required): does this need to strictly precede updating RedundantBefore? Because updating the global map is more expensive. node.markDurable(syncPoint.route.toRanges(), syncIdWithFlags, durability.compareTo(Universal) >= 0 ? syncIdWithFlags : TxnId.NONE) .invoke((success, fail) -> { if (fail != null) node.reply(replyTo, replyContext, null, fail); diff --git a/accord-core/src/main/java/accord/primitives/AbstractRanges.java b/accord-core/src/main/java/accord/primitives/AbstractRanges.java index 7205a5f380..3f01fefa4a 100644 --- a/accord-core/src/main/java/accord/primitives/AbstractRanges.java +++ b/accord-core/src/main/java/accord/primitives/AbstractRanges.java @@ -733,6 +733,15 @@ public boolean equals(Object that) return Arrays.equals(this.ranges, ((AbstractRanges) that).ranges); } + public boolean hasSameRanges(Object that) + { + if (this == that) + return true; + if (!(that instanceof AbstractRanges)) + return false; + return Arrays.equals(this.ranges, ((AbstractRanges) that).ranges); + } + @Override public Iterator iterator() { @@ -899,7 +908,7 @@ static RS ofSortedAndDeoverlapped(Function 0) + if (ranges[i] == null || ranges[i - 1].end().compareTo(ranges[i].start()) > 0) throw illegalArgument(Arrays.toString(ranges) + " is not correctly sorted or deoverlapped"); } diff --git a/accord-core/src/main/java/accord/primitives/KeyDeps.java b/accord-core/src/main/java/accord/primitives/KeyDeps.java index fe19b5a7da..15fae72381 100644 --- a/accord-core/src/main/java/accord/primitives/KeyDeps.java +++ b/accord-core/src/main/java/accord/primitives/KeyDeps.java @@ -31,6 +31,7 @@ import accord.utils.SortedArrays.SortedArrayList; import accord.utils.SymmetricComparator; import accord.utils.TriFunction; +import accord.utils.UnhandledEnum; import java.util.*; import java.util.function.BiConsumer; @@ -42,6 +43,8 @@ import javax.annotation.Nullable; +import static accord.api.ProtocolModifiers.RangeSpec.isEndInclusive; +import static accord.api.ProtocolModifiers.RangeSpec.isStartExclusive; import static accord.primitives.RoutingKeys.toRoutingKeys; import static accord.primitives.Timestamp.Flag.UNSTABLE; import static accord.primitives.TxnId.NO_TXNIDS; @@ -562,11 +565,11 @@ public TxnId minTxnId(Range range, TxnId orElse) keysToTxnIds(); int startKeyIndex = keys.indexOf(range.start()); if (startKeyIndex < 0) startKeyIndex = -1 - startKeyIndex; - else if (!range.startInclusive()) ++startKeyIndex; + else if (isStartExclusive()) ++startKeyIndex; int endKeyIndex = keys.indexOf(range.end()); if (endKeyIndex < 0) endKeyIndex = -1 - startKeyIndex; - else if (range.endInclusive()) ++endKeyIndex; + else if (isEndInclusive()) ++endKeyIndex; if (endKeyIndex <= startKeyIndex) return orElse; @@ -592,11 +595,11 @@ public TxnId maxTxnId(Range range, TxnId orElse) keysToTxnIds(); int startKeyIndex = keys.indexOf(range.start()); if (startKeyIndex < 0) startKeyIndex = -1 - startKeyIndex; - else if (!range.startInclusive()) ++startKeyIndex; + else if (isStartExclusive()) ++startKeyIndex; int endKeyIndex = keys.indexOf(range.end()); if (endKeyIndex < 0) endKeyIndex = -1 - startKeyIndex; - else if (range.endInclusive()) ++endKeyIndex; + else if (isEndInclusive()) ++endKeyIndex; if (endKeyIndex <= startKeyIndex) return orElse; @@ -630,46 +633,75 @@ public void forEach(RoutingKey key, IndexedConsumer forEach) } } - public void forEach(Ranges ranges, int inclIdx, int exclIdx, P1 p1, P2 p2, IndexedBiConsumer forEach) + public void forEach(Unseekables unseekables, P1 p1, P2 p2, IndexedBiConsumer forEach) { - for (int i = 0; i < ranges.size(); ++i) - forEach(ranges.get(i), inclIdx, exclIdx, p1, p2, forEach); + Routable.Domain domain = unseekables.domain(); + switch (domain) + { + default: throw new UnhandledEnum(domain); + case Key: forEach((AbstractUnseekableKeys) unseekables, p1, p2, forEach); break; + case Range: forEach((AbstractRanges) unseekables, p1, p2, forEach); break; + } } - - public void forEach(Range range, P1 p1, P2 p2, IndexedBiConsumer forEach) + public void forEach(AbstractUnseekableKeys keys, P1 p1, P2 p2, IndexedBiConsumer forEach) { - forEach(range, 0, keys.size(), p1, p2, forEach); + if (keys.isEmpty()) + return; + + int[] keysToTxnIds = keysToTxnIds(); + int searchFromIndex = -1; + + for (int i = 0; i < keys.size() ; ++i) + { + int keyIndex = searchFromIndex < 0 ? this.keys.indexOf(keys.get(i)) + : this.keys.findNext(searchFromIndex, keys.get(i), FAST); + + if (keyIndex < 0) searchFromIndex = -1 - keyIndex; + else + { + int index = startOffset(keyIndex); + int end = endOffset(keyIndex); + while (index < end) + { + int txnIdx = keysToTxnIds[index++]; + forEach.accept(p1, p2, txnIdx); + } + searchFromIndex = keyIndex + 1; + } + } } - public void forEach(Range range, int inclKeyIdx, int exclKeyIdx, P1 p1, P2 p2, IndexedBiConsumer forEach) + public void forEach(AbstractRanges ranges, P1 p1, P2 p2, IndexedBiConsumer forEach) { - forEach(range, inclKeyIdx, exclKeyIdx, forEach, p1, p2, IndexedBiConsumer::accept); + for (int i = 0; i < ranges.size(); ++i) + forEach(ranges.get(i), p1, p2, forEach); } - public void forEach(Range range, P1 p1, P2 p2, P3 p3, IndexedTriConsumer forEach) + public int forEach(Range range, P1 p1, P2 p2, IndexedBiConsumer forEach) { - forEach(range, 0, keys.size(), p1, p2, p3, forEach); + return forEach(range, forEach, p1, p2, IndexedBiConsumer::accept); } - public void forEach(Range range, int inclKeyIdx, int exclKeyIdx, P1 p1, P2 p2, P3 p3, IndexedTriConsumer forEach) + public int forEach(Range range, P1 p1, P2 p2, P3 p3, IndexedTriConsumer forEach) { int[] keysToTxnIds = keysToTxnIds(); int start = keys.indexOf(range.start()); if (start < 0) start = -1 - start; - else if (!range.startInclusive()) ++start; + else if (isStartExclusive()) ++start; start = startOffset(start); int end = keys.indexOf(range.end()); if (end < 0) end = -1 - end; - else if (range.endInclusive()) ++end; + else if (isEndInclusive()) ++end; end = startOffset(end); while (start < end) { int txnIdx = keysToTxnIds[start++]; - if (txnIdx >= inclKeyIdx && txnIdx < exclKeyIdx) - forEach.accept(p1, p2, p3, txnIdx); + forEach.accept(p1, p2, p3, txnIdx); } + + return end; } public V foldEachKey(int txnIdx, P1 p1, V accumulate, TriFunction fold) @@ -741,11 +773,11 @@ public DepRelationList txnIdsWithFlags(Range range) { int startIndex = keys.indexOf(range.start()); if (startIndex < 0) startIndex = -1 - startIndex; - else if (!range.startInclusive()) ++startIndex; + else if (isStartExclusive()) ++startIndex; int endIndex = keys.indexOf(range.end()); if (endIndex < 0) endIndex = -1 - endIndex; - else if (range.endInclusive()) ++endIndex; + else if (isEndInclusive()) ++endIndex; if (startIndex == endIndex) return DepRelationList.EMPTY; diff --git a/accord-core/src/main/java/accord/primitives/KnownMap.java b/accord-core/src/main/java/accord/primitives/KnownMap.java index bc9fbf6df8..a11d2c6ae1 100644 --- a/accord-core/src/main/java/accord/primitives/KnownMap.java +++ b/accord-core/src/main/java/accord/primitives/KnownMap.java @@ -27,6 +27,8 @@ import accord.api.RoutingKey; import accord.utils.ReducingRangeMap; +import static accord.api.ProtocolModifiers.RangeSpec.isEndInclusive; +import static accord.api.ProtocolModifiers.RangeSpec.isStartInclusive; import static accord.primitives.Known.Definition.DefinitionErased; import static accord.primitives.Known.Definition.DefinitionKnown; import static accord.primitives.Known.KnownDeps.DepsErased; @@ -126,9 +128,9 @@ public String toString() public static class SerializerSupport { - public static KnownMap create(boolean inclusiveEnds, RoutingKey[] ends, MinAndMaxKnown[] values) + public static KnownMap create(RoutingKey[] ends, MinAndMaxKnown[] values) { - return new KnownMap(inclusiveEnds, ends, values); + return new KnownMap(ends, values); } } @@ -141,14 +143,14 @@ private KnownMap() this.validForAll = Known.Nothing; } - public KnownMap(boolean inclusiveEnds, RoutingKey[] starts, MinAndMaxKnown[] values) + public KnownMap(RoutingKey[] starts, MinAndMaxKnown[] values) { - this(inclusiveEnds, starts, values, Known.Nothing); + this(starts, values, Known.Nothing); } - private KnownMap(boolean inclusiveEnds, RoutingKey[] starts, MinAndMaxKnown[] values, Known validForAll) + private KnownMap(RoutingKey[] starts, MinAndMaxKnown[] values, Known validForAll) { - super(inclusiveEnds, starts, values); + super(starts, values); this.validForAll = validForAll; } @@ -199,7 +201,7 @@ public KnownMap with(Known validForAll) } if (i == size()) - return new KnownMap(inclusiveEnds(), starts, values, validForAll); + return new KnownMap(starts, values, validForAll); RoutingKey[] newStarts = new RoutingKey[size() + 1]; MinAndMaxKnown[] newValues = new MinAndMaxKnown[size()]; @@ -222,7 +224,7 @@ public KnownMap with(Known validForAll) newValues = Arrays.copyOf(newValues, count); newStarts = Arrays.copyOf(newStarts, count + 1); } - return new KnownMap(inclusiveEnds(), newStarts, newValues, validForAll); + return new KnownMap(newStarts, newValues, validForAll); } public boolean hasAnyFullyTruncated(Routables routables) @@ -303,7 +305,7 @@ public Participants knownFor(Known required, Participants expect) return prev.slice(-1 - i, prev.size()); if (prev.domain() == Routable.Domain.Key) - return prev.slice(i + (inclusiveEnds() ? 1 : 0), prev.size()); + return prev.slice(i + (isEndInclusive() ? 1 : 0), prev.size()); Range r = prev.get(i).asRange(); prev = prev.slice(i, prev.size()); @@ -317,7 +319,7 @@ public Participants knownFor(Known required, Participants expect) return prev.slice(0, -1 - i); if (prev.domain() == Routable.Domain.Key) - return prev.slice(0, i + (inclusiveStarts() ? 0 : 1)); + return prev.slice(0, i + (isStartInclusive() ? 0 : 1)); Range r = prev.get(i).asRange(); prev = prev.slice(0, i); @@ -331,15 +333,15 @@ public Participants knownFor(Known required, Participants expect) public static class Builder extends AbstractBoundariesBuilder { - public Builder(boolean inclusiveEnds, int capacity) + public Builder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override protected KnownMap buildInternal() { - return new KnownMap(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new MinAndMaxKnown[0])); + return new KnownMap(starts.toArray(new RoutingKey[0]), values.toArray(new MinAndMaxKnown[0])); } } } diff --git a/accord-core/src/main/java/accord/primitives/LatestDeps.java b/accord-core/src/main/java/accord/primitives/LatestDeps.java index 0d645651f0..ba67c5f539 100644 --- a/accord-core/src/main/java/accord/primitives/LatestDeps.java +++ b/accord-core/src/main/java/accord/primitives/LatestDeps.java @@ -60,9 +60,9 @@ public class LatestDeps extends ReducingRangeMap public static class SerializerSupport { - public static LatestDeps create(boolean inclusiveEnds, RoutingKey[] starts, LatestEntry[] values) + public static LatestDeps create(RoutingKey[] starts, LatestEntry[] values) { - return new LatestDeps(inclusiveEnds, starts, values); + return new LatestDeps(starts, values); } } @@ -287,12 +287,12 @@ public boolean equals(Object obj) private LatestDeps() { - super(false, RoutingKeys.EMPTY_KEYS_ARRAY, new LatestEntry[0]); + super(RoutingKeys.EMPTY_KEYS_ARRAY, new LatestEntry[0]); } - private LatestDeps(boolean inclusiveEnds, RoutingKey[] starts, LatestEntry[] values) + private LatestDeps(RoutingKey[] starts, LatestEntry[] values) { - super(inclusiveEnds, starts, values); + super(starts, values); } public Deps merge() @@ -310,7 +310,7 @@ public static LatestDeps create(Participants participants, KnownDeps knownDep if (participants.isEmpty()) return new LatestDeps(); - Builder builder = new Builder(participants.get(0).asRange().endInclusive(), participants.size() * 2); + Builder builder = new Builder(participants.size() * 2); for (int i = 0 ; i < participants.size() ; ++i) { Range cur = participants.get(i).asRange(); @@ -330,9 +330,9 @@ private static Deps slice(Ranges ranges, @Nullable Deps deps) static class Builder extends AbstractIntervalBuilder { - protected Builder(boolean inclusiveEnds, int capacity) + protected Builder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override @@ -350,7 +350,7 @@ protected LatestEntry reduce(LatestEntry a, LatestEntry b) @Override protected LatestDeps buildInternal() { - return new LatestDeps(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new LatestEntry[0])); + return new LatestDeps(starts.toArray(new RoutingKey[0]), values.toArray(new LatestEntry[0])); } } @@ -409,12 +409,12 @@ private Merge() private Merge(LatestDeps convert) { - super(convert.inclusiveEnds(), convert.starts, convert(convert.values)); + super(convert.starts, convert(convert.values)); } - private Merge(boolean inclusiveEnds, RoutingKey[] starts, MergeEntry[] values) + private Merge(RoutingKey[] starts, MergeEntry[] values) { - super(inclusiveEnds, starts, values); + super(starts, values); } static Merge merge(Merge a, Merge b) @@ -588,9 +588,9 @@ private static TriFunction, static class MergeBuilder extends AbstractIntervalBuilder { - protected MergeBuilder(boolean inclusiveEnds, int capacity) + protected MergeBuilder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override @@ -629,7 +629,7 @@ protected MergeEntry tryMergeEqual(MergeEntry a, MergeEntry b) @Override protected Merge buildInternal() { - return new Merge(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new MergeEntry[0])); + return new Merge(starts.toArray(new RoutingKey[0]), values.toArray(new MergeEntry[0])); } } } diff --git a/accord-core/src/main/java/accord/primitives/Range.java b/accord-core/src/main/java/accord/primitives/Range.java index 661829252d..5a5718903f 100644 --- a/accord-core/src/main/java/accord/primitives/Range.java +++ b/accord-core/src/main/java/accord/primitives/Range.java @@ -28,6 +28,7 @@ import javax.annotation.Nullable; +import static accord.api.ProtocolModifiers.RangeSpec.END_INCLUSIVE; import static accord.utils.SortedArrays.Search.CEIL; import static accord.utils.SortedArrays.Search.FAST; @@ -41,11 +42,13 @@ public static class EndInclusive extends Range public EndInclusive(RoutingKey start, RoutingKey end) { super(start, end); + Invariants.require(END_INCLUSIVE); } public EndInclusive(RoutingKey start, RoutingKey end, AntiRangeMarker antiRange) { super(start, end, antiRange); + Invariants.require(END_INCLUSIVE); } @Override @@ -96,11 +99,13 @@ public static class StartInclusive extends Range public StartInclusive(RoutingKey start, RoutingKey end) { super(start, end); + Invariants.require(!END_INCLUSIVE); } public StartInclusive(RoutingKey start, RoutingKey end, AntiRangeMarker antiRange) { super(start, end, antiRange); + Invariants.require(!END_INCLUSIVE); } @Override @@ -148,68 +153,8 @@ public Range newRange(RoutingKey start, RoutingKey end) public static Range range(RoutingKey start, RoutingKey end, boolean startInclusive, boolean endInclusive) { - return new Range(start, end) { - - @Override - public boolean startInclusive() - { - return startInclusive; - } - - @Override - public boolean endInclusive() - { - return endInclusive; - } - - @Override - public Range newRange(RoutingKey start, RoutingKey end) - { - throw new UnsupportedOperationException("subRange"); - } - - @Override - public int compareTo(RoutableKey key) - { - if (startInclusive) - { - if (key.compareTo(start()) < 0) - return 1; - } - else - { - if (key.compareTo(start()) <= 0) - return 1; - } - if (endInclusive) - { - if (key.compareTo(end()) > 0) - return -1; - } - else - { - if (key.compareTo(end()) >= 0) - return -1; - } - return 0; - } - - @Override - public int compareStartTo(RoutableKey key) - { - int c = start().compareTo(key); - if (c == 0 && !startInclusive) c = 1; - return c; - } - - @Override - public int compareEndTo(RoutableKey key) - { - int c = end().compareTo(key); - if (c == 0 && !endInclusive) c = -1; - return c; - } - }; + Invariants.require(startInclusive != endInclusive); + return startInclusive ? new StartInclusive(start, end) : new EndInclusive(start, end); } // used to construct an unsafe Range used only for representing an absence of information. Imposes weaker invariants. @@ -223,16 +168,13 @@ private Range(RoutingKey start, RoutingKey end) // TODO (expected): should we at least relax to permit an empty Range? Invariants.requireArgument(start.compareTo(end) < 0, "%s >= %s", start, end); Invariants.requireArgument(Objects.equals(start.prefix(), end.prefix()), "Range bounds must share their prefix: %s vs %s", start, end); - Invariants.require(startInclusive() != endInclusive(), "Range must have one side inclusive, and the other exclusive. Range of different types should not be mixed."); this.start = start; this.end = end; } private Range(RoutingKey start, RoutingKey end, AntiRangeMarker antiRange) { - // TODO (expected): should we at least relax to permit an empty Range? Invariants.requireArgument(start.compareTo(end) < 0, "%s >= %s", start, end); - Invariants.require(startInclusive() != endInclusive(), "Range must have one side inclusive, and the other exclusive. Range of different types should not be mixed."); this.start = start; this.end = end; } diff --git a/accord-core/src/main/java/accord/primitives/SaveStatus.java b/accord-core/src/main/java/accord/primitives/SaveStatus.java index efa8bac87f..38b8911581 100644 --- a/accord-core/src/main/java/accord/primitives/SaveStatus.java +++ b/accord-core/src/main/java/accord/primitives/SaveStatus.java @@ -30,8 +30,10 @@ import accord.primitives.Known.KnownRoute; import accord.primitives.Known.Outcome; import accord.primitives.Status.Phase; +import accord.utils.BitUtils; import static accord.local.CommandSummaries.SummaryStatus.ACCEPTED; +import static accord.local.CommandSummaries.SummaryStatus.APPLIED; import static accord.primitives.Known.PrivilegedVote.NoVote; import static accord.primitives.Known.PrivilegedVote.VotePreAccept; import static accord.primitives.Known.KnownDeps.DepsFromCoordinator; @@ -107,8 +109,8 @@ public enum SaveStatus Applied (Status.Applied, LocalExecution.Applied), // TruncatedApplyWithOutcomeAndDeps exists to support re-populating CommandsForKey on replay with any dependencies needed for computing recovery superseding-rejects decisions // TODO (expected): test replay - TruncatedApplyWithOutcome (Status.Truncated, FullRoute, DefinitionErased, ApplyAtKnown, DepsErased, Apply, CleaningUp), - TruncatedApply (Status.Truncated, FullRoute, DefinitionErased, ApplyAtKnown, DepsErased, WasApply,CleaningUp), + TruncatedApplyWithOutcome (Status.Truncated, APPLIED, FullRoute, DefinitionErased, ApplyAtKnown, DepsErased, Apply, CleaningUp), + TruncatedApply (Status.Truncated, APPLIED, FullRoute, DefinitionErased, ApplyAtKnown, DepsErased, WasApply,CleaningUp), TruncatedUnapplied (Status.Truncated, MaybeRoute, DefinitionErased, ExecuteAtKnown, DepsErased, WasApply,CleaningUp), // Vestigial means the command cannot be completed and is either pre-bootstrap, did not commit, or did not participate in this shard's epoch // TODO (expected): should Vestigial NOT be a Truncated status? We should really only use Known or KnownMap to make decisions, so we don't interpret Truncated as implying a decision, e.g. in Invalidate (as of this commit) @@ -167,6 +169,8 @@ public enum LocalExecution } private static final SaveStatus[] lookup = values(); + public static final int ENCODING_BITS = BitUtils.numberOfBitsToRepresent(lookup.length); + public static final int ENCODING_MASK = (1 << ENCODING_BITS) - 1; public final Status status; public final Phase phase; diff --git a/accord-core/src/main/java/accord/primitives/Status.java b/accord-core/src/main/java/accord/primitives/Status.java index e7d7d9ea2d..4e4d1ad68c 100644 --- a/accord-core/src/main/java/accord/primitives/Status.java +++ b/accord-core/src/main/java/accord/primitives/Status.java @@ -137,7 +137,8 @@ public static final class Durability private static final int SHARDS_SHIFT = 3; private static final int PHASE_SHIFT = 1; private static final int PHASE_MASK = 0x3; - public static final int TOTAL_ENCODING_BITS = 6; + public static final int ENCODING_BITS = 6; + public static final int ENCODING_MASK = (1 << ENCODING_BITS) - 1; private static final Durability[] lookup = values(); public static final Durability NotDurable = get(HasDecision.None, HasOutcome.None, HasOutcome.None, false); diff --git a/accord-core/src/main/java/accord/primitives/Timestamp.java b/accord-core/src/main/java/accord/primitives/Timestamp.java index 0164c56755..841c4f3890 100644 --- a/accord-core/src/main/java/accord/primitives/Timestamp.java +++ b/accord-core/src/main/java/accord/primitives/Timestamp.java @@ -18,6 +18,9 @@ package accord.primitives; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + import accord.local.Node.Id; import accord.utils.Invariants; import javax.annotation.Nonnull; @@ -26,6 +29,7 @@ import static accord.primitives.Timestamp.Flag.REJECTED; import static accord.primitives.Timestamp.Flag.SHARD_BOUND; import static accord.primitives.Timestamp.Flag.UNSTABLE; +import static accord.utils.Invariants.illegalArgument; /** * TxnId flag bits: @@ -92,8 +96,6 @@ public enum Flag /** * The set of flags we want to retain as we merge timestamps (e.g. when taking mergeMax). - * Today this is only the REJECTED_FLAG, but we may include additional flags in future (such as Committed, Applied..) - * which we may also want to retain when merging in other contexts (such as in Deps). */ static final int MERGE_FLAGS = REJECTED.bit | UNSTABLE.bit | HLC_BOUND.bit | SHARD_BOUND.bit; public static final long IDENTITY_LSB = 0xFFFFFFFF_FFFF00FFL; @@ -102,7 +104,7 @@ public enum Flag public static final int KIND_AND_DOMAIN_FLAGS = 0x00000000_0000000F; public static final long MAX_EPOCH = (1L << 48) - 1; private static final long HLC_INCR = 1L << 16; - static final long MAX_FLAGS = HLC_INCR - 1; + static final int MAX_FLAGS = (int) (HLC_INCR - 1); public static Timestamp fromBits(long msb, long lsb, Id node) { @@ -121,12 +123,22 @@ public static Timestamp fromValues(long epoch, long hlc, int flags, Id node) public static Timestamp maxForEpoch(long epoch) { - return new Timestamp(epochMsb(epoch) | 0x7fff, Long.MAX_VALUE, Id.MAX); + return maxForEpoch(epoch, Timestamp::fromBits); + } + + public static T maxForEpoch(long epoch, RawFactory factory) + { + return factory.create(epochMsb(epoch) | 0x7fff, Long.MAX_VALUE, Id.MAX); } public static Timestamp minForEpoch(long epoch) { - return new Timestamp(epochMsb(epoch), 0, Id.NONE); + return minForEpoch(epoch, Timestamp::fromBits); + } + + public static T minForEpoch(long epoch, RawFactory factory) + { + return factory.create(epochMsb(epoch), 0, Id.NONE); } public final long msb; @@ -295,13 +307,29 @@ public Timestamp logicalNext(Id node) return new Timestamp(msb, lsb, node); } + public Timestamp logicalPrev(Id node) + { + long lsb = this.lsb - HLC_INCR; + long msb = this.msb; + if (lowHlc(this.lsb) == 0) + --msb; // overflow of lsb + return new Timestamp(msb, lsb, node); + } + public Timestamp next() { - if (node.id < Long.MAX_VALUE) + if (node.id < Integer.MAX_VALUE) return new Timestamp(msb, lsb, new Id(node.id + 1)); return logicalNext(Id.NONE); } + public Timestamp prev() + { + if (node.id > 0) + return new Timestamp(msb, lsb, new Id(node.id - 1)); + return logicalPrev(Id.MAX); + } + @Override public int compareTo(@Nonnull Timestamp that) { @@ -429,6 +457,17 @@ public static Timestamp mergeMax(Timestamp a, Timestamp b) : b.withEpochAtLeast(a.epoch()); } + /** + * The resulting timestamp will have max(a.hlc,b.hlc) and max(a.epoch, b.epoch) + */ + public static T mergeMax(T a, T b, ValueFactory factory) + { + // Note: it is not safe to take the highest HLC while retaining the current node; + // however, it is safe to take the highest epoch, as the originating node will always advance the hlc() + return a.compareToWithoutEpoch(b) >= 0 ? withEpochAtLeast(b.epoch(), a, factory) + : withEpochAtLeast(a.epoch(), b, factory); + } + /** * The resulting timestamp will have min(a.hlc,b.hlc) and min(a.epoch, b.epoch) */ @@ -526,18 +565,53 @@ public String toStandardString() return "[" + epoch() + ',' + hlc + (hlc == uniqueHlc ? "" : "+" + (uniqueHlc - hlc)) + ',' + flags() + ',' + node + ']'; } - public static Timestamp fromString(String string) + private static final Pattern PARSE = Pattern.compile("\\[(?[0-9]+),(?[0-9]+)(\\+(?[0-9]+>))?,(?[0-9]+)(?\\([KR][REWXV]\\))?,(?[0-9]+)]"); + public static Timestamp parse(String timestampString) + { + Matcher m = PARSE.matcher(timestampString); + if (!m.matches()) + throw illegalArgument("Invalid Timestamp string: " + timestampString); + return fromMatcher(timestampString, m, true); + } + + public static Timestamp tryParse(String timestampString) + { + Matcher m = PARSE.matcher(timestampString); + if (!m.matches()) + return null; + return fromMatcher(timestampString, m, false); + } + + private static Timestamp fromMatcher(String timestampString, Matcher m, boolean failIfInvalid) + { + String uniqueHlc = m.group("uniqueHlc"); + String kind = m.group("kind"); + if (uniqueHlc != null && kind != null) + return failOrNull(timestampString, failIfInvalid); + + long epoch = Long.parseLong(m.group("epoch")); + long hlc = Long.parseLong(m.group("hlc")); + int flags = Integer.parseInt(m.group("flags")); + if (flags > 0xffff) + return failOrNull(timestampString, failIfInvalid); + + Id node = new Id(Integer.parseInt(m.group("node"))); + if (uniqueHlc != null) + return new TimestampWithUniqueHlc(epoch, hlc, Long.parseLong(uniqueHlc), flags, node); + + if (kind != null) + { + // TODO (expected): validate kind vs flags + return TxnId.fromValues(epoch, hlc, flags, node); + } + + return fromValues(epoch, hlc, flags, node); + } + + private static T failOrNull(String timestampString, boolean fail) { - String[] split = string.replaceFirst("\\[", "").replaceFirst("\\]", "").split(","); - assert split.length == 4; - int indexOfUniqueHlc = split[2].indexOf('+'); - int flags = Integer.parseInt(indexOfUniqueHlc < 0 ? split[2] : split[2].substring(0, indexOfUniqueHlc)); - Timestamp result = Timestamp.fromValues(Long.parseLong(split[0]), - Long.parseLong(split[1]), - flags, - new Id(Integer.parseInt(split[3]))); - if (indexOfUniqueHlc < 0) - return result; - return new TimestampWithUniqueHlc(result, Integer.parseInt(split[2].substring(indexOfUniqueHlc + 1))); + if (fail) + throw illegalArgument("Invalid Timestamp string: " + timestampString); + return null; } } diff --git a/accord-core/src/main/java/accord/primitives/TxnId.java b/accord-core/src/main/java/accord/primitives/TxnId.java index e945f28893..994f9491c8 100644 --- a/accord-core/src/main/java/accord/primitives/TxnId.java +++ b/accord-core/src/main/java/accord/primitives/TxnId.java @@ -23,6 +23,7 @@ import javax.annotation.Nullable; +import accord.local.Node; import accord.local.Node.Id; import accord.primitives.Routable.Domain; import accord.primitives.Txn.Kind; @@ -319,6 +320,11 @@ public final boolean isVisible() return Kind.isVisible(kindOrdinal(flagsUnmasked())); } + public final boolean hasFastPath() + { + return !isSyncPoint(); + } + public final boolean isSyncPoint() { return Kind.isSyncPoint(kindOrdinal(flagsUnmasked())); @@ -497,12 +503,12 @@ public static int domainOrdinal(int flags) public static TxnId maxForEpoch(long epoch) { - return new TxnId(epochMsb(epoch) | 0x7fff, Long.MAX_VALUE, Id.MAX); + return maxForEpoch(epoch, TxnId::fromBits); } public static TxnId minForEpoch(long epoch) { - return new TxnId(epochMsb(epoch), 0, Id.NONE); + return minForEpoch(epoch, TxnId::fromBits); } public static TxnId noneIfNull(TxnId id) @@ -515,13 +521,20 @@ public static TxnId maxIfNull(TxnId id) return id == null ? MAX : id; } + public static TxnId atLeast(Timestamp timestamp) + { + if (timestamp.flags() != 0) + timestamp = timestamp.next(); + return new TxnId(timestamp, 0, Read, Domain.Key, Any); + } + private static final Pattern PARSE = Pattern.compile("\\[(?[0-9]+),(?[0-9]+),(?[0-9]+)\\([KR][REWXV]\\),(?[0-9]+)]"); public static TxnId parse(String txnIdString) { Matcher m = PARSE.matcher(txnIdString); if (!m.matches()) throw illegalArgument("Invalid TxnId string: " + txnIdString); - return fromValues(Long.parseLong(m.group("epoch")), Long.parseLong(m.group("hlc")), Integer.parseInt(m.group("flags")), new Id(Integer.parseInt(m.group("node")))); + return fromMatcher(txnIdString, m, true); } public static TxnId tryParse(String txnIdString) @@ -529,7 +542,27 @@ public static TxnId tryParse(String txnIdString) Matcher m = PARSE.matcher(txnIdString); if (!m.matches()) return null; - return fromValues(Long.parseLong(m.group("epoch")), Long.parseLong(m.group("hlc")), Integer.parseInt(m.group("flags")), new Id(Integer.parseInt(m.group("node")))); + return fromMatcher(txnIdString, m, false); + } + + private static TxnId fromMatcher(String txnIdString, Matcher m, boolean failIfInvalid) + { + long epoch = Long.parseLong(m.group("epoch")); + long hlc = Long.parseLong(m.group("hlc")); + int flags = Integer.parseInt(m.group("flags")); + if (flags > 0xffff) + return failOrNull(txnIdString, failIfInvalid); + + Id node = new Id(Integer.parseInt(m.group("node"))); + // TODO (expected): validate kind vs flags + return TxnId.fromValues(epoch, hlc, flags, node); + } + + private static T failOrNull(String timestampString, boolean fail) + { + if (fail) + throw illegalArgument("Invalid TxnId string: " + timestampString); + return null; } public static boolean equalsStrict(TxnId[] a, TxnId[] b) diff --git a/accord-core/src/main/java/accord/topology/Topology.java b/accord-core/src/main/java/accord/topology/Topology.java index cba5a4c20a..f57f339e76 100644 --- a/accord-core/src/main/java/accord/topology/Topology.java +++ b/accord-core/src/main/java/accord/topology/Topology.java @@ -55,6 +55,7 @@ import org.agrona.collections.Int2ObjectHashMap; import org.agrona.collections.IntArrayList; +import static accord.api.ProtocolModifiers.RangeSpec.isEndInclusive; import static accord.utils.Invariants.illegalArgument; import static accord.utils.SortedArrays.Search.FAST; import static accord.utils.SortedArrays.Search.FLOOR; @@ -421,7 +422,7 @@ public Topology select(SortedArrayList nodes) for (int i = shards.firstSetBit() ; i >= 0 ; i = shards.nextSetBit(i + 1, -1)) { supersetIndexes[count] = this.supersetIndexes[i]; - ranges[count] = this.shards[this.supersetIndexes[i]].range; + ranges[count++] = this.shards[this.supersetIndexes[i]].range; } subsetOfRanges = Ranges.ofSortedAndDeoverlapped(ranges); } @@ -600,7 +601,7 @@ public T foldlWithDefault(Routables select, IndexedTriFunction, V> { - protected final boolean inclusiveEnds; protected final Object[] tree; public BTreeReducingIntervalMap() { - this(false); + this(BTree.empty()); } - public BTreeReducingIntervalMap(boolean inclusiveEnds) + protected BTreeReducingIntervalMap(Object[] tree) { - this(inclusiveEnds, BTree.empty()); - } - - protected BTreeReducingIntervalMap(boolean inclusiveEnds, Object[] tree) - { - this.inclusiveEnds = inclusiveEnds; this.tree = tree; } - public boolean inclusiveEnds() - { - return inclusiveEnds; - } - - public final boolean inclusiveStarts() - { - return !inclusiveEnds; - } - public boolean isEmpty() { return BTree.isEmpty(tree); @@ -94,7 +79,7 @@ public V get(K key) int idx = BTree.findIndex(tree, EntryComparator.instance(), key); if (idx < 0) idx = -2 - idx; - else if (inclusiveEnds) --idx; + else if (isEndInclusive()) --idx; return idx < 0 || idx >= size() ? null @@ -164,11 +149,11 @@ public String toString(Predicate include) if (!isFirst) builder.append(", "); - builder.append(inclusiveStarts() ? '[' : '(') + builder.append(isStartInclusive() ? '[' : '(') .append(iter.start()) .append(',') .append(iter.end()) - .append(inclusiveEnds() ? ']' : ')') + .append(isEndInclusive() ? ']' : ')') .append('=') .append(iter.value()); @@ -184,13 +169,13 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) return false; @SuppressWarnings("unchecked") BTreeReducingIntervalMap that = (BTreeReducingIntervalMap) o; - return this.inclusiveEnds == that.inclusiveEnds && BTree.equals(this.tree, that.tree); + return BTree.equals(this.tree, that.tree); } @Override public int hashCode() { - return Boolean.hashCode(inclusiveEnds) + 31 * BTree.hashCode(tree); + return 31 * BTree.hashCode(tree); } public static class EntryComparator, V> implements AsymmetricComparator> @@ -281,8 +266,7 @@ protected static , V, M extends BTreeReducingInt if (historyRight == null || historyRight.isEmpty()) return historyLeft; - boolean inclusiveEnds = inclusiveEnds(historyLeft.inclusiveEnds, !historyLeft.isEmpty(), historyRight.inclusiveEnds, !historyRight.isEmpty()); - AbstractIntervalBuilder builder = factory.create(inclusiveEnds, historyLeft.size() + historyRight.size()); + AbstractIntervalBuilder builder = factory.create(historyLeft.size() + historyRight.size()); WithBoundsIterator left = historyLeft.withBoundsIterator(); WithBoundsIterator right = historyRight.withBoundsIterator(); @@ -359,8 +343,7 @@ protected static , V, M extends BTreeReducingInt if (historyRight == null || historyRight.isEmpty()) return historyLeft; - boolean inclusiveEnds = inclusiveEnds(historyLeft.inclusiveEnds, !historyLeft.isEmpty(), historyRight.inclusiveEnds, !historyRight.isEmpty()); - AbstractBoundariesBuilder builder = factory.create(inclusiveEnds, historyLeft.size() + historyRight.size()); + AbstractBoundariesBuilder builder = factory.create(historyLeft.size() + historyRight.size()); WithBoundsIterator left = historyLeft.withBoundsIterator(); WithBoundsIterator right = historyRight.withBoundsIterator(); @@ -463,19 +446,16 @@ private static , V> V sliceAndReduce(K start, K public interface BoundariesBuilderFactory, V, M extends BTreeReducingIntervalMap> { - AbstractBoundariesBuilder create(boolean inclusiveEnds, int capacity); + AbstractBoundariesBuilder create(int capacity); } public static abstract class AbstractBoundariesBuilder, V, M extends BTreeReducingIntervalMap> { - protected final boolean inclusiveEnds; - private final BTree.Builder> treeBuilder; private final TinyKVBuffer buffer; - protected AbstractBoundariesBuilder(boolean inclusiveEnds, int capacity) + protected AbstractBoundariesBuilder(int capacity) { - this.inclusiveEnds = inclusiveEnds; this.treeBuilder = BTree.builder(Comparator.naturalOrder(), capacity); this.buffer = new TinyKVBuffer<>(); } @@ -550,20 +530,18 @@ public final M build() public interface IntervalBuilderFactory, V, M extends BTreeReducingIntervalMap> { - AbstractIntervalBuilder create(boolean inclusiveEnds, int capacity); + AbstractIntervalBuilder create(int capacity); } public static abstract class AbstractIntervalBuilder, V, M> { - protected final boolean inclusiveEnds; private final BTree.Builder> treeBuilder; private final TinyKVBuffer buffer; private K prevEnd; - protected AbstractIntervalBuilder(boolean inclusiveEnds, int capacity) + protected AbstractIntervalBuilder(int capacity) { - this.inclusiveEnds = inclusiveEnds; this.treeBuilder = BTree.builder(Comparator.naturalOrder(), capacity); this.buffer = new TinyKVBuffer<>(); } diff --git a/accord-core/src/main/java/accord/utils/BTreeReducingRangeMap.java b/accord-core/src/main/java/accord/utils/BTreeReducingRangeMap.java index 159124fb8b..e0e6f41cb3 100644 --- a/accord-core/src/main/java/accord/utils/BTreeReducingRangeMap.java +++ b/accord-core/src/main/java/accord/utils/BTreeReducingRangeMap.java @@ -28,8 +28,10 @@ import java.util.Iterator; import java.util.List; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.Predicate; +import static accord.api.ProtocolModifiers.RangeSpec.isEndInclusive; import static accord.utils.SortedArrays.Search.FAST; public class BTreeReducingRangeMap extends BTreeReducingIntervalMap @@ -39,9 +41,9 @@ public BTreeReducingRangeMap() super(); } - protected BTreeReducingRangeMap(boolean inclusiveEnds, Object[] tree) + protected BTreeReducingRangeMap(Object[] tree) { - super(inclusiveEnds, tree); + super(tree); } public V foldl(Routables routables, BiFunction fold, V accumulator) @@ -74,20 +76,20 @@ public V2 foldl(AbstractKeys keys, QuadFunction= treeSize - 1) return accumulator; int nexti = keys.findNext(i, startAt(idx + 1), FAST); if (nexti < 0) nexti = -1 -nexti; - else if (inclusiveEnds) ++nexti; + else if (isEndInclusive()) ++nexti; Entry entry = entryAt(idx); if (i != nexti && entry.hasValue() && entry.value() != null) @@ -112,7 +114,7 @@ public V2 foldl(AbstractRanges ranges, QuadFunction V2 foldl(AbstractRanges ranges, QuadFunction> M create(AbstractRanges ra { Invariants.requireArgument(value != null, "value is null"); - AbstractBoundariesBuilder builder = factory.create(ranges.get(0).endInclusive(), ranges.size() * 2); + AbstractBoundariesBuilder builder = factory.create(ranges.size() * 2); for (Range cur : ranges) { builder.append(cur.start(), value, (a, b) -> { throw new IllegalStateException(); }); @@ -209,7 +211,7 @@ public static > M create(AbstractUnseekabl { Invariants.requireArgument(value != null, "value is null"); - AbstractBoundariesBuilder builder = factory.create(keys.get(0).asRange().endInclusive(), keys.size() * 2); + AbstractBoundariesBuilder builder = factory.create(keys.size() * 2); for (int i = 0 ; i < keys.size() ; ++i) { Range range = keys.get(i).asRange(); @@ -228,7 +230,7 @@ public static > M create(Keys keys, V valu AbstractBoundariesBuilder builder; { Range range = prev.asRange(); - builder = factory.create(prev.asRange().endInclusive(), keys.size() * 2); + builder = factory.create(keys.size() * 2); builder.append(range.start(), value, (a, b) -> { throw new IllegalStateException(); }); builder.append(range.end(), null, (a, b) -> { throw new IllegalStateException(); }); } @@ -268,7 +270,7 @@ public static BTreeReducingRangeMap merge(BTreeReducingRangeMap histor */ public static , V> M update( M map, Unseekables keysOrRanges, V value, BiFunction valueResolver, - BiFunction factory, BoundariesBuilderFactory builderFactory) + Function factory, BoundariesBuilderFactory builderFactory) { if (keysOrRanges.isEmpty()) return map; @@ -276,14 +278,11 @@ public static , V> M update( if (map.isEmpty()) return create(keysOrRanges, value, builderFactory); - if (map.inclusiveEnds() != keysOrRanges.get(0).toUnseekable().asRange().endInclusive()) - throw new IllegalStateException("Mismatching bound inclusivity/exclusivity - can't be updated"); - return update(map, keysOrRanges, value, valueResolver, factory); } private static , V> M update( - M map, Unseekables keysOrRanges, V value, BiFunction valueResolver, BiFunction factory) + M map, Unseekables keysOrRanges, V value, BiFunction valueResolver, Function factory) { Accumulator acc = accumulator(); @@ -382,7 +381,7 @@ else if (endIns == treeSize) // range ends after last entry in the map Object[] updated = acc.apply(map.tree); acc.reuse(); return map.tree == updated ? map - : factory.apply(map.inclusiveEnds(), updated); + : factory.apply(updated); } private static final ThreadLocal> accumulator = ThreadLocal.withInitial(Accumulator::new); @@ -482,15 +481,15 @@ private static V ifSupersedes(Entry entry, V newValue, BiFunc static class Builder extends AbstractBoundariesBuilder> { - protected Builder(boolean inclusiveEnds, int capacity) + protected Builder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override protected BTreeReducingRangeMap buildInternal(Object[] tree) { - return new BTreeReducingRangeMap<>(inclusiveEnds, tree); + return new BTreeReducingRangeMap<>(tree); } } diff --git a/accord-core/src/main/java/accord/utils/IntrusiveLinkedList.java b/accord-core/src/main/java/accord/utils/IntrusiveLinkedList.java index 209ea90a3a..53d09ffb9b 100644 --- a/accord-core/src/main/java/accord/utils/IntrusiveLinkedList.java +++ b/accord-core/src/main/java/accord/utils/IntrusiveLinkedList.java @@ -29,10 +29,10 @@ * A simple intrusive double-linked list for maintaining a list of tasks, * useful for invalidating queued ordered tasks * - * TODO (low priority): COPIED FROM CASSANDRA + * TODO (low priority): COPIED FROM CASSANDRA (TEST CLASSES). NOW USED IN CASSANDRA IMPL VIA HERE... SHOULD CLEANUP. */ -@SuppressWarnings("unchecked") +@SuppressWarnings({"unchecked", "unused"}) public class IntrusiveLinkedList extends IntrusiveLinkedListNode implements Iterable { public IntrusiveLinkedList() diff --git a/accord-core/src/main/java/accord/utils/IntrusiveStack.java b/accord-core/src/main/java/accord/utils/IntrusiveStack.java new file mode 100644 index 0000000000..dedc02078c --- /dev/null +++ b/accord-core/src/main/java/accord/utils/IntrusiveStack.java @@ -0,0 +1,241 @@ +/* + * 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.utils; + +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.Function; + +import net.nicoulaj.compilecommand.annotations.Inline; + +/** + * An efficient stack/list that is expected to be ordinarily either empty or close to, and for which + * we need concurrent insertions and do not need to support removal - i.e. the list is semi immutable. + * + * This is an intrusive stack, and for simplicity we treat all + * + * TODO (desired): COPIED FROM CASSANDRA - SHARE WITH COMMON SUBMODULE + * + * @param + */ +public class IntrusiveStack> implements Iterable +{ + static class Itr> implements Iterator + { + private T next; + + Itr(T next) + { + this.next = next; + } + + @Override + public boolean hasNext() + { + return next != null; + } + + @Override + public T next() + { + T result = next; + next = result.next; + return result; + } + } + + T next; + + @Inline + protected static > T getAndPush(AtomicReferenceFieldUpdater headUpdater, O owner, T prepend) + { + return getAndPush(headUpdater, owner, prepend, (prev, next) -> { + next.next = prev; + return next; + }); + } + + protected static > T getAndPush(AtomicReferenceFieldUpdater headUpdater, O owner, T prepend, BiFunction combine) + { + while (true) + { + T head = headUpdater.get(owner); + T newHead = combine.apply(head, prepend); + if (headUpdater.compareAndSet(owner, head, newHead)) + return head; + } + } + + protected static > T pushAndGet(AtomicReferenceFieldUpdater headUpdater, O owner, T prepend, BiFunction combine) + { + while (true) + { + T head = headUpdater.get(owner); + T newHead = combine.apply(head, prepend); + if (head == newHead || headUpdater.compareAndSet(owner, head, newHead)) + return newHead; + } + } + + protected interface Setter + { + public boolean compareAndSet(O owner, T expect, T update); + } + + @Inline + protected static > T getAndPush(Function getter, Setter setter, O owner, T prepend) + { + return getAndPush(getter, setter, owner, prepend, (prev, next) -> { + next.next = prev; + return next; + }); + } + + protected static > T getAndPush(Function getter, Setter setter, O owner, T prepend, BiFunction combine) + { + while (true) + { + T head = getter.apply(owner); + if (setter.compareAndSet(owner, head, combine.apply(head, prepend))) + return head; + } + } + + protected static > void pushExclusive(AtomicReferenceFieldUpdater headUpdater, O owner, T prepend, BiFunction combine) + { + T head = headUpdater.get(owner); + headUpdater.lazySet(owner, combine.apply(head, prepend)); + } + + protected static , O> void pushExclusive(AtomicReferenceFieldUpdater headUpdater, O owner, T prepend) + { + prepend.next = headUpdater.get(owner); + headUpdater.lazySet(owner, prepend); + } + + protected static > T pushExclusive(T head, T prepend) + { + prepend.next = head; + return prepend; + } + + protected static , O> Iterable iterable(AtomicReferenceFieldUpdater headUpdater, O owner) + { + return iterable(headUpdater.get(owner)); + } + + protected static > Iterable iterable(T list) + { + return list == null ? () -> iterator(null) : list; + } + + protected static > Iterator iterator(T list) + { + return new Itr<>(list); + } + + protected static int size(IntrusiveStack list) + { + int size = 0; + while (list != null) + { + ++size; + list = list.next; + } + return size; + } + + protected static boolean isSize(int size, IntrusiveStack list) + { + while (list != null && --size >= 0) + list = list.next; + return list == null && size == 0; + } + + // requires exclusive ownership (incl. with readers) + protected T reverse() + { + return reverse((T) this); + } + + // requires exclusive ownership (incl. with readers) + protected static > T reverse(T list) + { + T prev = null; + T cur = list; + while (cur != null) + { + T next = cur.next; + cur.next = prev; + prev = cur; + cur = next; + } + return prev; + } + + @Override + public void forEach(Consumer forEach) + { + forEach((T)this, forEach); + } + + public V foldl(TriFunction foldl, P param, V accumulator) + { + T list = (T) this; + while (list != null) + { + accumulator = foldl.apply(list, param, accumulator); + list = list.next; + } + return accumulator; + } + + protected static > void forEach(T list, Consumer forEach) + { + forEach(list, Function.identity(), forEach); + } + + protected static , P> void forEach(T list, BiConsumer forEach, P param) + { + forEach(list, Function.identity(), forEach, param); + } + + protected static , V> void forEach(T list, Function getter, Consumer forEach) + { + forEach(list, getter, Consumer::accept, forEach); + } + + protected static , V> void forEach(T list, Function getter, BiConsumer forEach, P param) + { + while (list != null) + { + forEach.accept(param, getter.apply(list)); + list = list.next; + } + } + + @Override + public Iterator iterator() + { + return new Itr<>((T) this); + } +} diff --git a/accord-core/src/main/java/accord/utils/PersistentField.java b/accord-core/src/main/java/accord/utils/PersistentField.java index cc63d899d9..79034fd994 100644 --- a/accord-core/src/main/java/accord/utils/PersistentField.java +++ b/accord-core/src/main/java/accord/utils/PersistentField.java @@ -19,13 +19,18 @@ package accord.utils; import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; import java.util.TreeSet; +import java.util.concurrent.Executor; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Supplier; import javax.annotation.Nonnull; -import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; import accord.utils.async.AsyncResult; import accord.utils.async.AsyncResults; @@ -40,41 +45,57 @@ public interface Persister Saved load(); } - private static class Pending + private static class PendingInput + { + final AsyncResult.Settable done = AsyncResults.settable(); + final Input input; + + private PendingInput(Input input) + { + this.input = input; + } + } + + private static class PendingSave { final int id; final Saved saving; - private Pending(int id, Saved saving) + private PendingSave(int id, Saved saving) { this.id = id; this.saving = saving; } } - @Nonnull private final Supplier currentValue; - @Nonnull - private final BiFunction merge; - @Nonnull + private final BiFunction mergeInputs; + private final BiFunction mergeToSave; private final Persister persister; - @Nonnull private final Consumer set; + private final Executor mergeExecutor; - private Saved latestPending; + private Saved latestSave; private int nextId; - private final ArrayDeque> pending = new ArrayDeque<>(); + private final List> inputBuffer = new ArrayList<>(); + private final ArrayDeque> inputs = new ArrayDeque<>(); + private final ArrayDeque> saves = new ArrayDeque<>(); private final TreeSet complete = new TreeSet<>(); + private final Lock mergeLock = new ReentrantLock(); - public PersistentField(@Nonnull Supplier currentValue, @Nonnull BiFunction merge, @Nonnull Persister persister, @Nullable Consumer set) + public PersistentField(@Nonnull Supplier currentValue, @Nonnull BiFunction mergeInputs, @Nonnull BiFunction mergeToSave, @Nonnull Persister persister, Consumer set, Executor mergeExecutor) { Invariants.nonNull(currentValue, "currentValue cannot be null"); + Invariants.nonNull(mergeInputs, "mergeInputs cannot be null"); + Invariants.nonNull(mergeToSave, "mergeToSave cannot be null"); Invariants.nonNull(persister, "persist cannot be null"); Invariants.nonNull(set, "set cannot be null"); this.currentValue = currentValue; - this.merge = merge; + this.mergeInputs = mergeInputs; + this.mergeToSave = mergeToSave; this.persister = persister; this.set = set; + this.mergeExecutor = mergeExecutor; } public void load() @@ -82,30 +103,79 @@ public void load() set.accept(persister.load()); } - public synchronized AsyncResult mergeAndUpdate(@Nonnull Input inputValue) + public AsyncResult save(@Nonnull Input inputValue) { - Invariants.nonNull(merge, "merge cannot be null"); Invariants.nonNull(inputValue, "inputValue cannot be null"); - return mergeAndUpdate(inputValue, merge); + PendingInput submit = new PendingInput<>(inputValue); + synchronized (this) + { + inputs.add(submit); + } + trySave(); + return submit.done; + } + + private void trySave() + { + if (mergeLock.tryLock()) + { + try { save(); } + finally { mergeLock.unlock(); } + + synchronized (this) + { + if (!inputs.isEmpty()) + mergeExecutor.execute(this::trySave); + } + } } - private AsyncResult mergeAndUpdate(@Nullable Input inputValue, @Nonnull BiFunction merge) + @GuardedBy("mergeLock") + private void save() { - Invariants.nonNull(merge, "merge cannot be null"); - Saved startingValue = latestPending; - if (startingValue == null) + Saved startingValue; + synchronized (this) { - Invariants.require(pending.isEmpty()); - startingValue = currentValue.get(); + if (inputs.isEmpty()) + return; + + inputBuffer.clear(); + inputBuffer.addAll(inputs); + inputs.clear(); + + startingValue = latestSave; + if (startingValue == null) + { + Invariants.require(saves.isEmpty()); + startingValue = currentValue.get(); + } } - Saved newValue = merge.apply(inputValue, startingValue); + + Input inputValue = inputBuffer.get(0).input; + for (int i = 1; i < inputBuffer.size() ; ++i) + inputValue = mergeInputs.apply(inputValue, inputBuffer.get(i).input); + + Saved newValue = mergeToSave.apply(inputValue, startingValue); if (newValue == startingValue) - return AsyncResults.success(null); - this.latestPending = newValue; - int id = ++nextId; - pending.add(new Pending<>(id, newValue)); + { + inputBuffer.forEach(i -> i.done.setSuccess(null)); + inputBuffer.clear(); + return; + } + + final List> notifyOnDone = new ArrayList<>(inputBuffer.size()); + for (PendingInput pending : inputBuffer) + notifyOnDone.add(pending.done); + inputBuffer.clear(); + + int id; + synchronized (this) + { + this.latestSave = newValue; + id = ++nextId; + saves.add(new PendingSave<>(id, newValue)); + } - AsyncResult.Settable result = AsyncResults.settable(); AsyncResult pendingWrite = persister.persist(inputValue, newValue); pendingWrite.invoke((success, fail) -> { synchronized (this) @@ -113,17 +183,15 @@ private AsyncResult mergeAndUpdate(@Nullable Input inputValue, @Nonnull BiFun complete.add(id); boolean upd = false; Saved latest = null; - while (!complete.isEmpty() && pending.peek().id == complete.first()) + while (!complete.isEmpty() && saves.peek().id == complete.first()) { - latest = pending.poll().saving; + latest = saves.poll().saving; complete.pollFirst(); upd = true; } if (upd) set.accept(latest); - result.setSuccess(null); + notifyOnDone.forEach(i -> i.setSuccess(null)); } }); - - return result; } } diff --git a/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java b/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java index bb88c0f0c7..9575fec8a6 100644 --- a/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java +++ b/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java @@ -31,6 +31,9 @@ import com.google.common.annotations.VisibleForTesting; +import static accord.api.ProtocolModifiers.RangeSpec.isEndInclusive; +import static accord.api.ProtocolModifiers.RangeSpec.isStartExclusive; +import static accord.api.ProtocolModifiers.RangeSpec.isStartInclusive; import static accord.utils.Invariants.illegalState; /** @@ -51,30 +54,21 @@ public class ReducingIntervalMap, V> @SuppressWarnings("rawtypes") public static final Comparable[] NO_OBJECTS = new Comparable[0]; - // for simplicity at construction, we permit this to be overridden by the first insertion - protected final boolean inclusiveEnds; // starts is 1 longer than values, so that starts[0] == start of values[0] protected final K[] starts; protected final V[] values; - public ReducingIntervalMap() - { - this(false); - } - @SuppressWarnings("unchecked") - public ReducingIntervalMap(boolean inclusiveEnds) + public ReducingIntervalMap() { - this.inclusiveEnds = inclusiveEnds; this.starts = (K[]) NO_OBJECTS; this.values = (V[]) NO_OBJECTS; } @VisibleForTesting - ReducingIntervalMap(boolean inclusiveEnds, K[] starts, V[] values) + ReducingIntervalMap(K[] starts, V[] values) { Invariants.requireArgument(starts.length == values.length + 1 || (starts.length + values.length) == 0); - this.inclusiveEnds = inclusiveEnds; this.starts = starts; this.values = values; } @@ -142,7 +136,7 @@ public String toString(Predicate include) { return IntStream.range(0, values.length) .filter(i -> include.test(values[i])) - .mapToObj(i -> (inclusiveStarts() ? "[" : "(") + starts[i] + "," + starts[i + 1] + (inclusiveEnds ? "]" : ")") + "=" + values[i]) + .mapToObj(i -> (isStartInclusive() ? "[" : "(") + starts[i] + "," + starts[i + 1] + (isEndInclusive() ? "]" : ")") + "=" + values[i]) .collect(Collectors.joining(", ", "{", "}")); } @@ -152,7 +146,7 @@ public boolean equals(Object o) if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; ReducingIntervalMap that = (ReducingIntervalMap) o; - return inclusiveEnds == that.inclusiveEnds && Arrays.equals(starts, that.starts) && Arrays.equals(values, that.values); + return Arrays.equals(starts, that.starts) && Arrays.equals(values, that.values); } public int hashCode() @@ -160,11 +154,6 @@ public int hashCode() return Arrays.hashCode(values); } - public boolean inclusiveEnds() - { - return inclusiveEnds; - } - public V get(K key) { int idx = find(key); @@ -200,15 +189,10 @@ private int find(K key) { int idx = Arrays.binarySearch(starts, key); if (idx < 0) idx = -2 - idx; - else if (inclusiveEnds) --idx; + else if (isEndInclusive()) --idx; return idx; } - protected final boolean inclusiveStarts() - { - return !inclusiveEnds; - } - public int size() { return values.length; @@ -234,8 +218,7 @@ protected static , V, M extends ReducingInterval if (historyRight == null || historyRight.values.length == 0) return historyLeft; - boolean inclusiveEnds = inclusiveEnds(historyLeft.inclusiveEnds, historyLeft.size() > 0, historyRight.inclusiveEnds, historyRight.size() > 0); - IntervalBuilder builder = factory.create(inclusiveEnds, historyLeft.size() + historyRight.size()); + IntervalBuilder builder = factory.create(historyLeft.size() + historyRight.size()); ReducingIntervalMap.RangeIterator left = historyLeft.rangeIterator(); ReducingIntervalMap.RangeIterator right = historyRight.rangeIterator(); @@ -307,8 +290,7 @@ protected static , V, M extends ReducingInterval if (historyRight == null || historyRight.values.length == 0) return historyLeft; - boolean inclusiveEnds = inclusiveEnds(historyLeft.inclusiveEnds, historyLeft.size() > 0, historyRight.inclusiveEnds, historyRight.size() > 0); - AbstractBoundariesBuilder builder = factory.create(inclusiveEnds, historyLeft.size() + historyRight.size()); + AbstractBoundariesBuilder builder = factory.create(historyLeft.size() + historyRight.size()); ReducingIntervalMap.RangeIterator left = historyLeft.rangeIterator(); ReducingIntervalMap.RangeIterator right = historyRight.rangeIterator(); @@ -396,11 +378,11 @@ RangeIterator intersecting(K start, K end) { int from = Arrays.binarySearch(starts, start); if (from < 0) from = Math.max(0, -2 - from); - else if (!inclusiveStarts()) ++from; + else if (isStartExclusive()) ++from; int to = Arrays.binarySearch(starts, end); if (to < 0) to = -1 - to; - else if (inclusiveStarts()) ++to; + else if (isStartInclusive()) ++to; return new RangeIterator(from, to); } @@ -448,18 +430,16 @@ V value() protected interface BuilderFactory, V, M extends ReducingIntervalMap> { - AbstractBoundariesBuilder create(boolean inclusiveEnds, int capacity); + AbstractBoundariesBuilder create(int capacity); } protected static abstract class AbstractBoundariesBuilder, V, M extends ReducingIntervalMap> { - protected final boolean inclusiveEnds; protected final List starts; protected final List values; private K safeToAdd; - protected AbstractBoundariesBuilder(boolean inclusiveEnds, int estimatedCapacity) + protected AbstractBoundariesBuilder(int estimatedCapacity) { - this.inclusiveEnds = inclusiveEnds; this.starts = new ArrayList<>(estimatedCapacity); this.values = new ArrayList<>(estimatedCapacity + 1); } @@ -564,7 +544,7 @@ public void clear() protected interface IntervalBuilderFactory, V, M extends ReducingIntervalMap> { - IntervalBuilder create(boolean inclusiveEnds, int capacity); + IntervalBuilder create(int capacity); } protected static abstract class IntervalBuilder, V, M> @@ -581,14 +561,12 @@ protected V tryMergeEqual(V a, V b) protected static abstract class AbstractIntervalBuilder, V, M> extends IntervalBuilder { - protected final boolean inclusiveEnds; protected final List starts; protected final List values; private K prevEnd; - protected AbstractIntervalBuilder(boolean inclusiveEnds, int capacity) + protected AbstractIntervalBuilder(int capacity) { - this.inclusiveEnds = inclusiveEnds; this.starts = new ArrayList<>(capacity); this.values = new ArrayList<>(capacity + 1); } diff --git a/accord-core/src/main/java/accord/utils/ReducingRangeMap.java b/accord-core/src/main/java/accord/utils/ReducingRangeMap.java index 409eb17de2..972df635e3 100644 --- a/accord-core/src/main/java/accord/utils/ReducingRangeMap.java +++ b/accord-core/src/main/java/accord/utils/ReducingRangeMap.java @@ -21,11 +21,13 @@ import accord.primitives.*; import java.util.Arrays; +import java.util.Objects; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.IntFunction; import java.util.function.Predicate; +import static accord.api.ProtocolModifiers.RangeSpec.isEndInclusive; import static accord.utils.SortedArrays.Search.FAST; import static accord.utils.SortedArrays.exponentialSearch; @@ -38,20 +40,20 @@ public interface ReduceFunction public static class SerializerSupport { - public static ReducingRangeMap create(boolean inclusiveEnds, RoutingKey[] starts, V[] values) + public static ReducingRangeMap create(RoutingKey[] starts, V[] values) { - return new ReducingRangeMap<>(inclusiveEnds, starts, values); + return new ReducingRangeMap<>(starts, values); } } public ReducingRangeMap() { - super(false, RoutingKeys.EMPTY_KEYS_ARRAY, (V[])NO_OBJECTS); + super(RoutingKeys.EMPTY_KEYS_ARRAY, (V[])NO_OBJECTS); } - protected ReducingRangeMap(boolean inclusiveEnds, RoutingKey[] starts, V[] values) + protected ReducingRangeMap(RoutingKey[] starts, V[] values) { - super(inclusiveEnds, starts, values); + super(starts, values); } public V2 foldl(Routables routables, BiFunction fold, V2 accumulator) @@ -136,21 +138,21 @@ public A foldl(AbstractKeys keys, ReduceFunction fo int i = 0, j = keys.find(starts[0], FAST); if (j < 0) j = -1 - j; - else if (inclusiveEnds) ++j; + else if (isEndInclusive()) ++j; while (j < keys.size()) { // TODO (desired): first search should be binarySearch i = exponentialSearch(starts, i, starts.length, keys.get(j)); if (i < 0) i = -2 - i; - else if (inclusiveEnds) --i; + else if (isEndInclusive()) --i; if (i >= values.length) return accumulator; int nextj = keys.findNext(j, starts[i + 1], FAST); if (nextj < 0) nextj = -1 -nextj; - else if (inclusiveEnds) ++nextj; + else if (isEndInclusive()) ++nextj; if (j != nextj && values[i] != null) { @@ -187,7 +189,7 @@ public V2 foldl(AbstractRanges ranges, ReduceFunction V2 foldl(AbstractRanges ranges, ReduceFunction i && inclusiveEnds) i = nexti - 1; + else if (nexti > i && isEndInclusive()) i = nexti - 1; else i = nexti; if (i >= values.length) @@ -209,7 +211,7 @@ public V2 foldl(AbstractRanges ranges, ReduceFunction V2 foldlWithDefault(AbstractKeys keys, ReduceFunction 0) accumulator = fold.apply(defaultValue, accumulator, p1, p2, 0, j, 0); @@ -253,14 +255,14 @@ private V2 foldlWithDefault(AbstractKeys keys, ReduceFunction= values.length) return fold.apply(defaultValue, accumulator, p1, p2, j, keys.size(), i); int nextj = keys.findNext(j, starts[i + 1], FAST); if (nextj < 0) nextj = -1 -nextj; - else if (inclusiveEnds) ++nextj; + else if (isEndInclusive()) ++nextj; if (j != nextj) { @@ -288,7 +290,7 @@ private V2 foldlWithDefault(AbstractRanges ranges, ReduceFunction 0 || starts[0].compareTo(ranges.get(0).start()) > 0) accumulator = fold.apply(defaultValue, accumulator, p1, p2, 0, j, 0); @@ -299,7 +301,7 @@ private V2 foldlWithDefault(AbstractRanges ranges, ReduceFunction i && inclusiveEnds) i = nexti - 1; + else if (nexti > i && isEndInclusive()) i = nexti - 1; else i = nexti; if (i >= values.length) @@ -313,7 +315,7 @@ private V2 foldlWithDefault(AbstractRanges ranges, ReduceFunction ReducingRangeMap create(AbstractRanges ranges, V value) public static ReducingRangeMap create(Range range, V value) { - return new ReducingRangeMap<>(range.endInclusive(), new RoutingKey[] { range.start(), range.end() }, (V[])new Object[] { value }); + return new ReducingRangeMap<>(new RoutingKey[] { range.start(), range.end() }, (V[])new Object[] { value }); } protected static > M create(Unseekables keysOrRanges, V value, BuilderFactory builder) @@ -391,7 +393,7 @@ protected static > M create(AbstractRanges rang if (value == null) throw new IllegalArgumentException("value is null"); - AbstractBoundariesBuilder builder = factory.create(ranges.get(0).endInclusive(), ranges.size() * 2); + AbstractBoundariesBuilder builder = factory.create(ranges.size() * 2); for (Range cur : ranges) { builder.appendNoOverlap(cur.start(), value); @@ -406,7 +408,7 @@ public static > M create(AbstractUnseekableKeys if (value == null) throw new IllegalArgumentException("value is null"); - AbstractBoundariesBuilder builder = factory.create(keys.get(0).asRange().endInclusive(), keys.size() * 2); + AbstractBoundariesBuilder builder = factory.create(keys.size() * 2); for (int i = 0 ; i < keys.size() ; ++i) { Range range = keys.get(i).asRange(); @@ -426,7 +428,7 @@ public static > M create(Keys keys, V value, Bu AbstractBoundariesBuilder builder; { Range range = prev.asRange(); - builder = factory.create(prev.asRange().endInclusive(), keys.size() * 2); + builder = factory.create(keys.size() * 2); builder.appendNoOverlap(range.start(), value); builder.appendNoOverlap(range.end(), null); } @@ -464,16 +466,16 @@ public static ReducingRangeMap merge(ReducingRangeMap historyLeft, Red public static class Builder extends AbstractBoundariesBuilder> { - public Builder(boolean inclusiveEnds, int capacity) + public Builder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @SuppressWarnings("unchecked") @Override protected ReducingRangeMap buildInternal() { - return new ReducingRangeMap<>(inclusiveEnds, starts.toArray(new RoutingKey[0]), (V[])values.toArray(new Object[0])); + return new ReducingRangeMap<>(starts.toArray(new RoutingKey[0]), (V[])values.toArray(new Object[0])); } } @@ -493,9 +495,47 @@ public Ranges ranges(Predicate include) public ReducingRangeMap map(Function map, IntFunction allocator) { + RoutingKey[] starts = null; V2[] output = allocator.apply(values.length); + int count = 0; for (int i = 0 ; i < values.length ; ++i) - output[i] = map.apply(values[i]); - return new ReducingRangeMap<>(inclusiveEnds, starts, output); + { + V2 next = map.apply(values[i]); + if (count == 0 ? next == null : (Objects.equals(next, output[i - 1]))) + { + if (starts == null) + { + starts = new RoutingKey[values.length]; + System.arraycopy(this.starts, 0, starts, 0, count); + } + continue; + } + if (starts != null) + starts[count] = this.starts[i]; + output[count++] = next; + } + + if (count > 0) + { + if (starts != null) + { + starts[count] = this.starts[this.starts.length - 1]; + if (output[count - 1] == null) + --count; + + starts = Arrays.copyOf(starts, count + 1); + output = Arrays.copyOf(output, count); + } + else + { + Invariants.require(count == values.length); + starts = this.starts; + } + } + + if (count == 0) + return new ReducingRangeMap<>(); + + return new ReducingRangeMap<>(starts, output); } } diff --git a/accord-core/src/main/java/accord/utils/SemiSyncCollection.java b/accord-core/src/main/java/accord/utils/SemiSyncCollection.java new file mode 100644 index 0000000000..87e6488ac0 --- /dev/null +++ b/accord-core/src/main/java/accord/utils/SemiSyncCollection.java @@ -0,0 +1,86 @@ +/* + * 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.utils; + +import java.util.Collection; +import java.util.List; +import javax.annotation.Nonnull; + +public abstract class SemiSyncCollection extends SemiSyncValue> +{ + protected static class Edit extends SemiSyncValue.Edit> + { + public final Object group; + public final E with; + public final E replace; + + public Edit(Object group, E with, E replace) + { + this.group = group; + this.with = with; + this.replace = replace; + } + + @Nonnull + @Override + protected Object group() + { + return group; + } + + @Override + protected Edit merge(Edit next) + { + return new Edit<>(next.group, next.with, replace); + } + } + + protected abstract ES merge(List es); + protected abstract V applyOne(V value, E add, E remove); + protected abstract V applyMultiple(V value, ES add, ES remove); + + protected V merge(V value, Edit edit) + { + return applyOne(value, edit.with, edit.replace); + } + + protected V merge(V value, Collection> edits) + { + ES add, remove; + try (ArrayBuffers.BufferList adds = new ArrayBuffers.BufferList<>(); + ArrayBuffers.BufferList removes = new ArrayBuffers.BufferList<>()) + { + for (Edit edit : edits) + { + if (edit.with != null) + adds.add(edit.with); + if (edit.replace != null) + removes.add(edit.replace); + } + add = merge(adds); + remove = merge(removes); + } + return applyMultiple(value, add, remove); + } + + protected SemiSyncCollection(V initialValue) + { + super(initialValue); + } +} diff --git a/accord-core/src/main/java/accord/utils/SemiSyncIntervalTree.java b/accord-core/src/main/java/accord/utils/SemiSyncIntervalTree.java new file mode 100644 index 0000000000..050eb9e8da --- /dev/null +++ b/accord-core/src/main/java/accord/utils/SemiSyncIntervalTree.java @@ -0,0 +1,66 @@ +/* + * 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.utils; + +import java.util.List; + +import accord.utils.btree.IntervalBTree; + +public abstract class SemiSyncIntervalTree extends SemiSyncCollection +{ + final IntervalBTree.IntervalComparators comparators; + + protected SemiSyncIntervalTree(IntervalBTree.IntervalComparators comparators) + { + super(IntervalBTree.empty()); + this.comparators = comparators; + } + + protected abstract Object[] tree(E edit); + + protected void pushEdit(Object group, E add, E remove) + { + pushEdit(new Edit<>(group, add, remove)); + } + + protected Object[] merge(List es) + { + if (es.isEmpty()) + return IntervalBTree.empty(); + Object[] v = tree(es.get(0)); + for (int i = 1; i < es.size() ; ++i) + v = IntervalBTree.update(v, tree(es.get(i)), comparators); + return v; + } + + protected Object[] applyOne(Object[] value, E add, E remove) + { + return applyMultiple(value, add == null ? null : tree(add), remove == null ? null : tree(remove)); + } + + protected Object[] applyMultiple(Object[] value, Object[] add, Object[] remove) + { + Object[] result = value; + if (remove != null) + result = IntervalBTree.subtract(value, remove, comparators); + if (add != null) + return IntervalBTree.update(result, add, comparators); + return result; + } +} diff --git a/accord-core/src/main/java/accord/utils/SemiSyncValue.java b/accord-core/src/main/java/accord/utils/SemiSyncValue.java new file mode 100644 index 0000000000..d86d7a5892 --- /dev/null +++ b/accord-core/src/main/java/accord/utils/SemiSyncValue.java @@ -0,0 +1,188 @@ +/* + * 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.utils; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.concurrent.locks.ReentrantLock; + +import javax.annotation.Nonnull; + +/** + * A value that may be updated asynchronously. Edits to the value are queued, and when prompted by the implementation + * an attempt is made to exclusively drain any pending edits. Before reading the value any pending edits are + * drained synchronously. + * + * Depending on the underlying collection once a read has been served it may be possible to update a collection + * before the read is complete (i.e. if the collection is immutable/copy-on-write). It is up to the implementation + * or user to ensure any mutual exclusivity required for correctness otherwise. + */ +public abstract class SemiSyncValue> implements Runnable +{ + protected static abstract class Edit> extends IntrusiveStack + { + static > boolean push(E edit, SemiSyncValue owner) + { + return null == IntrusiveStack.getAndPush(pendingEditsUpdater, (SemiSyncValue)owner, (Edit)edit); + } + + // may be some constant; if multiple pending edits, group them by this identity before applying + protected abstract @Nonnull Object group(); + + // merge two Edits for the same group + protected abstract E merge(E next); + } + + protected V value; + protected volatile E pendingEdits; + private final ReentrantLock drainPendingEditsLock = new ReentrantLock(); + private static final AtomicReferenceFieldUpdater pendingEditsUpdater = AtomicReferenceFieldUpdater.newUpdater(SemiSyncValue.class, Edit.class, "pendingEdits"); + + /** + * We have added an edit onto an empty queue; the implementation may request an immediate or asynchronous flush if helpful + */ + protected void onNewEdits() {} + + /** + * We have completed a drain and found more edits pending; the implementation may request an immediate or asynchronous flush if helpful + */ + protected void onRemainingEdits() {} + + protected abstract V merge(V value, E edit); + + protected V merge(V value, Collection edits) + { + for (E edit : edits) + value = merge(value, edit); + return value; + } + + protected SemiSyncValue(V initialValue) + { + this.value = initialValue; + } + + protected void pushEdit(E edit) + { + if (Edit.push(edit, this)) + onNewEdits(); + } + + @Override + public void run() + { + tryDrainPendingEdits(); + } + + protected V get() + { + return drainPendingEdits(); + } + + protected boolean tryDrainPendingEdits() + { + if (!tryLock()) + return false; + + try + { + drainPendingEditsExclusive(); + return true; + } + finally + { + unlock(); + } + } + + protected V drainPendingEdits() + { + lock(); + try + { + drainPendingEditsExclusive(); + return value; + } + finally + { + unlock(); + } + } + + protected void drainPendingEditsExclusive() + { + E edits = (E) pendingEditsUpdater.getAndSet(this, null); + if (edits == null) + return; + + edits = edits.reverse(); + Map editMap = null; + E pending = edits; + Object pendingKey = pending.group(); + for (E next = edits.next; next != null ; next = next.next) + { + Object nextKey = next.group(); + if (!pendingKey.equals(nextKey)) + { + if (editMap == null) editMap = new HashMap<>(); + editMap.merge(pendingKey, pending, Edit::merge); + pending = next; + pendingKey = nextKey; + } + else + { + pending = pending.merge(next); + } + } + + if (editMap == null) + { + value = merge(value, pending); + } + else + { + editMap.merge(pendingKey, pending, Edit::merge); + value = merge(value, editMap.values()); + } + } + + protected final void lock() + { + drainPendingEditsLock.lock(); + } + + protected final boolean tryLock() + { + return drainPendingEditsLock.tryLock(); + } + + protected final void unlock() + { + drainPendingEditsLock.unlock(); + postUnlock(); + } + + private void postUnlock() + { + if (pendingEdits != null && !drainPendingEditsLock.isHeldByCurrentThread()) + onRemainingEdits(); + } +} diff --git a/accord-core/src/main/java/accord/utils/async/AsyncCallbacks.java b/accord-core/src/main/java/accord/utils/async/AsyncCallbacks.java index aeea27c73b..c7ca2a60f5 100644 --- a/accord-core/src/main/java/accord/utils/async/AsyncCallbacks.java +++ b/accord-core/src/main/java/accord/utils/async/AsyncCallbacks.java @@ -30,6 +30,7 @@ public class AsyncCallbacks // a runnable interface that may be directly failed public interface RunOrFail extends Runnable { + // run should not throw any exception void run(); void fail(Throwable fail); } diff --git a/accord-core/src/main/java/accord/utils/btree/BTree.java b/accord-core/src/main/java/accord/utils/btree/BTree.java index ea3babc844..69f74d79e6 100644 --- a/accord-core/src/main/java/accord/utils/btree/BTree.java +++ b/accord-core/src/main/java/accord/utils/btree/BTree.java @@ -3378,6 +3378,7 @@ void reset() { branch.count = 0; branch.hasRightChild = false; + branch.clearSourceNode(); clearBranchBuffer(branch.buffer); if (branch.savedBuffer != null && branch.savedBuffer[0] != null) Arrays.fill(branch.savedBuffer, null); // by definition full, if non-empty @@ -3385,6 +3386,15 @@ void reset() branch = branch.parent; } Invariants.require(branch == null || (branch.count == 0 && !branch.hasRightChild)); + if (Invariants.isParanoid() && branch != null) + { + while (branch != null) + { + Invariants.require(!branch.inUse); + Invariants.require(branch.sourceNode == null); + branch = branch.parent; + } + } } /** diff --git a/accord-core/src/main/java/accord/utils/btree/IntervalBTree.java b/accord-core/src/main/java/accord/utils/btree/IntervalBTree.java index 34bd263e05..771112ab68 100644 --- a/accord-core/src/main/java/accord/utils/btree/IntervalBTree.java +++ b/accord-core/src/main/java/accord/utils/btree/IntervalBTree.java @@ -98,12 +98,12 @@ public static class InclusiveEndHelper /** * Apply the accumulation function over all intersecting intervals in the tree */ - public static V accumulate(Object[] btree, WithIntervalComparators comparators, Find find, QuadFunction function, P1 p1, P2 p2, V accumulate) + public static V accumulate(Object[] btree, WithIntervalComparators comparators, Find find, QuadFunction function, P1 p1, P2 p2, V accumulate) { if (isLeaf(btree)) { - AsymmetricComparator startWithEnd = comparators.startWithEndSeeker(); - AsymmetricComparator endWithStart = comparators.endWithStartSeeker(); + AsymmetricComparator startWithEnd = comparators.startWithEndSeeker(); + AsymmetricComparator endWithStart = comparators.endWithStartSeeker(); int keyEnd = getLeafKeyEnd(btree); for (int i = 0; i < keyEnd; ++i) { @@ -171,10 +171,10 @@ public static V accumulate(Object[] btree, QuadFunction V accumulateMaxOnly(int ifChildBefore, int ifKeyBefore, Object[] btree, WithIntervalComparators comparators, Find find, QuadFunction function, P1 p1, P2 p2, V accumulate) + private static V accumulateMaxOnly(int ifChildBefore, int ifKeyBefore, Object[] btree, WithIntervalComparators comparators, Find find, QuadFunction function, P1 p1, P2 p2, V accumulate) { - AsymmetricComparator startWithEnd = comparators.startWithEndSeeker(); - AsymmetricComparator endWithStart = comparators.endWithStartSeeker(); + AsymmetricComparator startWithEnd = comparators.startWithEndSeeker(); + AsymmetricComparator endWithStart = comparators.endWithStartSeeker(); if (isLeaf(btree)) { Invariants.require(ifChildBefore == Integer.MAX_VALUE); @@ -607,7 +607,7 @@ public static Object[] build(Collection build, IntervalComparators com /** * Build a tree of unknown size, in order. */ - public static FastIntervalTreeBuilder fastBuilder(IntervalComparators comparators) + public static FastIntervalTreeBuilder fastBuilder(IntervalComparators comparators) { TinyThreadLocalPool.TinyPool> pool = FastIntervalTreeBuilder.POOL.get(); FastIntervalTreeBuilder builder = (FastIntervalTreeBuilder) pool.poll(); diff --git a/accord-core/src/test/java/accord/impl/LocalListenersTest.java b/accord-core/src/test/java/accord/impl/LocalListenersTest.java index c4621a0ca3..811a130b2f 100644 --- a/accord-core/src/test/java/accord/impl/LocalListenersTest.java +++ b/accord-core/src/test/java/accord/impl/LocalListenersTest.java @@ -19,6 +19,7 @@ package accord.impl; import java.util.ArrayList; +import java.util.Arrays; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; @@ -39,12 +40,15 @@ import accord.api.LocalListeners.ComplexListener; import accord.api.RemoteListeners.NoOpRemoteListeners; import accord.local.Command; +import accord.local.Node; import accord.local.SafeCommand; import accord.local.SafeCommandStore; +import accord.primitives.Routable; import accord.primitives.SaveStatus; import accord.primitives.Status.Durability; import accord.local.StoreParticipants; import accord.primitives.Ballot; +import accord.primitives.Txn; import accord.primitives.TxnId; import accord.utils.AccordGens; import accord.utils.Invariants; @@ -433,6 +437,49 @@ public Command updateAttributes(StoreParticipants participants, Ballot promised, } } + @Test + public void testTxnListeners() + { + DefaultLocalListeners listeners = new DefaultLocalListeners(new RemoteListenersTest.TestCommandStore(1), new DefaultRemoteListeners((DefaultRemoteListeners.NotifySink) null), null); + TxnId txnId1 = new TxnId(1, 1, Txn.Kind.Write, Routable.Domain.Key, new Node.Id(1)); + TxnId txnId2 = new TxnId(1, 2, Txn.Kind.Write, Routable.Domain.Key, new Node.Id(1)); + TxnId txnId3 = new TxnId(1, 3, Txn.Kind.Write, Routable.Domain.Key, new Node.Id(1)); + TxnId txnId4 = new TxnId(1, 4, Txn.Kind.Write, Routable.Domain.Key, new Node.Id(1)); + listeners.register(txnId1, SaveStatus.Applied, txnId2); + listeners.register(txnId1, SaveStatus.Applied, txnId3); + listeners.register(txnId1, SaveStatus.Applying, txnId2); + listeners.register(txnId2, SaveStatus.Applied, txnId3); + listeners.register(txnId2, SaveStatus.Applied, txnId4); + TreeMap>> actual = new TreeMap<>(); + listeners.txnListeners().forEach(l -> { + actual.computeIfAbsent(l.waitingOn, ignore -> new TreeMap<>()) + .computeIfAbsent(l.awaitingStatus, ignore -> new ArrayList<>()) + .add(l.waiter); + }); + TreeMap>> expected = new TreeMap<>(); + expected.computeIfAbsent(txnId1, ignore -> new TreeMap<>()) + .put(SaveStatus.Applying, Arrays.asList(txnId2)); + expected.computeIfAbsent(txnId1, ignore -> new TreeMap<>()) + .put(SaveStatus.Applied, Arrays.asList(txnId2, txnId3)); + expected.computeIfAbsent(txnId2, ignore -> new TreeMap<>()) + .put(SaveStatus.Applied, Arrays.asList(txnId3, txnId4)); + Assertions.assertEquals(expected, actual); + } - + @Test + public void testComplexListeners() + { + DefaultLocalListeners listeners = new DefaultLocalListeners(new RemoteListenersTest.TestCommandStore(1), new DefaultRemoteListeners((DefaultRemoteListeners.NotifySink) null), null); + TxnId txnId1 = new TxnId(1, 1, Txn.Kind.Write, Routable.Domain.Key, new Node.Id(1)); + ComplexListener listener1 = (safeStore, safeCommand) -> false; + ComplexListener listener2 = (safeStore, safeCommand) -> false; + listeners.register(txnId1, listener1); + listeners.register(txnId1, listener2); + List waiters = new ArrayList<>(); + listeners.complexListeners().forEach(l -> { + Assertions.assertEquals(txnId1, l.waitingOn()); + waiters.add(l.waiting()); + }); + Assertions.assertEquals(Arrays.asList(listener1, listener2), waiters); + } } diff --git a/accord-core/src/test/java/accord/impl/RemoteListenersTest.java b/accord-core/src/test/java/accord/impl/RemoteListenersTest.java index 32d6cc14f0..d1bceda87b 100644 --- a/accord-core/src/test/java/accord/impl/RemoteListenersTest.java +++ b/accord-core/src/test/java/accord/impl/RemoteListenersTest.java @@ -46,7 +46,6 @@ import accord.impl.LocalListenersTest.TestSafeCommand; import accord.local.CommandStore; import accord.local.CommandStores; -import accord.local.CommandSummaries; import accord.local.Node; import accord.local.NodeCommandStoreService; import accord.local.PreLoadContext; @@ -411,7 +410,7 @@ public Journal.Replayer replayer() } @Override protected void ensureDurable(Ranges ranges, RedundantBefore onCommandStoreDurable) {} - @Override public boolean inStore() { return false; } + @Override public boolean inStore() { return true; } @Override public AsyncChain chain(PreLoadContext context, Consumer consumer) { return null; } @Override public AsyncChain chain(PreLoadContext context, Function apply) { return null; } @Override public void shutdown() {} @@ -446,7 +445,7 @@ public void upsertRedundantBefore(RedundantBefore addRedundantBefore) } @Override public void visit(Unseekables keys, @Nullable Timestamp withLowerTxnId, Txn.Kind.Kinds kinds, ActiveCommandVisitor visit, P1 p1, P2 p2) { } - @Override public boolean visit(Unseekables keys, TxnId testTxnId, Txn.Kind.Kinds testKind, CommandSummaries.TestStartedAt testStartedAt, Timestamp testStartedAtTimestamp, ComputeIsDep computeIsDep, AllCommandVisitor visit) { return true; } + @Override public boolean visit(Unseekables keys, TxnId testTxnId, Txn.Kind.Kinds testKind, SupersedingCommandVisitor visit) { return true; } @Override public DataStore dataStore() { return null; } @Override public Agent agent() { return null; } @Override public ProgressLog progressLog() { return null; } 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 b1ad69edea..42d18f938f 100644 --- a/accord-core/src/test/java/accord/impl/basic/Cluster.java +++ b/accord-core/src/test/java/accord/impl/basic/Cluster.java @@ -751,7 +751,7 @@ public static Map run(Id[] nodes, int[] prefixes, MessageLis Journal.TopologyUpdate lastUpdate = null; { - Iterator iter = journal.replayTopologies().iterator(); + Iterator iter = journal.loadTopologies().iterator(); while (iter.hasNext()) { Journal.TopologyUpdate update = iter.next(); @@ -790,7 +790,7 @@ public static Map run(Id[] nodes, int[] prefixes, MessageLis listStore.restore(); for (CommandStore store : stores.all()) ((ListAgent) store.agent()).restore((InMemoryCommandStore) store); - journal.replay(stores); + journal.replay(stores, null); Catchup.catchup(node); // Re-enable safety checks 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 4cde69a4c5..22a29e5ba3 100644 --- a/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java +++ b/accord-core/src/test/java/accord/impl/basic/DelayedCommandStores.java @@ -31,6 +31,8 @@ import java.util.function.Consumer; import java.util.function.Function; +import javax.annotation.Nullable; + import com.google.common.collect.ImmutableSortedMap; import accord.api.Agent; @@ -40,10 +42,10 @@ import accord.api.ProgressLog; import accord.api.RoutingKey; import accord.impl.InMemoryCommandStore; +import accord.impl.InMemoryCommandStore.CommandsForRangeLoad; import accord.impl.InMemoryCommandStores; import accord.impl.InMemorySafeCommand; import accord.impl.InMemorySafeCommandsForKey; -import accord.impl.PrefixedIntHashKey; import accord.impl.basic.TaskExecutorService.Task; import accord.local.Command; import accord.local.CommandStore; @@ -54,12 +56,10 @@ import accord.local.SafeCommandStore; import accord.local.ShardDistributor; import accord.local.cfk.CommandsForKey; -import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.RoutableKey; import accord.primitives.Timestamp; import accord.primitives.TxnId; -import accord.topology.Topology; import accord.utils.Invariants; import accord.utils.RandomSource; import accord.utils.async.AsyncChain; @@ -91,11 +91,11 @@ public void validateShardStateForTesting(Journal.TopologyUpdate lastUpdate) { Snapshot current = current(); RangesForEpoch ranges = e.getValue(); - CommandStore commandStore = null; + DelayedCommandStore commandStore = null; for (ShardHolder shard : current) { if (shard.ranges().equals(ranges)) - commandStore = shard.store; + commandStore = (DelayedCommandStore) shard.store; } Invariants.nonNull(commandStore, "Each set of ranges should have a corresponding command store, but %d did not:(%s)", ranges, Arrays.toString(shards)) @@ -144,17 +144,6 @@ protected void loadSnapshot(Snapshot nextSnapshot) super.loadSnapshot(nextSnapshot); } - private static boolean contains(Topology previous, int searchPrefix) - { - for (Range range : previous.ranges()) - { - int prefix = ((PrefixedIntHashKey) range.start()).prefix; - if (prefix == searchPrefix) - return true; - } - return false; - } - public static class DelayedCommandStore extends InMemoryCommandStore { public class DelayedTask extends Task @@ -169,6 +158,16 @@ private DelayedTask(Callable call, Pending origin) super(call, origin); } + private DelayedTask(Callable fn, @Nullable Cancellable ifCancelled) + { + super(fn, ifCancelled); + } + + private DelayedTask(Callable fn, Pending origin, @Nullable Cancellable ifCancelled) + { + super(fn, origin, ifCancelled); + } + public DelayedCommandStore owner() { return DelayedCommandStore.this; @@ -305,13 +304,13 @@ public boolean inStore() @Override public AsyncChain chain(PreLoadContext context, Consumer consumer) { - return submit(newTask(context, i -> { consumer.accept(i); return null; })); + return chain(context, i -> { consumer.accept(i); return null; }); } @Override public AsyncChain chain(PreLoadContext context, Function function) { - return submit(newTask(context, function)); + return submit(newTask(context, cfrLoad(context), function)); } @Override @@ -338,12 +337,12 @@ private void execute(DelayedTask task) runNextTask(); } - private DelayedTask newTask(PreLoadContext context, Function function) + private DelayedTask newTask(PreLoadContext context, @Nullable CommandsForRangeLoad cfrLoad, Function function) { Pending origin = Pending.Global.activeOrigin(); if (RecurringPendingRunnable.isRecurring(origin) && context.primaryTxnId() != null && !context.primaryTxnId().isSystemTxn()) origin = null; - return new DelayedTask<>(() -> executeInContext(this, context, function), origin); + return new DelayedTask<>(() -> executeInContext(this, context, cfrLoad, function), origin, cfrLoad); } private AsyncChain submit(DelayedTask task) @@ -403,9 +402,9 @@ public void shutdown() } @Override - protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges, Map commands, Map commandsForKeys) + protected InMemorySafeStore createSafeStore(PreLoadContext context, CommandsForRangeLoad cfrLoad, Map commands, Map commandsForKeys) { - return new DelayedSafeStore(this, ranges, context, commands, commandsForKeys, cacheLoading); + return new DelayedSafeStore(this, context, cfrLoad, commands, commandsForKeys, cacheLoading); } } @@ -416,13 +415,13 @@ public static class DelayedSafeStore extends InMemoryCommandStore.InMemorySafeSt private final CacheLoading cacheLoading; public DelayedSafeStore(DelayedCommandStore commandStore, - RangesForEpoch ranges, PreLoadContext context, + CommandsForRangeLoad cfrLoad, Map commands, Map commandsForKey, CacheLoading cacheLoading) { - super(commandStore, ranges, context, commands, commandsForKey); + super(commandStore, context, cfrLoad, commands, commandsForKey); this.commandStore = commandStore; this.cacheLoading = cacheLoading; ++counter; diff --git a/accord-core/src/test/java/accord/impl/basic/InMemoryJournal.java b/accord-core/src/test/java/accord/impl/basic/InMemoryJournal.java index f079237ae0..5d08735455 100644 --- a/accord-core/src/test/java/accord/impl/basic/InMemoryJournal.java +++ b/accord-core/src/test/java/accord/impl/basic/InMemoryJournal.java @@ -127,6 +127,11 @@ public InMemoryJournal(Node.Id id, RandomSource random) this.partialCompactionChance = 1f - (random.nextFloat()/2); } + @Override + public void open(Node node) + { + } + public void start(Node node) { this.node = node; @@ -247,7 +252,7 @@ public void saveCommand(int commandStoreId, CommandUpdate update, Runnable onFlu } @Override - public List replayTopologies() + public List loadTopologies() { return new ArrayList<>(topologyUpdates); } @@ -617,7 +622,7 @@ public void purge(CommandStores commandStores, EpochSupplier minEpoch) } @Override - public boolean replay(CommandStores commandStores) + public boolean replay(CommandStores commandStores, Object param) { for (Map.Entry> diffEntry : diffsPerCommandStore.entrySet()) { diff --git a/accord-core/src/test/java/accord/impl/basic/LoggingJournal.java b/accord-core/src/test/java/accord/impl/basic/LoggingJournal.java index 00f675fc6b..5c342440c9 100644 --- a/accord-core/src/test/java/accord/impl/basic/LoggingJournal.java +++ b/accord-core/src/test/java/accord/impl/basic/LoggingJournal.java @@ -75,7 +75,12 @@ private synchronized void log(String format, Object... objects) } @Override + public void open(Node node) + { + delegate.open(node); + } + @Override public void start(Node node) { delegate.start(node); @@ -107,10 +112,10 @@ public void saveCommand(int store, CommandUpdate update, Runnable onFlush) } @Override - public List replayTopologies() + public List loadTopologies() { log("REPLAY TOPOLOGIES\n"); - return delegate.replayTopologies(); + return delegate.loadTopologies(); } @Override @@ -130,9 +135,9 @@ public void purge(CommandStores commandStores, EpochSupplier minEpoch) } @Override - public boolean replay(CommandStores commandStores) + public boolean replay(CommandStores commandStores, Object param) { - return delegate.replay(commandStores); + return delegate.replay(commandStores, null); } @Override diff --git a/accord-core/src/test/java/accord/impl/basic/TaskExecutorService.java b/accord-core/src/test/java/accord/impl/basic/TaskExecutorService.java index f8a79e5f5a..17c4a34429 100644 --- a/accord-core/src/test/java/accord/impl/basic/TaskExecutorService.java +++ b/accord-core/src/test/java/accord/impl/basic/TaskExecutorService.java @@ -28,23 +28,38 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import javax.annotation.Nullable; + import accord.api.AsyncExecutor; import accord.utils.async.AsyncResults; +import accord.utils.async.Cancellable; public abstract class TaskExecutorService extends AbstractExecutorService implements AsyncExecutor { static abstract class Task extends AsyncResults.RunnableResult implements Pending, RunnableFuture { final Pending origin; + final @Nullable Cancellable ifCancelled; + public Task(Callable fn) { - this(fn, Pending.Global.activeOrigin()); + this(fn, (Cancellable) null); + } + public Task(Callable fn, @Nullable Cancellable ifCancelled) + { + this(fn, Pending.Global.activeOrigin(), ifCancelled); } public Task(Callable fn, Pending origin) + { + this(fn, origin, null); + } + + public Task(Callable fn, Pending origin, @Nullable Cancellable ifCancelled) { super(fn); this.origin = origin == null ? this : origin; + this.ifCancelled = ifCancelled; } public Pending origin() 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 1458e0eb7e..aaeb04684f 100644 --- a/accord-core/src/test/java/accord/impl/list/ListStore.java +++ b/accord-core/src/test/java/accord/impl/list/ListStore.java @@ -164,20 +164,12 @@ private Snapshot(NavigableMap> data, List { - if (fail == null) commandStore.execute((PreLoadContext.Empty)()->"Report DataStore Durable", safeStore -> safeStore.upsertRedundantBefore(onSuccess)); + if (fail == null) commandStore.execute((PreLoadContext.Empty)()->"Report DataStore Durable", safeStore -> safeStore.reportDurable(onSuccess, flags)); }); } 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 22a498861a..e75b136222 100644 --- a/accord-core/src/test/java/accord/impl/mock/MockStore.java +++ b/accord-core/src/test/java/accord/impl/mock/MockStore.java @@ -159,8 +159,8 @@ public FetchResult fetch(Node node, SafeCommandStore safeStore, Ranges ranges, S } @Override - public void ensureDurable(CommandStore commandStore, Ranges ranges, RedundantBefore reportOnSuccess) + public void ensureDurable(CommandStore commandStore, RedundantBefore reportOnSuccess, int flags) { - commandStore.execute((PreLoadContext.Empty)() -> "Report CommandStore Durable", safeStore -> safeStore.upsertRedundantBefore(reportOnSuccess)); + commandStore.execute((PreLoadContext.Empty)() -> "Report CommandStore Durable", safeStore -> safeStore.reportDurable(reportOnSuccess, flags)); } } diff --git a/accord-core/src/test/java/accord/local/ImmutableCommandTest.java b/accord-core/src/test/java/accord/local/ImmutableCommandTest.java index 037325fbdf..4b4a66103d 100644 --- a/accord-core/src/test/java/accord/local/ImmutableCommandTest.java +++ b/accord-core/src/test/java/accord/local/ImmutableCommandTest.java @@ -134,7 +134,7 @@ void noConflictWitnessTest() } commands.execute(() -> { - SafeCommandStore safeStore = commands.beginOperation(PreLoadContext.contextFor(txnId, "Test")); + SafeCommandStore safeStore = commands.beginOperation(PreLoadContext.contextFor(txnId, "Test"), null); try { StoreParticipants participants = StoreParticipants.update(safeStore, ROUTE, txnId.epoch(), txnId, txnId.epoch()); 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 dd98350e6a..d49179301d 100644 --- a/accord-core/src/test/java/accord/local/cfk/CommandsForKeyTest.java +++ b/accord-core/src/test/java/accord/local/cfk/CommandsForKeyTest.java @@ -848,7 +848,7 @@ public PreLoadContext context() } @Override - public boolean visit(Unseekables keysOrRanges, TxnId testTxnId, Txn.Kind.Kinds testKind, TestStartedAt testStartedAt, Timestamp testStartAtTimestamp, ComputeIsDep computeIsDep, AllCommandVisitor visit) + public boolean visit(Unseekables keysOrRanges, TxnId testTxnId, Txn.Kind.Kinds testKind, SupersedingCommandVisitor visit) { return false; } diff --git a/accord-core/src/test/java/accord/local/durability/DurabilityQueueTest.java b/accord-core/src/test/java/accord/local/durability/DurabilityQueueTest.java index 327103036a..0e8d6a5d1a 100644 --- a/accord-core/src/test/java/accord/local/durability/DurabilityQueueTest.java +++ b/accord-core/src/test/java/accord/local/durability/DurabilityQueueTest.java @@ -324,7 +324,7 @@ public ExecuteSyncPoint.DurabilityResults execute(PartialSyncPoint syncPoint, in private ReducingRangeMap result(SortedArrayList including, Topology topology, MinimalSyncPoint syncPoint) { SyncLocal syncLocal = including.contains(self) ? Self : NoLocal; - ReducingRangeMap.Builder builder = new ReducingRangeMap.Builder<>(syncPoint.route.get(0).endInclusive(), topology.size()); + ReducingRangeMap.Builder builder = new ReducingRangeMap.Builder<>(topology.size()); for (Shard shard : topology.shards()) { SortedArrayList shardIncluding = shard.nodes.intersecting(including); diff --git a/accord-core/src/test/java/accord/primitives/TxnIdTest.java b/accord-core/src/test/java/accord/primitives/TxnIdTest.java index 5fb21f1a22..e1fa5f3b98 100644 --- a/accord-core/src/test/java/accord/primitives/TxnIdTest.java +++ b/accord-core/src/test/java/accord/primitives/TxnIdTest.java @@ -33,7 +33,7 @@ void stringSerde() qt().forAll(AccordGens.txnIds()).check(id -> { Assertions.assertThat(TxnId.parse(id.toString())).isEqualTo(id); - Assertions.assertThat(Timestamp.fromString(id.toStandardString())).isEqualTo(new Timestamp(id)); + Assertions.assertThat(Timestamp.parse(id.toStandardString())).isEqualTo(new Timestamp(id)); }); } } \ No newline at end of file diff --git a/accord-core/src/test/java/accord/utils/AccordGens.java b/accord-core/src/test/java/accord/utils/AccordGens.java index 037f525eb3..b808cb12f5 100644 --- a/accord-core/src/test/java/accord/utils/AccordGens.java +++ b/accord-core/src/test/java/accord/utils/AccordGens.java @@ -637,7 +637,7 @@ public static Gen redundantBefore(Gen rangesGen, return rs -> { Ranges ranges = rangesGen.next(rs); if (ranges.isEmpty()) return RedundantBefore.EMPTY; - RedundantBefore.Builder builder = new RedundantBefore.Builder(ranges.get(0).endInclusive(), ranges.size()); + RedundantBefore.Builder builder = new RedundantBefore.Builder(ranges.size()); ranges.forEach(r -> builder.append(r.start(), r.end(), entryGen.apply(rs, r))); return builder.build(); }; diff --git a/accord-core/src/test/java/accord/utils/BTreeReducingRangeMapTest.java b/accord-core/src/test/java/accord/utils/BTreeReducingRangeMapTest.java index 34c16a8e4d..00e4c8ca16 100644 --- a/accord-core/src/test/java/accord/utils/BTreeReducingRangeMapTest.java +++ b/accord-core/src/test/java/accord/utils/BTreeReducingRangeMapTest.java @@ -120,7 +120,7 @@ private static BTreeReducingRangeMap h(Pair... for (int i = 1 ; i < length ; ++i) builder.append(points[i - 1].left, points[i].right); builder.append(points[length - 1].left); - return builder.build(BTreeReducingRangeMap::new); + return builder.build((inclusiveEnds, tree) -> new BTreeReducingRangeMap(tree)); } static @@ -314,9 +314,9 @@ RandomWithCanonical merge(Random random, RandomWithCanonical other) static class IntervalBuilder extends BTreeReducingIntervalMap.AbstractIntervalBuilder> { - protected IntervalBuilder(boolean inclusiveEnds, int capacity) + protected IntervalBuilder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override @@ -340,7 +340,7 @@ protected Timestamp tryMergeEqual(@Nonnull Timestamp a, Timestamp b) @Override protected BTreeReducingRangeMap buildInternal(Object[] tree) { - return new BTreeReducingRangeMap<>(inclusiveEnds, tree); + return new BTreeReducingRangeMap<>(tree); } } diff --git a/accord-core/src/test/java/accord/utils/ReducingRangeMapTest.java b/accord-core/src/test/java/accord/utils/ReducingRangeMapTest.java index a1a84a7be4..1aadfe8093 100644 --- a/accord-core/src/test/java/accord/utils/ReducingRangeMapTest.java +++ b/accord-core/src/test/java/accord/utils/ReducingRangeMapTest.java @@ -126,7 +126,7 @@ private static ReducingRangeMap h(Pair... poin timestamps[i - 1] = points[i].right; } routingKeys[length - 1] = points[length - 1].left; - return new ReducingRangeMap<>(true, routingKeys, timestamps); + return new ReducingRangeMap<>(routingKeys, timestamps); } static @@ -319,9 +319,9 @@ RandomWithCanonical merge(Random random, RandomWithCanonical other) static class IntervalBuilder extends ReducingIntervalMap.AbstractIntervalBuilder> { - protected IntervalBuilder(boolean inclusiveEnds, int capacity) + protected IntervalBuilder(int capacity) { - super(inclusiveEnds, capacity); + super(capacity); } @Override @@ -345,7 +345,7 @@ protected Timestamp tryMergeEqual(Timestamp a, Timestamp b) @Override protected ReducingRangeMap buildInternal() { - return new ReducingRangeMap<>(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new Timestamp[0])); + return new ReducingRangeMap<>(starts.toArray(new RoutingKey[0]), values.toArray(new Timestamp[0])); } } diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java b/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java index 9578254b2a..aa2d9892af 100644 --- a/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java +++ b/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java @@ -388,15 +388,16 @@ public static void run(Id[] nodes, QueueSupplier queueSupplier, Consumer public static class NoOpJournal implements Journal { + @Override public void open(Node node) { } @Override public void start(Node node) { } @Override public Command loadCommand(int store, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) { throw new IllegalStateException("Not impelemented"); } @Override public Command.Minimal loadMinimal(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) { throw new IllegalStateException("Not impelemented"); } @Override public Command.MinimalWithDeps loadMinimalWithDeps(int store, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) { throw new IllegalStateException("Not impelemented"); } @Override public void saveCommand(int store, CommandUpdate value, Runnable onFlush) { throw new IllegalStateException("Not impelemented"); } - @Override public List replayTopologies() { throw new IllegalStateException("Not impelemented"); } + @Override public List loadTopologies() { throw new IllegalStateException("Not impelemented"); } @Override public void saveTopology(TopologyUpdate topologyUpdate, Runnable onFlush) { throw new IllegalStateException("Not impelemented"); } @Override public void purge(CommandStores commandStores, EpochSupplier minEpoch) { throw new IllegalStateException("Not impelemented"); } - @Override public boolean replay(CommandStores commandStores) { throw new IllegalStateException("Not impelemented"); } + @Override public boolean replay(CommandStores commandStores, Object param) { throw new IllegalStateException("Not impelemented"); } @Override public RedundantBefore loadRedundantBefore(int store) { throw new IllegalStateException("Not impelemented"); } @Override public NavigableMap loadBootstrapBeganAt(int store) { throw new IllegalStateException("Not impelemented"); } @Override public NavigableMap loadSafeToRead(int store) { throw new IllegalStateException("Not impelemented"); } diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromStore.java b/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromStore.java index 241100c5ef..75ad833b5b 100644 --- a/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromStore.java +++ b/accord-maelstrom/src/main/java/accord/maelstrom/MaelstromStore.java @@ -62,7 +62,7 @@ public FetchResult fetch(Node node, SafeCommandStore safeStore, Ranges ranges, S } @Override - public void ensureDurable(CommandStore commandStore, Ranges ranges, RedundantBefore reportOnSuccess) + public void ensureDurable(CommandStore commandStore, RedundantBefore reportOnSuccess, int flags) { - commandStore.execute((PreLoadContext.Empty)() -> "Report CommandStore Durable", safeStore -> safeStore.upsertRedundantBefore(reportOnSuccess)); + commandStore.execute((PreLoadContext.Empty)() -> "Report CommandStore Durable", safeStore -> safeStore.reportDurable(reportOnSuccess, flags)); }}