Refactor CommandsForKey for efficiency, and to support transitive dependency elision
patch by Benedict; reviewed by Aleksey Yeshchenko for CASSANDRA-19310
diff --git a/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java b/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java
index b49ac43..4e17cf2 100644
--- a/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java
+++ b/accord-core/src/main/java/accord/coordinate/AbstractCoordinatePreAccept.java
@@ -23,12 +23,14 @@
import java.util.Set;
import java.util.function.BiConsumer;
+import javax.annotation.Nullable;
+
import accord.coordinate.tracking.QuorumTracker;
import accord.local.Node;
import accord.local.Node.Id;
import accord.messages.Callback;
import accord.primitives.FullRoute;
-import accord.primitives.Txn;
+import accord.primitives.Seekables;
import accord.primitives.TxnId;
import accord.topology.Topologies;
import accord.utils.Invariants;
@@ -95,7 +97,6 @@
final Node node;
final TxnId txnId;
- final Txn txn;
final FullRoute<?> route;
private Topologies topologies;
@@ -103,16 +104,15 @@
private ExtraEpochs extraEpochs;
private Map<Id, Object> debug = Invariants.debug() ? new LinkedHashMap<>() : null;
- AbstractCoordinatePreAccept(Node node, FullRoute<?> route, TxnId txnId, Txn txn)
+ AbstractCoordinatePreAccept(Node node, FullRoute<?> route, TxnId txnId)
{
- this(node, txnId, txn, route, node.topology().withUnsyncedEpochs(route, txnId, txnId));
+ this(node, route, txnId, node.topology().withUnsyncedEpochs(route, txnId, txnId));
}
- AbstractCoordinatePreAccept(Node node, TxnId txnId, Txn txn, FullRoute<?> route, Topologies topologies)
+ AbstractCoordinatePreAccept(Node node, FullRoute<?> route, @Nullable TxnId txnId, Topologies topologies)
{
this.node = node;
this.txnId = txnId;
- this.txn = txn;
this.route = route;
this.topologies = topologies;
}
@@ -122,6 +122,7 @@
contact(topologies.nodes(), topologies, this);
}
+ abstract Seekables<?, ?> keysOrRanges();
abstract void contact(Set<Id> nodes, Topologies topologies, Callback<R> callback);
abstract void onSuccessInternal(Id from, R reply);
/**
@@ -163,7 +164,22 @@
@Override
public final void setFailure(Throwable failure)
{
- Invariants.checkState(!initialRoundIsDone || (extraEpochs != null && !extraEpochs.extraRoundIsDone));
+ super.setFailure(failure);
+ onFailure(failure);
+ }
+
+ @Override
+ public final boolean tryFailure(Throwable failure)
+ {
+ if (!super.tryFailure(failure))
+ return false;
+ onFailure(failure);
+ return true;
+ }
+
+ private void onFailure(Throwable failure)
+ {
+ // we may already be complete, as we may receive a failure from a later phase; but it's fine to redundantly mark done
initialRoundIsDone = true;
if (extraEpochs != null)
extraEpochs.extraRoundIsDone = true;
@@ -178,7 +194,7 @@
else if (failure instanceof Invalidated)
node.agent().metricsEventsListener().onInvalidated(txnId);
}
- super.setFailure(failure);
+
}
final void onPreAcceptedOrNewEpoch()
@@ -203,7 +219,7 @@
// TODO (desired, efficiency): check if we have already have a valid quorum for the future epoch
// (noting that nodes may have adopted new ranges, in which case they should be discounted, and quorums may have changed shape)
node.withEpoch(latestEpoch, () -> {
- TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(latestEpoch), txnId, route.homeKey(), txn.keys());
+ TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(latestEpoch), txnId, route.homeKey(), keysOrRanges());
if (mismatch != null)
{
initialRoundIsDone = true;
@@ -230,23 +246,7 @@
@Override
public final void accept(T success, Throwable failure)
{
- if (success != null)
- {
- trySuccess(success);
- }
- else
- {
- if (failure instanceof CoordinationFailed)
- {
- ((CoordinationFailed) failure).set(txnId, route.homeKey());
- if (failure instanceof Preempted)
- node.agent().metricsEventsListener().onPreempted(txnId);
- else if (failure instanceof Timeout)
- node.agent().metricsEventsListener().onTimeout(txnId);
- else if (failure instanceof Invalidated)
- node.agent().metricsEventsListener().onInvalidated(txnId);
- }
- tryFailure(failure);
- }
+ if (success != null) trySuccess(success);
+ else tryFailure(failure);
}
}
diff --git a/accord-core/src/main/java/accord/coordinate/Barrier.java b/accord-core/src/main/java/accord/coordinate/Barrier.java
index ca5d346..8ce342f 100644
--- a/accord-core/src/main/java/accord/coordinate/Barrier.java
+++ b/accord-core/src/main/java/accord/coordinate/Barrier.java
@@ -18,7 +18,6 @@
package accord.coordinate;
-import java.util.Objects;
import javax.annotation.Nonnull;
import accord.local.*;
@@ -30,7 +29,7 @@
import accord.api.Key;
import accord.api.RoutingKey;
import accord.local.SafeCommandStore.TestDep;
-import accord.local.SafeCommandStore.TestTimestamp;
+import accord.local.SafeCommandStore.TestStartedAt;
import accord.primitives.Routable.Domain;
import accord.primitives.Seekables;
import accord.primitives.SyncPoint;
@@ -42,6 +41,7 @@
import static accord.local.PreLoadContext.contextFor;
import static accord.primitives.Txn.Kind.Kinds.AnyGloballyVisible;
+import static accord.local.SafeCommandStore.TestStatus.IS_STABLE;
import static accord.utils.Invariants.checkArgument;
import static accord.utils.Invariants.checkState;
import static accord.utils.Invariants.illegalState;
@@ -109,15 +109,7 @@
return;
}
- if (barrierTxn != null)
- {
- if (barrierTxn.status.equals(Status.Applied))
- {
- doBarrierSuccess(barrierTxn.executeAt);
- }
- // A listener was added to the transaction already
- }
- else
+ if (barrierTxn == null)
{
createSyncPoint();
}
@@ -143,7 +135,8 @@
private void createSyncPoint()
{
- coordinateSyncPoint = CoordinateSyncPoint.inclusive(node, seekables, barrierType.async);
+ coordinateSyncPoint = barrierType.async ? CoordinateSyncPoint.inclusive(node, seekables)
+ : CoordinateSyncPoint.inclusiveAndAwaitQuorum(node, seekables);
coordinateSyncPoint.addCallback((syncPoint, syncPointFailure) -> {
if (syncPointFailure != null)
{
@@ -153,7 +146,7 @@
// Need to wait for the local transaction to finish since coordinate sync point won't wait on anything
// if async was requested or there were no deps found
- if (syncPoint.finishedAsync)
+ if (barrierType.async)
{
TxnId txnId = syncPoint.syncId;
long epoch = txnId.epoch();
@@ -201,7 +194,7 @@
ExistingTransactionCheck check = new ExistingTransactionCheck();
Key k = seekables.get(0).asKey();
node.commandStores().mapReduceConsume(
- contextFor(k, KeyHistory.ALL),
+ contextFor(k, KeyHistory.COMMANDS),
k.toUnseekable(),
minEpoch,
Long.MAX_VALUE,
@@ -217,23 +210,13 @@
@Nonnull
public final Timestamp executeAt;
@Nonnull
- public final Status status;
- @Nonnull
public final Key key;
- public BarrierTxn(@Nonnull TxnId txnId, @Nonnull Timestamp executeAt, @Nonnull Status status, Key key)
+ public BarrierTxn(@Nonnull TxnId txnId, @Nonnull Timestamp executeAt, Key key)
{
this.txnId = txnId;
this.executeAt = executeAt;
- this.status = status;
this.key = key;
}
-
- public BarrierTxn max(BarrierTxn other)
- {
- if (other == null)
- return this;
- return status.compareTo(other.status) >= 0 ? this : other;
- }
}
/*
@@ -248,31 +231,29 @@
@Override
public BarrierTxn apply(SafeCommandStore safeStore)
{
- BarrierTxn found = safeStore.mapReduce(
- seekables,
- safeStore.ranges().allAfter(minEpoch),
- KeyHistory.ALL,
- // 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
- AnyGloballyVisible,
- TestTimestamp.EXECUTES_AFTER,
- TxnId.minForEpoch(minEpoch),
- TestDep.ANY_DEPS,
- null,
- Status.Committed,
- Status.Applied,
- (p1, keyOrRange, txnId, executeAt, status, deps, barrierTxn) -> {
- if (keyOrRange.domain() == Domain.Key)
- return new BarrierTxn(txnId, executeAt, status, keyOrRange.asKey());
- return null;
- },
- null,
- null,
- // Take the first one we find, and call it good enough to wait on
- Objects::nonNull);
+ // TODO (required): consider these semantics
+ BarrierTxn found = safeStore.mapReduceFull(
+ seekables,
+ safeStore.ranges().allAfter(minEpoch),
+ // 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
+ TxnId.minForEpoch(minEpoch),
+ AnyGloballyVisible,
+ TestStartedAt.STARTED_AFTER,
+ TestDep.ANY_DEPS,
+ IS_STABLE,
+ (p1, keyOrRange, txnId, executeAt, barrierTxn) -> {
+ if (barrierTxn != null)
+ return barrierTxn;
+ if (keyOrRange.domain() == Domain.Key)
+ return new BarrierTxn(txnId, executeAt, keyOrRange.asKey());
+ return null;
+ },
+ null,
+ null);
// It's not applied so add a listener to find out when it is applied
- if (found != null && !found.status.equals(Status.Applied))
+ if (found != null)
{
safeStore.commandStore().execute(
contextFor(found.txnId),
diff --git a/accord-core/src/main/java/accord/coordinate/BlockOnDeps.java b/accord-core/src/main/java/accord/coordinate/BlockOnDeps.java
deleted file mode 100644
index 65abe73..0000000
--- a/accord-core/src/main/java/accord/coordinate/BlockOnDeps.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.coordinate;
-
-import java.util.function.BiConsumer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import accord.api.Result;
-import accord.coordinate.tracking.QuorumTracker;
-import accord.coordinate.tracking.RequestStatus;
-import accord.local.Node;
-import accord.local.Node.Id;
-import accord.messages.Callback;
-import accord.messages.Commit;
-import accord.messages.ReadData.CommitOrReadNack;
-import accord.messages.ReadData.ReadReply;
-import accord.primitives.Deps;
-import accord.primitives.FullRoute;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
-import accord.topology.Topologies;
-
-import static accord.coordinate.tracking.RequestStatus.Failed;
-import static accord.utils.Invariants.illegalState;
-
-/**
- * Block on deps at quorum for a sync point transaction, and then move the transaction to the applied state
- */
-public class BlockOnDeps implements Callback<ReadReply>
-{
- @SuppressWarnings("unused")
- private static final Logger logger = LoggerFactory.getLogger(BlockOnDeps.class);
-
- final Node node;
- final TxnId txnId;
- final Txn txn;
- final FullRoute<?> route;
- final Deps deps;
- final Topologies blockOn;
- final QuorumTracker tracker;
- final BiConsumer<? super Result, Throwable> callback;
-
- private boolean isDone = false;
-
- private BlockOnDeps(Node node, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, BiConsumer<? super Result, Throwable> callback)
- {
- this.node = node;
- this.txnId = txnId;
- this.txn = txn;
- this.route = route;
- this.deps = deps;
- this.blockOn = node.topology().forEpoch(route, txnId.epoch());
- this.tracker = new QuorumTracker(blockOn);
- this.callback = callback;
- }
-
- public static void blockOnDeps(Node node, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, BiConsumer<? super Result, Throwable> callback)
- {
- BlockOnDeps blockOnDeps = new BlockOnDeps(node, txnId, txn, route, deps, callback);
- blockOnDeps.start();
- }
-
- void start()
- {
- Commit.stableMaximalAndBlockOnDeps(node, tracker.topologies(), txnId, txn, route, txnId, deps, this);
- }
-
- @Override
- public void onSuccess(Id from, ReadReply reply)
- {
- if (isDone)
- return;
-
- if (reply.isOk())
- {
- if (tracker.recordSuccess(from) == RequestStatus.Success)
- {
- isDone = true;
- callback.accept(txn.result(txnId, txnId, null), null);
- }
- return;
- }
-
- CommitOrReadNack nack = (CommitOrReadNack) reply;
- switch (nack)
- {
- default: throw illegalState();
- case Redundant:
- // WaitUntilApplied only sends Redundant on truncation which implies durable and applied
- isDone = true;
- callback.accept(txn.result(txnId, txnId, null), null);
- break;
- case Insufficient:
- throw illegalState("Received `NotCommitted` response after sending maximal stable as part of `BlockOnDeps`");
- case Invalid:
- onFailure(from, illegalState("Submitted a read command to a replica that did not own the range"));
- break;
- }
- }
-
- @Override
- public void onFailure(Id from, Throwable failure)
- {
- if (tracker.recordFailure(from) == Failed)
- {
- isDone = true;
- callback.accept(null, failure);
- }
- }
-
- @Override
- public void onCallbackFailure(Id from, Throwable failure)
- {
- isDone = true;
- callback.accept(null, failure);
- }
-}
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java b/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java
index b07d677..fa94afb 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateEphemeralRead.java
@@ -31,6 +31,7 @@
import accord.messages.GetEphemeralReadDeps.GetEphemeralReadDepsOk;
import accord.primitives.Deps;
import accord.primitives.FullRoute;
+import accord.primitives.Seekables;
import accord.primitives.Txn;
import accord.primitives.TxnId;
import accord.topology.Topologies;
@@ -67,19 +68,28 @@
return coordinate;
}
+ private final Txn txn;
+
private final QuorumTracker tracker;
private final List<GetEphemeralReadDepsOk> oks;
private long executeAtEpoch;
CoordinateEphemeralRead(Node node, Topologies topologies, FullRoute<?> route, TxnId txnId, Txn txn)
{
- super(node, route, txnId, txn);
+ super(node, route, txnId);
+ this.txn = txn;
this.tracker = new QuorumTracker(topologies);
this.executeAtEpoch = txnId.epoch();
this.oks = new ArrayList<>(topologies.estimateUniqueNodes());
}
@Override
+ Seekables<?, ?> keysOrRanges()
+ {
+ return txn.keys();
+ }
+
+ @Override
void contact(Set<Node.Id> nodes, Topologies topologies, Callback<GetEphemeralReadDepsOk> callback)
{
CommandStore commandStore = CommandStore.maybeCurrent();
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateGloballyDurable.java b/accord-core/src/main/java/accord/coordinate/CoordinateGloballyDurable.java
index 5186d5d..94528bb 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateGloballyDurable.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateGloballyDurable.java
@@ -31,11 +31,15 @@
import accord.utils.async.AsyncResult;
import accord.utils.async.AsyncResults.SettableResult;
+import static accord.primitives.Routable.Domain.Range;
+import static accord.primitives.Txn.Kind.LocalOnly;
+
// TODO (expected): this does not need to query every shard; can disseminate globally any sub-range of the ring
// (indeed, we could slice both the query and dissemination only so that they always overlap)
public class CoordinateGloballyDurable extends SettableResult<Void> implements Callback<DurableBeforeReply>
{
final Node node;
+ final TxnId txnId; // only used for key into journal
// TODO (expected): this can be a ReadTracker, we only need one response from each shard
final QuorumTracker tracker;
private DurableBefore durableBefore = DurableBefore.EMPTY;
@@ -44,6 +48,7 @@
{
Topologies topologies = node.topology().preciseEpochs(epoch);
this.node = node;
+ this.txnId = node.nextTxnId(LocalOnly, Range);
this.tracker = new QuorumTracker(topologies);
}
@@ -65,8 +70,8 @@
durableBefore = DurableBefore.merge(durableBefore, reply.durableBeforeMap);
if (tracker.recordSuccess(from) == RequestStatus.Success)
{
- if (durableBefore != null && !durableBefore.equals(TxnId.NONE))
- node.send(tracker.nodes(), new SetGloballyDurable(durableBefore));
+ if (durableBefore != null && durableBefore.size() != 0)
+ node.send(tracker.nodes(), new SetGloballyDurable(txnId, durableBefore));
trySuccess(null);
}
}
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateNoOp.java b/accord-core/src/main/java/accord/coordinate/CoordinateNoOp.java
deleted file mode 100644
index 40c009d..0000000
--- a/accord-core/src/main/java/accord/coordinate/CoordinateNoOp.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.coordinate;
-
-import java.util.List;
-
-import accord.api.Result;
-import accord.local.Node;
-import accord.messages.Apply;
-import accord.messages.PreAccept.PreAcceptOk;
-import accord.primitives.Ballot;
-import accord.primitives.Deps;
-import accord.primitives.FullRoute;
-import accord.primitives.Seekables;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
-import accord.primitives.Writes;
-import accord.topology.Topologies;
-import accord.utils.Invariants;
-import accord.utils.async.AsyncResult;
-import accord.utils.async.AsyncResults;
-
-import static accord.coordinate.Propose.Invalidate.proposeAndCommitInvalidate;
-import static accord.primitives.Txn.Kind.NoOp;
-
-/**
- * Perform initial rounds of PreAccept and Accept until we have reached agreement about when we should execute.
- * If we are preempted by a recovery coordinator, we abort and let them complete (and notify us about the execution result)
- *
- * TODO (desired, testing): dedicated burn test to validate outcomes
- */
-public class CoordinateNoOp extends CoordinatePreAccept<Timestamp>
-{
- private CoordinateNoOp(Node node, TxnId txnId, Txn txn, FullRoute<?> route)
- {
- super(node, txnId, txn, route);
- }
-
- public static AsyncResult<Timestamp> coordinate(Node node, Seekables<?, ?> keysOrRanges)
- {
- TxnId txnId = node.nextTxnId(NoOp, keysOrRanges.domain());
- return coordinate(node, txnId, keysOrRanges);
- }
-
- public static AsyncResult<Timestamp> coordinate(Node node, TxnId txnId, Seekables<?, ?> keysOrRanges)
- {
- Invariants.checkArgument(txnId.kind() == NoOp);
- FullRoute<?> route = node.computeRoute(txnId, keysOrRanges);
- TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(txnId.epoch()), txnId, route.homeKey(), keysOrRanges);
- if (mismatch != null)
- return AsyncResults.failure(mismatch);
- CoordinateNoOp coordinate = new CoordinateNoOp(node, txnId, node.agent().emptyTxn(NoOp, keysOrRanges), route);
- coordinate.start();
- return coordinate;
- }
-
- @Override
- void onPreAccepted(Topologies topologies, Timestamp executeAt, List<PreAcceptOk> oks)
- {
- if (executeAt.isRejected())
- {
- proposeAndCommitInvalidate(node, Ballot.ZERO, txnId, route.homeKey(), route, executeAt, this);
- }
- else
- {
- Deps preacceptDeps = Deps.merge(oks, ok -> ok.deps);
- new Propose<Timestamp>(node, topologies, Ballot.ZERO, txnId, txn, route, executeAt, preacceptDeps, this)
- {
- @Override
- void onAccepted()
- {
- Writes writes = txn.execute(txnId, txnId, null);
- Result result = txn.result(txnId, executeAt, null);
- Deps acceptDeps = Deps.merge(this.acceptOks, ok -> ok.deps);
- Apply.sendMaximal(node, txnId, route, txn, executeAt, acceptDeps, writes, result);
- accept(executeAt, null);
- }
- }.start();
- }
- }
-}
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java b/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java
index bfa0748..74979f4 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinatePreAccept.java
@@ -32,6 +32,7 @@
import accord.messages.PreAccept.PreAcceptReply;
import accord.primitives.Ballot;
import accord.primitives.FullRoute;
+import accord.primitives.Seekables;
import accord.primitives.Timestamp;
import accord.primitives.Txn;
import accord.primitives.TxnId;
@@ -51,6 +52,7 @@
{
final FastPathTracker tracker;
private final List<PreAcceptOk> oks;
+ final Txn txn;
CoordinatePreAccept(Node node, TxnId txnId, Txn txn, FullRoute<?> route)
{
@@ -59,9 +61,10 @@
CoordinatePreAccept(Node node, TxnId txnId, Txn txn, FullRoute<?> route, Topologies topologies)
{
- super(node, route, txnId, txn);
+ super(node, route, txnId);
this.tracker = new FastPathTracker(topologies);
this.oks = new ArrayList<>(topologies.estimateUniqueNodes());
+ this.txn = txn;
}
void contact(Set<Id> nodes, Topologies topologies, Callback<PreAcceptReply> callback)
@@ -80,6 +83,12 @@
}
@Override
+ Seekables<?, ?> keysOrRanges()
+ {
+ return txn.keys();
+ }
+
+ @Override
public void onFailureInternal(Id from, Throwable failure)
{
switch (tracker.recordFailure(from))
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java b/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java
index 28487c9..61b2e17 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateShardDurable.java
@@ -19,93 +19,39 @@
package accord.coordinate;
import accord.coordinate.tracking.AppliedTracker;
-import accord.coordinate.tracking.RequestStatus;
import accord.local.Node;
import accord.messages.Callback;
-import accord.messages.ReadData;
import accord.messages.ReadData.ReadReply;
import accord.messages.SetShardDurable;
import accord.messages.WaitUntilApplied;
import accord.primitives.Ranges;
import accord.primitives.SyncPoint;
-import accord.topology.Topologies;
import accord.utils.async.AsyncResult;
-import accord.utils.async.AsyncResults.SettableResult;
-/**
- *
- */
-public class CoordinateShardDurable extends SettableResult<Void> implements Callback<ReadReply>
+public class CoordinateShardDurable extends ExecuteSyncPoint<Ranges> implements Callback<ReadReply>
{
- final Node node;
- final AppliedTracker tracker;
- final SyncPoint<Ranges> exclusiveSyncPoint;
-
private CoordinateShardDurable(Node node, SyncPoint<Ranges> exclusiveSyncPoint)
{
- // TODO (required): this isn't correct, we need to potentially perform a second round if a dependency executes in a future epoch and we have lost ownership of that epoch
- Topologies topologies = node.topology().forEpoch(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.sourceEpoch());
- this.node = node;
- this.tracker = new AppliedTracker(topologies);
- this.exclusiveSyncPoint = exclusiveSyncPoint;
+ super(node, new AppliedTracker(node.topology().forEpoch(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.sourceEpoch())), exclusiveSyncPoint);
}
- public static AsyncResult<Void> coordinate(Node node, SyncPoint<Ranges> exclusiveSyncPoint)
+ public static AsyncResult<SyncPoint<Ranges>> coordinate(Node node, SyncPoint<Ranges> exclusiveSyncPoint)
{
CoordinateShardDurable coordinate = new CoordinateShardDurable(node, exclusiveSyncPoint);
coordinate.start();
return coordinate;
}
- private void start()
+ protected void start()
{
- node.send(tracker.nodes(), to -> new WaitUntilApplied(to, tracker.topologies(), exclusiveSyncPoint.syncId, exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.syncId.epoch()), this);
+ node.send(tracker.nodes(), to -> new WaitUntilApplied(to, tracker.topologies(), syncPoint.syncId, syncPoint.keysOrRanges, syncPoint.syncId.epoch()), this);
}
@Override
- public void onSuccess(Node.Id from, ReadReply reply)
+ protected void onSuccess()
{
- if (!reply.isOk())
- {
- switch ((ReadData.CommitOrReadNack)reply)
- {
- default: throw new AssertionError("Unhandled: " + reply);
-
- case Insufficient:
- CoordinateSyncPoint.sendApply(node, from, exclusiveSyncPoint);
- return;
-
- case Redundant:
- tryFailure(new RuntimeException("Unexpected reply"));
- return;
-
- case Invalid:
- tryFailure(new Invalidated(exclusiveSyncPoint.syncId, exclusiveSyncPoint.homeKey));
- return;
- }
- }
- else
- {
- // TODO (required): we also need to handle ranges not being safe to read
- if (tracker.recordSuccess(from) == RequestStatus.Success)
- {
- node.configService().reportEpochRedundant(exclusiveSyncPoint.keysOrRanges, exclusiveSyncPoint.syncId.epoch());
- node.send(tracker.nodes(), new SetShardDurable(exclusiveSyncPoint));
- trySuccess(null);
- }
- }
- }
-
- @Override
- public void onFailure(Node.Id from, Throwable failure)
- {
- if (tracker.recordFailure(from) == RequestStatus.Failed)
- tryFailure(new Exhausted(exclusiveSyncPoint.syncId, exclusiveSyncPoint.homeKey));
- }
-
- @Override
- public void onCallbackFailure(Node.Id from, Throwable failure)
- {
- tryFailure(failure);
+ node.configService().reportEpochRedundant(syncPoint.keysOrRanges, syncPoint.syncId.epoch());
+ node.send(tracker.nodes(), new SetShardDurable(syncPoint));
+ super.onSuccess();
}
}
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java b/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java
index a1987fd..49e170b 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateSyncPoint.java
@@ -19,11 +19,11 @@
package accord.coordinate;
import java.util.List;
-import java.util.function.BiConsumer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import accord.coordinate.CoordinationAdapter.Adapters;
import accord.local.Node;
import accord.messages.Apply;
import accord.messages.PreAccept.PreAcceptOk;
@@ -40,6 +40,9 @@
import accord.utils.async.AsyncResult;
import accord.utils.async.AsyncResults;
+import static accord.coordinate.CoordinationAdapter.Invoke.execute;
+import static accord.coordinate.CoordinationAdapter.Invoke.propose;
+import static accord.coordinate.ExecutePath.FAST;
import static accord.coordinate.Propose.Invalidate.proposeAndCommitInvalidate;
import static accord.primitives.Timestamp.mergeMax;
import static accord.primitives.Txn.Kind.ExclusiveSyncPoint;
@@ -57,76 +60,53 @@
@SuppressWarnings("unused")
private static final Logger logger = LoggerFactory.getLogger(CoordinateSyncPoint.class);
- final S keysOrRanges;
- // Whether to wait on the dependencies applying globally before returning a result
- final boolean async;
+ final CoordinationAdapter<SyncPoint<S>> adapter;
- private CoordinateSyncPoint(Node node, TxnId txnId, Txn txn, FullRoute<?> route, S keysOrRanges, boolean async)
+ private CoordinateSyncPoint(Node node, TxnId txnId, Txn txn, FullRoute<?> route, CoordinationAdapter<SyncPoint<S>> adapter)
{
super(node, txnId, txn, route, node.topology().withOpenEpochs(route, txnId, txnId));
- checkArgument(txnId.kind() == Kind.SyncPoint || async, "Exclusive sync points only support async application");
- this.keysOrRanges = keysOrRanges;
- this.async = async;
+ this.adapter = adapter;
}
public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> exclusive(Node node, S keysOrRanges)
{
- return coordinate(node, ExclusiveSyncPoint, keysOrRanges, true);
+ return coordinate(node, ExclusiveSyncPoint, keysOrRanges, Adapters.exclusiveSyncPoint());
}
public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> exclusive(Node node, TxnId txnId, S keysOrRanges)
{
- return coordinate(node, txnId, keysOrRanges, true);
+ return coordinate(node, txnId, keysOrRanges, Adapters.exclusiveSyncPoint());
}
- public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> inclusive(Node node, S keysOrRanges, boolean async)
+ public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> inclusive(Node node, S keysOrRanges)
{
- return coordinate(node, Kind.SyncPoint, keysOrRanges, async);
+ return coordinate(node, Kind.SyncPoint, keysOrRanges, Adapters.inclusiveSyncPoint());
}
- private static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> coordinate(Node node, Kind kind, S keysOrRanges, boolean async)
+ public static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> inclusiveAndAwaitQuorum(Node node, S keysOrRanges)
+ {
+ return coordinate(node, Kind.SyncPoint, keysOrRanges, Adapters.inclusiveSyncPointBlocking());
+ }
+
+ private static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> coordinate(Node node, Kind kind, S keysOrRanges, CoordinationAdapter<SyncPoint<S>> adapter)
{
checkArgument(kind == Kind.SyncPoint || kind == ExclusiveSyncPoint);
- node.nextTxnId(Kind.SyncPoint, keysOrRanges.domain());
TxnId txnId = node.nextTxnId(kind, keysOrRanges.domain());
- return node.withEpoch(txnId.epoch(), () -> coordinate(node, txnId, keysOrRanges, async)).beginAsResult();
+ return node.withEpoch(txnId.epoch(), () -> coordinate(node, txnId, keysOrRanges, adapter)).beginAsResult();
}
- private static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> coordinate(Node node, TxnId txnId, S keysOrRanges, boolean async)
+ private static <S extends Seekables<?, ?>> AsyncResult<SyncPoint<S>> coordinate(Node node, TxnId txnId, S keysOrRanges, CoordinationAdapter<SyncPoint<S>> adapter)
{
checkArgument(txnId.kind() == Kind.SyncPoint || txnId.kind() == ExclusiveSyncPoint);
- FullRoute route = node.computeRoute(txnId, keysOrRanges);
+ FullRoute<?> route = node.computeRoute(txnId, keysOrRanges);
TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(txnId.epoch()), txnId, route.homeKey(), keysOrRanges);
if (mismatch != null)
return AsyncResults.failure(mismatch);
- CoordinateSyncPoint<S> coordinate = new CoordinateSyncPoint(node, txnId, node.agent().emptyTxn(txnId.kind(), keysOrRanges), route, keysOrRanges, async);
+ CoordinateSyncPoint<S> coordinate = new CoordinateSyncPoint<>(node, txnId, node.agent().emptyTxn(txnId.kind(), keysOrRanges), route, adapter);
coordinate.start();
return coordinate;
}
- static <S extends Seekables<?, ?>> void blockOnDeps(Node node, Txn txn, TxnId txnId, FullRoute<?> route, S keysOrRanges, Deps deps, BiConsumer<SyncPoint<S>, Throwable> callback, boolean async)
- {
- // If deps are empty there is nothing to wait on application for so we can return immediately
- boolean processAsyncCompletion = deps.isEmpty() || async;
- BlockOnDeps.blockOnDeps(node, txnId, txn, route, deps, (result, throwable) -> {
- // Don't want to process completion twice
- if (processAsyncCompletion)
- {
- // Don't lose the error
- if (throwable != null)
- node.agent().onUncaughtException(throwable);
- return;
- }
- if (throwable != null)
- callback.accept(null, throwable);
- else
- callback.accept(new SyncPoint<S>(txnId, deps, keysOrRanges, route, false), null);
- });
- // Notify immediately and the caller can add a listener to command completion to track local application
- if (processAsyncCompletion)
- callback.accept(new SyncPoint<S>(txnId, deps, keysOrRanges, route, true), null);
- }
-
@Override
long executeAtEpoch()
{
@@ -146,14 +126,15 @@
{
// we don't need to fetch deps from Accept replies, so we don't need to contact unsynced epochs
topologies = node.topology().forEpoch(route, txnId.epoch());
+ // TODO (required): consider the required semantics of a SyncPoint
if (tracker.hasFastPathAccepted() && txnId.kind() == Kind.SyncPoint)
- blockOnDeps(node, txn, txnId, route, keysOrRanges, deps, this, async);
+ execute(adapter, node, topologies, route, FAST, txnId, txn, executeAt, deps, this);
else
- ProposeSyncPoint.proposeSyncPoint(node, topologies, Ballot.ZERO, txnId, txn, route, deps, this, async, tracker.nodes(), keysOrRanges);
+ propose(adapter, node, topologies, route, Ballot.ZERO, txnId, txn, executeAt, deps, this);
}
}
- public static void sendApply(Node node, Node.Id to, SyncPoint syncPoint)
+ public static void sendApply(Node node, Node.Id to, SyncPoint<?> syncPoint)
{
TxnId txnId = syncPoint.syncId;
Timestamp executeAt = txnId;
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java b/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java
index 39b9784..dc4395c 100644
--- a/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java
+++ b/accord-core/src/main/java/accord/coordinate/CoordinateTransaction.java
@@ -21,6 +21,7 @@
import java.util.List;
import accord.api.Result;
+import accord.coordinate.CoordinationAdapter.Adapters;
import accord.messages.PreAccept;
import accord.topology.Topologies;
import accord.local.Node;
@@ -34,9 +35,11 @@
import accord.utils.async.AsyncResult;
import accord.utils.async.AsyncResults;
-import static accord.coordinate.Execute.Path.FAST;
+import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue;
+import static accord.coordinate.CoordinationAdapter.Invoke.execute;
+import static accord.coordinate.CoordinationAdapter.Invoke.propose;
+import static accord.coordinate.ExecutePath.FAST;
import static accord.coordinate.Propose.Invalidate.proposeAndCommitInvalidate;
-import static accord.coordinate.ProposeAndExecute.proposeAndExecute;
/**
* Perform initial rounds of PreAccept and Accept until we have reached agreement about when we should execute.
@@ -46,9 +49,12 @@
*/
public class CoordinateTransaction extends CoordinatePreAccept<Result>
{
+ final Txn txn;
+
private CoordinateTransaction(Node node, TxnId txnId, Txn txn, FullRoute<?> route)
{
super(node, txnId, txn, route);
+ this.txn = txn;
}
public static AsyncResult<Result> coordinate(Node node, FullRoute<?> route, TxnId txnId, Txn txn)
@@ -67,7 +73,7 @@
if (tracker.hasFastPathAccepted())
{
Deps deps = Deps.merge(oks, ok -> ok.witnessedAt.equals(txnId) ? ok.deps : null);
- Execute.execute(node, topologies, route, FAST, txnId, txn, txnId, deps, this);
+ execute(executeAdapter(), node, topologies, route, FAST, txnId, txn, txnId, deps, settingCallback());
node.agent().metricsEventsListener().onFastPathTaken(txnId, deps);
}
else
@@ -87,10 +93,21 @@
if (PreAccept.rejectExecuteAt(txnId, topologies))
proposeAndCommitInvalidate(node, Ballot.ZERO, txnId, route.homeKey(), route, executeAt, this);
else
- proposeAndExecute(node, topologies, Ballot.ZERO, txnId, txn, route, executeAt, deps, this);
+ propose(proposeAdapter(), node, topologies, route, Ballot.ZERO, txnId, txn, executeAt, deps, this);
}
node.agent().metricsEventsListener().onSlowPathTaken(txnId, deps);
}
}
+
+ protected CoordinationAdapter<Result> proposeAdapter()
+ {
+ return Adapters.standard();
+ }
+
+ // TODO (expected): override in C* rather than default to configurability here
+ protected CoordinationAdapter<Result> executeAdapter()
+ {
+ return node.coordinationAdapter(txnId, Continue);
+ }
}
diff --git a/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java b/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java
new file mode 100644
index 0000000..00d3460
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/CoordinationAdapter.java
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.coordinate;
+
+import java.util.function.BiConsumer;
+
+import javax.annotation.Nullable;
+
+import accord.api.Result;
+import accord.coordinate.ExecuteSyncPoint.ExecuteAtQuorum;
+import accord.local.Node;
+import accord.messages.Apply;
+import accord.primitives.Ballot;
+import accord.primitives.Deps;
+import accord.primitives.FullRoute;
+import accord.primitives.Participants;
+import accord.primitives.Ranges;
+import accord.primitives.Seekables;
+import accord.primitives.SyncPoint;
+import accord.primitives.Timestamp;
+import accord.primitives.Txn;
+import accord.primitives.TxnId;
+import accord.primitives.Writes;
+import accord.topology.Topologies;
+import accord.utils.Faults;
+
+import static accord.coordinate.ExecutePath.FAST;
+import static accord.coordinate.ExecutePath.SLOW;
+import static accord.messages.Apply.Kind.Maximal;
+import static accord.messages.Apply.Kind.Minimal;
+import static accord.primitives.Routable.Domain.Range;
+
+public interface CoordinationAdapter<R>
+{
+ interface Factory
+ {
+ enum Step { Continue, InitiateRecovery }
+ <R> CoordinationAdapter<R> get(TxnId txnId, Step step);
+ }
+
+ void propose(Node node, Topologies withUnsynced, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback);
+ void stabilise(Node node, Topologies coordinates, Topologies all, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback);
+ void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback);
+ void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super R, Throwable> callback);
+
+ class DefaultFactory implements Factory
+ {
+ @Override
+ public <R> CoordinationAdapter<R> get(TxnId txnId, Step step)
+ {
+ switch (step)
+ {
+ default: throw new AssertionError("Unhandled step: " + step);
+ case Continue: return (CoordinationAdapter<R>) Adapters.standard();
+ case InitiateRecovery: return (CoordinationAdapter<R>) Adapters.recovery();
+ }
+ }
+ }
+
+ /**
+ * Utility methods for correctly invoking the next phase of the state machine via a CoordinationAdapter.
+ * Simply ensures the topologies are correct before being passed to the instance method.
+ */
+ class Invoke
+ {
+ public static <R> void propose(CoordinationAdapter<R> adapter, Node node, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
+ {
+ propose(adapter, node, node.topology().withUnsyncedEpochs(route, txnId, executeAt), route, ballot, txnId, txn, executeAt, deps, callback);
+ }
+
+ public static <R> void propose(CoordinationAdapter<R> adapter, Node node, Topologies withUnsynced, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
+ {
+ adapter.propose(node, withUnsynced, route, ballot, txnId, txn, executeAt, deps, callback);
+ }
+
+ public static <R> void stabilise(CoordinationAdapter<R> adapter, Node node, Topologies any, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
+ {
+ Topologies coordinates = any.forEpochs(txnId.epoch(), txnId.epoch());
+ Topologies all;
+ if (txnId.epoch() == executeAt.epoch()) all = coordinates;
+ else if (any.currentEpoch() >= executeAt.epoch() && any.oldestEpoch() <= txnId.epoch()) all = any.forEpochs(txnId.epoch(), executeAt.epoch());
+ else all = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
+
+ adapter.stabilise(node, any, all, route, ballot, txnId, txn, executeAt, deps, callback);
+ }
+
+ public static <R> void execute(CoordinationAdapter<R> adapter, Node node, Topologies any, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
+ {
+ if (any.oldestEpoch() <= txnId.epoch() && any.currentEpoch() >= executeAt.epoch()) any = any.forEpochs(txnId.epoch(), executeAt.epoch());
+ else any = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
+ adapter.execute(node, any, route, path, txnId, txn, executeAt, deps, callback);
+ }
+
+ public static <R> void persist(CoordinationAdapter<R> adapter, Node node, Topologies any, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, @Nullable BiConsumer<? super R, Throwable> callback)
+ {
+ if (any.oldestEpoch() <= txnId.epoch() && any.currentEpoch() >= executeAt.epoch()) any = any.forEpochs(txnId.epoch(), executeAt.epoch());
+ else any = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
+ Topologies executes = any.forEpochs(executeAt.epoch(), executeAt.epoch());
+
+ adapter.persist(node, any, route, txnId, txn, executeAt, deps, writes, result, callback);
+ }
+
+ public static <R> void persist(CoordinationAdapter<R> adapter, Node node, FullRoute<?> route, Participants<?> sendTo, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, @Nullable BiConsumer<? super R, Throwable> callback)
+ {
+ Topologies all = node.topology().preciseEpochs(sendTo, txnId.epoch(), executeAt.epoch());
+ Topologies executes = all.forEpochs(executeAt.epoch(), executeAt.epoch());
+
+ adapter.persist(node, all, route, txnId, txn, executeAt, deps, writes, result, callback);
+ }
+ }
+
+ class Adapters
+ {
+ public static CoordinationAdapter<Result> standard()
+ {
+ return StandardTxnAdapter.INSTANCE;
+ }
+
+ // note that by default the recovery adapter is only used for the initial recovery decision - if e.g. propose is initiated
+ // then we revert back to standard adapter behaviour for later steps
+ public static CoordinationAdapter<Result> recovery()
+ {
+ return RecoveryTxnAdapter.INSTANCE;
+ }
+
+ public static <S extends Seekables<?, ?>> CoordinationAdapter<SyncPoint<S>> inclusiveSyncPoint()
+ {
+ return InclusiveSyncPointAdapter.INSTANCE;
+ }
+
+ public static <S extends Seekables<?, ?>> CoordinationAdapter<SyncPoint<S>> inclusiveSyncPointBlocking()
+ {
+ return InclusiveSyncPointBlockingAdapter.INSTANCE;
+ }
+
+ public static <S extends Seekables<?, ?>> CoordinationAdapter<SyncPoint<S>> exclusiveSyncPoint()
+ {
+ return ExclusiveSyncPointAdapter.INSTANCE;
+ }
+
+ public static abstract class AbstractTxnAdapter implements CoordinationAdapter<Result>
+ {
+ @Override
+ public void propose(Node node, Topologies withUnsynced, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
+ {
+ new ProposeTxn(node, withUnsynced, route, ballot, txnId, txn, executeAt, deps, callback).start();
+ }
+
+ @Override
+ public void stabilise(Node node, Topologies coordinates, Topologies all, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
+ {
+ if (Faults.TRANSACTION_INSTABILITY) execute(node, all, route, SLOW, txnId, txn, executeAt, deps, callback);
+ else new StabiliseTxn(node, coordinates, all, route, ballot, txnId, txn, executeAt, deps, callback).start();
+ }
+
+ @Override
+ public void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
+ {
+ if (txn.read().keys().isEmpty()) Invoke.persist(this, node, all, route, txnId, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null), callback);
+ else new ExecuteTxn(node, all, route, path, txnId, txn, txn.read().keys().toParticipants(), executeAt, deps, callback).start();
+ }
+ }
+
+ public static class StandardTxnAdapter extends AbstractTxnAdapter
+ {
+ public static final StandardTxnAdapter INSTANCE = new StandardTxnAdapter();
+ @Override
+ public void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super Result, Throwable> callback)
+ {
+ if (callback != null) callback.accept(result, null);
+ new PersistTxn(node, all, txnId, route, txn, executeAt, deps, writes, result)
+ .start(Apply.FACTORY, Minimal, all, writes, result);
+ }
+ }
+
+ public static class RecoveryTxnAdapter extends AbstractTxnAdapter
+ {
+ public static final RecoveryTxnAdapter INSTANCE = new RecoveryTxnAdapter();
+ @Override
+ public void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super Result, Throwable> callback)
+ {
+ if (callback != null) callback.accept(result, null);
+ new PersistTxn(node, all, txnId, route, txn, executeAt, deps, writes, result)
+ .start(Apply.FACTORY, Maximal, all, writes, result);
+ }
+ }
+
+ static abstract class AbstractSyncPointAdapter<S extends Seekables<?, ?>> implements CoordinationAdapter<SyncPoint<S>>
+ {
+ void invokeSuccess(Node node, FullRoute<?> route, TxnId txnId, Txn txn, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+ {
+ if (txn.keys().domain() == Range)
+ node.configService().reportEpochClosed((Ranges)txn.keys(), txnId.epoch());
+ callback.accept(new SyncPoint<>(txnId, deps, (S)txn.keys(), route), null);
+ }
+
+ @Override
+ public void propose(Node node, Topologies withUnsynced, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+ {
+ new ProposeSyncPoint<>(this, node, withUnsynced, route, ballot, txnId, txn, executeAt, deps, callback).start();
+ }
+
+ @Override
+ public void stabilise(Node node, Topologies coordinates, Topologies all, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+ {
+ new StabiliseSyncPoint<>(this, node, coordinates, all, route, ballot, txnId, txn, executeAt, deps, callback).start();
+ }
+
+ @Override
+ public void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+ {
+ Invoke.persist(this, node, all, route, txnId, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null), callback);
+ }
+
+ @Override
+ public void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+ {
+ invokeSuccess(node, route, txnId, txn, deps, callback);
+ new PersistTxn(node, all, txnId, route, txn, executeAt, deps, writes, result)
+ .start(Apply.FACTORY, Maximal, all, writes, result);
+ }
+ }
+
+ public static class ExclusiveSyncPointAdapter<S extends Seekables<?, ?>> extends AbstractSyncPointAdapter<S>
+ {
+ private static final ExclusiveSyncPointAdapter INSTANCE = new ExclusiveSyncPointAdapter();
+
+ @Override
+ public void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+ {
+ // TODO (required): remember and document why we don't use fast path for exclusive sync points
+ if (path == FAST)
+ {
+ Invoke.stabilise(this, node, all, route, Ballot.ZERO, txnId, txn, executeAt, deps, callback);
+ }
+ else
+ {
+ super.execute(node, all, route, path, txnId, txn, executeAt, deps, callback);
+ }
+ }
+ }
+
+ public static class InclusiveSyncPointAdapter<S extends Seekables<?, ?>> extends AbstractSyncPointAdapter<S>
+ {
+ private static final InclusiveSyncPointAdapter INSTANCE = new InclusiveSyncPointAdapter();
+ }
+
+ public static class InclusiveSyncPointBlockingAdapter<S extends Seekables<?, ?>> extends AbstractSyncPointAdapter<S>
+ {
+ private static final InclusiveSyncPointBlockingAdapter INSTANCE = new InclusiveSyncPointBlockingAdapter();
+
+ @Override
+ public void execute(Node node, Topologies all, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+ {
+ ExecuteAtQuorum<S> execute = new ExecuteAtQuorum<S>(node, all, new SyncPoint<>(txnId, deps, (S)txn.keys(), route));
+ execute.addCallback(callback);
+ execute.start();
+ }
+
+ @Override
+ public void persist(Node node, Topologies all, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+ {
+ throw new UnsupportedOperationException();
+ }
+ }
+ }
+
+}
diff --git a/accord-core/src/main/java/accord/coordinate/Execute.java b/accord-core/src/main/java/accord/coordinate/Execute.java
deleted file mode 100644
index dec581d..0000000
--- a/accord-core/src/main/java/accord/coordinate/Execute.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.coordinate;
-
-import java.util.function.BiConsumer;
-
-import accord.api.Result;
-import accord.local.Node;
-import accord.primitives.Deps;
-import accord.primitives.FullRoute;
-import accord.primitives.Participants;
-import accord.primitives.Seekables;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
-import accord.primitives.Writes;
-import accord.primitives.Txn.Kind;
-import accord.topology.Topologies;
-
-import static accord.utils.Invariants.checkArgument;
-
-public interface Execute
-{
- enum Path { FAST, SLOW, RECOVER }
-
- interface Factory
- {
- Execute create(Node node, Topologies topologies, Path path, TxnId txnId, Txn txn, FullRoute<?> route, Participants<?> readScope, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback);
- }
-
- void start();
-
- static void execute(Node node, Topologies anyTopologies, FullRoute<?> route, Path path, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
- {
- Seekables<?, ?> readKeys = txn.read().keys();
- Participants<?> readScope = readKeys.toParticipants();
- // Recovery calls execute and we would like execute to run BlockOnDeps because that will notify the agent
- // of the local barrier
- if (txn.kind() == Kind.SyncPoint)
- {
- checkArgument(txnId.equals(executeAt));
- BlockOnDeps.blockOnDeps(node, txnId, txn, route, deps, callback);
- }
- else
- {
- if (readKeys.isEmpty())
- {
- Result result = txn.result(txnId, executeAt, null);
- Writes writes = txn.execute(txnId, executeAt, null);
- anyTopologies = anyTopologies.forEpochs(executeAt.epoch(), executeAt.epoch());
- Persist.persist(node, anyTopologies, route, txnId, txn, executeAt, deps, writes, result, callback);
- }
- else
- {
- if (anyTopologies.oldestEpoch() <= txnId.epoch() && anyTopologies.currentEpoch() >= executeAt.epoch()) anyTopologies = anyTopologies.forEpochs(txnId.epoch(), executeAt.epoch());
- else anyTopologies = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
-
- Execute execute = node.executionFactory().create(node, anyTopologies, path, txnId, txn, route, readScope, executeAt, deps, callback);
- execute.start();
- }
- }
- }
-}
diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java b/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java
index 07f22ef..81f7f6c 100644
--- a/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java
+++ b/accord-core/src/main/java/accord/coordinate/ExecuteEphemeralRead.java
@@ -112,6 +112,7 @@
default: throw new IllegalStateException();
case Redundant:
case Rejected:
+ // TODO (expected): shouldn't be preemptible (can be made redundant, but should be a special case)
callback.accept(null, new Preempted(txnId, route.homeKey()));
return Action.Aborted;
case Insufficient:
diff --git a/accord-core/src/main/java/accord/impl/DomainTimestamps.java b/accord-core/src/main/java/accord/coordinate/ExecutePath.java
similarity index 87%
rename from accord-core/src/main/java/accord/impl/DomainTimestamps.java
rename to accord-core/src/main/java/accord/coordinate/ExecutePath.java
index 2d614c8..15bce6b 100644
--- a/accord-core/src/main/java/accord/impl/DomainTimestamps.java
+++ b/accord-core/src/main/java/accord/coordinate/ExecutePath.java
@@ -16,11 +16,9 @@
* limitations under the License.
*/
-package accord.impl;
+package accord.coordinate;
-import accord.primitives.Timestamp;
-
-public interface DomainTimestamps
+public enum ExecutePath
{
- Timestamp max();
-}
\ No newline at end of file
+ FAST, SLOW, RECOVER
+}
diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java b/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java
new file mode 100644
index 0000000..713dd54
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/ExecuteSyncPoint.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.coordinate;
+
+import accord.api.Result;
+import accord.coordinate.tracking.AbstractSimpleTracker;
+import accord.coordinate.tracking.QuorumTracker;
+import accord.coordinate.tracking.RequestStatus;
+import accord.local.Node;
+import accord.messages.ApplyThenWaitUntilApplied;
+import accord.messages.Callback;
+import accord.messages.ReadData;
+import accord.messages.ReadData.ReadReply;
+import accord.primitives.Participants;
+import accord.primitives.Seekables;
+import accord.primitives.SyncPoint;
+import accord.primitives.Txn;
+import accord.primitives.Writes;
+import accord.topology.Topologies;
+import accord.utils.async.AsyncResults.SettableResult;
+
+public abstract class ExecuteSyncPoint<S extends Seekables<?, ?>> extends SettableResult<SyncPoint<S>> implements Callback<ReadReply>
+{
+ public static class SyncPointErased extends Throwable {}
+
+ public static class ExecuteAtQuorum<S extends Seekables<?, ?>> extends ExecuteSyncPoint<S>
+ {
+ ExecuteAtQuorum(Node node, Topologies topologies, SyncPoint<S> syncPoint)
+ {
+ super(node, new QuorumTracker(topologies), syncPoint);
+ }
+
+ @Override
+ protected void start()
+ {
+ Txn txn = node.agent().emptyTxn(syncPoint.syncId.kind(), syncPoint.keysOrRanges);
+ Writes writes = txn.execute(syncPoint.syncId, syncPoint.syncId, null);
+ Result result = txn.result(syncPoint.syncId, syncPoint.syncId, null);
+ node.send(tracker.topologies().nodes(), to -> {
+ Seekables<?, ?> notify = to.equals(node.id()) ? null : syncPoint.keysOrRanges;
+ Participants<?> participants = syncPoint.keysOrRanges.toParticipants();
+ return new ApplyThenWaitUntilApplied(to, tracker.topologies(), syncPoint.route(), syncPoint.syncId, txn, syncPoint.waitFor, participants, syncPoint.syncId.epoch(), writes, result, notify);
+ }, this);
+ }
+ }
+
+ final Node node;
+ final AbstractSimpleTracker<?> tracker;
+ final SyncPoint<S> syncPoint;
+
+ ExecuteSyncPoint(Node node, AbstractSimpleTracker<?> tracker, SyncPoint<S> syncPoint)
+ {
+ // TODO (required): this isn't correct, we need to potentially perform a second round if a dependency executes in a future epoch and we have lost ownership of that epoch
+ this.node = node;
+ this.tracker = tracker;
+ this.syncPoint = syncPoint;
+ }
+
+ protected abstract void start();
+
+ @Override
+ public synchronized void onSuccess(Node.Id from, ReadReply reply)
+ {
+ if (isDone()) return;
+
+ if (!reply.isOk())
+ {
+ switch ((ReadData.CommitOrReadNack)reply)
+ {
+ default: throw new AssertionError("Unhandled: " + reply);
+
+ case Insufficient:
+ CoordinateSyncPoint.sendApply(node, from, syncPoint);
+ return;
+
+ case Redundant:
+ tryFailure(new SyncPointErased());
+ return;
+
+ case Invalid:
+ tryFailure(new Invalidated(syncPoint.syncId, syncPoint.homeKey));
+ return;
+ }
+ }
+ else
+ {
+ // TODO (required): we also need to handle ranges not being safe to read
+ if (tracker.recordSuccess(from) == RequestStatus.Success)
+ onSuccess();
+ }
+ }
+
+ protected void onSuccess()
+ {
+ trySuccess(syncPoint);
+ }
+
+ @Override
+ public synchronized void onFailure(Node.Id from, Throwable failure)
+ {
+ if (isDone()) return;
+ if (tracker.recordFailure(from) == RequestStatus.Failed)
+ tryFailure(new Exhausted(syncPoint.syncId, syncPoint.homeKey));
+ }
+
+ @Override
+ public void onCallbackFailure(Node.Id from, Throwable failure)
+ {
+ tryFailure(failure);
+ }
+}
diff --git a/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java b/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java
index e46a0ff..be2623d 100644
--- a/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java
+++ b/accord-core/src/main/java/accord/coordinate/ExecuteTxn.java
@@ -42,6 +42,7 @@
import accord.topology.Topologies;
import org.agrona.collections.IntHashSet;
+import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue;
import static accord.coordinate.ReadCoordinator.Action.Approve;
import static accord.coordinate.ReadCoordinator.Action.ApprovePartial;
import static accord.messages.Commit.Kind.StableFastPath;
@@ -49,14 +50,12 @@
import static accord.messages.Commit.Kind.StableWithTxnAndDeps;
import static accord.utils.Invariants.illegalState;
-public class ExecuteTxn extends ReadCoordinator<ReadReply> implements Execute
+public class ExecuteTxn extends ReadCoordinator<ReadReply>
{
- public static final Execute.Factory FACTORY = ExecuteTxn::new;
-
@SuppressWarnings("unused")
private static final Logger logger = LoggerFactory.getLogger(ExecuteTxn.class);
- final Path path;
+ final ExecutePath path;
final Txn txn;
final Participants<?> readScope;
final FullRoute<?> route;
@@ -66,7 +65,7 @@
final BiConsumer<? super Result, Throwable> callback;
private Data data;
- private ExecuteTxn(Node node, Topologies topologies, Path path, TxnId txnId, Txn txn, FullRoute<?> route, Participants<?> readScope, Timestamp executeAt, Deps stableDeps, BiConsumer<? super Result, Throwable> callback)
+ ExecuteTxn(Node node, Topologies topologies, FullRoute<?> route, ExecutePath path, TxnId txnId, Txn txn, Participants<?> readScope, Timestamp executeAt, Deps stableDeps, BiConsumer<? super Result, Throwable> callback)
{
// we need to send Stable to the origin epoch as well as the execution epoch
// TODO (desired): permit slicing Topologies by key (though unnecessary if we eliminate the concept of non-participating home keys)
@@ -148,11 +147,16 @@
if (failure == null)
{
Result result = txn.result(txnId, executeAt, data);
- Persist.persist(node, allTopologies, route, txnId, txn, executeAt, stableDeps, txn.execute(txnId, executeAt, data), result, callback);
+ CoordinationAdapter.Invoke.persist(adapter(), node, allTopologies, route, txnId, txn, executeAt, stableDeps, txn.execute(txnId, executeAt, data), result, callback);
}
else
{
callback.accept(null, failure);
}
}
+
+ protected CoordinationAdapter<Result> adapter()
+ {
+ return node.coordinationAdapter(txnId, Continue);
+ }
}
diff --git a/accord-core/src/main/java/accord/coordinate/FetchData.java b/accord-core/src/main/java/accord/coordinate/FetchData.java
index 1ece467..b7d9cae 100644
--- a/accord-core/src/main/java/accord/coordinate/FetchData.java
+++ b/accord-core/src/main/java/accord/coordinate/FetchData.java
@@ -216,7 +216,10 @@
else
{
if (success == ReadCoordinator.Success.Success)
- Invariants.checkState(isSufficient(merged), "Status %s is not sufficient", merged);
+ {
+ if (!isSufficient(merged))
+ Invariants.checkState(isSufficient(merged), "Status %s is not sufficient", merged);
+ }
// TODO (expected): should we automatically trigger a new fetch if we find executeAt but did not request enough information? would be more rob ust
Propagate.propagate(node, txnId, sourceEpoch, success.withQuorum, route(), target, (CheckStatusOkFull) merged, callback);
diff --git a/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java b/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java
new file mode 100644
index 0000000..0793ae7
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/FetchMaxConflict.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.coordinate;
+
+import java.util.Set;
+
+import accord.coordinate.tracking.QuorumTracker;
+import accord.local.Node;
+import accord.messages.Callback;
+import accord.messages.GetMaxConflict;
+import accord.messages.GetMaxConflict.GetMaxConflictOk;
+import accord.primitives.FullRoute;
+import accord.primitives.Seekables;
+import accord.primitives.Timestamp;
+import accord.topology.Topologies;
+import accord.utils.async.AsyncResult;
+import accord.utils.async.AsyncResults;
+
+import static accord.coordinate.tracking.RequestStatus.Failed;
+import static accord.coordinate.tracking.RequestStatus.Success;
+
+/**
+ * Perform initial rounds of PreAccept and Accept until we have reached agreement about when we should execute.
+ * If we are preempted by a recovery coordinator, we abort and let them complete (and notify us about the execution result)
+ *
+ * TODO (desired, testing): dedicated burn test to validate outcomes
+ */
+public class FetchMaxConflict extends AbstractCoordinatePreAccept<Timestamp, GetMaxConflictOk>
+{
+ final QuorumTracker tracker;
+ final Seekables<?, ?> keysOrRanges;
+ Timestamp maxConflict;
+ long executionEpoch;
+
+ private FetchMaxConflict(Node node, FullRoute<?> route, Seekables<?, ?> keysOrRanges, long executionEpoch)
+ {
+ this(node, route, keysOrRanges, executionEpoch, node.topology().withUnsyncedEpochs(route, executionEpoch, executionEpoch));
+ }
+
+ private FetchMaxConflict(Node node, FullRoute<?> route, Seekables<?, ?> keysOrRanges, long executionEpoch, Topologies topologies)
+ {
+ super(node, route, null, topologies);
+ this.keysOrRanges = keysOrRanges;
+ this.maxConflict = Timestamp.NONE;
+ this.executionEpoch = executionEpoch;
+ this.tracker = new QuorumTracker(topologies);
+ }
+
+ public static AsyncResult<Timestamp> fetchMaxConflict(Node node, Seekables<?, ?> keysOrRanges)
+ {
+ long epoch = node.epoch();
+ FullRoute<?> route = node.computeRoute(epoch, keysOrRanges);
+ // TODO (required): need to ensure we permanently fail any bootstrap that is now impossible and mark as stale
+ TopologyMismatch mismatch = TopologyMismatch.checkForMismatch(node.topology().globalForEpoch(epoch), null, route.homeKey(), keysOrRanges);
+ if (mismatch != null)
+ return AsyncResults.failure(mismatch);
+ FetchMaxConflict coordinate = new FetchMaxConflict(node, route, keysOrRanges, epoch);
+ coordinate.start();
+ return coordinate;
+ }
+
+
+ @Override
+ Seekables<?, ?> keysOrRanges()
+ {
+ return keysOrRanges;
+ }
+
+ @Override
+ void contact(Set<Node.Id> nodes, Topologies topologies, Callback<GetMaxConflictOk> callback)
+ {
+ node.send(nodes, to -> new GetMaxConflict(to, topologies, route, keysOrRanges, executionEpoch));
+ }
+
+ @Override
+ void onSuccessInternal(Node.Id from, GetMaxConflictOk reply)
+ {
+ maxConflict = Timestamp.max(reply.maxConflict, maxConflict);
+ executionEpoch = Math.max(executionEpoch, reply.latestEpoch);
+
+ if (tracker.recordSuccess(from) == Success)
+ onPreAcceptedOrNewEpoch();
+ }
+
+ @Override
+ boolean onExtraSuccessInternal(Node.Id from, GetMaxConflictOk reply)
+ {
+ maxConflict = Timestamp.max(reply.maxConflict, maxConflict);
+ return true;
+ }
+
+ @Override
+ void onFailureInternal(Node.Id from, Throwable failure)
+ {
+ if (tracker.recordFailure(from) == Failed)
+ tryFailure(failure);
+ }
+
+ @Override
+ void onNewEpochTopologyMismatch(TopologyMismatch mismatch)
+ {
+ tryFailure(mismatch);
+ }
+
+ @Override
+ long executeAtEpoch()
+ {
+ return executionEpoch;
+ }
+
+ @Override
+ void onPreAccepted(Topologies topologies)
+ {
+ setSuccess(maxConflict);
+ }
+}
diff --git a/accord-core/src/main/java/accord/coordinate/Infer.java b/accord-core/src/main/java/accord/coordinate/Infer.java
index 7b1956a..f1b8016 100644
--- a/accord-core/src/main/java/accord/coordinate/Infer.java
+++ b/accord-core/src/main/java/accord/coordinate/Infer.java
@@ -295,7 +295,7 @@
}
}
- public static InvalidIfNot invalidIfNotAtLeast(SafeCommandStore safeStore, TxnId txnId, Unseekables<?> query)
+ public static InvalidIfNot invalidIfNot(SafeCommandStore safeStore, TxnId txnId, Unseekables<?> query)
{
if (safeStore.commandStore().globalDurability(txnId).compareTo(Majority) >= 0)
{
diff --git a/accord-core/src/main/java/accord/coordinate/Invalidate.java b/accord-core/src/main/java/accord/coordinate/Invalidate.java
index d71b8fc..1a409b8 100644
--- a/accord-core/src/main/java/accord/coordinate/Invalidate.java
+++ b/accord-core/src/main/java/accord/coordinate/Invalidate.java
@@ -260,7 +260,6 @@
{
// TODO (desired): merge with FetchData.InvalidateOnDone
// TODO (desired): when sending to network, register a callback for when local application of commitInvalidate message ahs been performed, so no need to special-case
- // TODO (required, consider): pick a reasonable upper bound, so we don't invalidate into an epoch/commandStore that no longer cares about this command
node.forEachLocal(contextFor(txnId), commitTo, txnId.epoch(), txnId.epoch(), safeStore -> {
Commands.commitInvalidate(safeStore, safeStore.get(txnId, txnId, commitTo), commitTo);
}).begin((s, f) -> {
diff --git a/accord-core/src/main/java/accord/coordinate/Persist.java b/accord-core/src/main/java/accord/coordinate/Persist.java
index 47e370a..e0e2c92 100644
--- a/accord-core/src/main/java/accord/coordinate/Persist.java
+++ b/accord-core/src/main/java/accord/coordinate/Persist.java
@@ -20,7 +20,6 @@
import java.util.HashSet;
import java.util.Set;
-import java.util.function.BiConsumer;
import accord.api.Result;
import accord.coordinate.tracking.QuorumTracker;
@@ -32,26 +31,17 @@
import accord.messages.InformDurable;
import accord.primitives.Deps;
import accord.primitives.FullRoute;
-import accord.primitives.PartialTxn;
import accord.primitives.Timestamp;
import accord.primitives.Txn;
import accord.primitives.TxnId;
-import accord.primitives.Unseekables;
import accord.primitives.Writes;
import accord.topology.Topologies;
import static accord.coordinate.tracking.RequestStatus.Success;
import static accord.local.Status.Durability.Majority;
-import static accord.messages.Apply.executes;
-import static accord.messages.Apply.participates;
public abstract class Persist implements Callback<ApplyReply>
{
- public interface Factory
- {
- Persist create(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result);
- }
-
protected final Node node;
protected final TxnId txnId;
protected final FullRoute<?> route;
@@ -65,36 +55,7 @@
protected final Set<Id> persistedOn;
boolean isDone;
- public static void persist(Node node, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
- {
- Topologies executes = executes(node, route, executeAt);
- persist(node, executes, route, txnId, txn, executeAt, stableDeps, writes, result, clientCallback);
- }
-
- public static void persist(Node node, Topologies executes, FullRoute<?> route, TxnId txnId, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
- {
- Topologies participates = participates(node, route, txnId, executeAt, executes);
- node.persistFactory().create(node, executes, txnId, route, txn, executeAt, stableDeps, writes, result)
- .applyMinimal(participates, executes, writes, result, clientCallback);
- }
-
- public static void persistMaximal(Node node, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
- {
- Topologies executes = executes(node, route, executeAt);
- Topologies participates = participates(node, route, txnId, executeAt, executes);
- node.persistFactory().create(node, participates, txnId, route, txn, executeAt, deps, writes, result)
- .applyMaximal(participates, executes, writes, result, null);
- }
-
- public static void persistPartialMaximal(Node node, TxnId txnId, Unseekables<?> sendTo, FullRoute<?> route, PartialTxn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
- {
- Topologies executes = executes(node, sendTo, executeAt);
- Topologies participates = participates(node, sendTo, txnId, executeAt, executes);
- Persist persist = node.persistFactory().create(node, participates, txnId, route, txn, executeAt, deps, writes, result);
- node.send(participates.nodes(), to -> Apply.applyMaximal(Apply.FACTORY, to, participates, executes, txnId, route, txn, executeAt, deps, writes, result), persist);
- }
-
- protected Persist(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
+ protected Persist(Node node, Topologies all, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
{
this.node = node;
this.txnId = txnId;
@@ -104,8 +65,8 @@
this.stableDeps = stableDeps;
this.writes = writes;
this.result = result;
- this.topologies = topologies;
- this.tracker = new QuorumTracker(topologies);
+ this.topologies = all;
+ this.tracker = new QuorumTracker(all);
this.persistedOn = new HashSet<>();
}
@@ -142,20 +103,13 @@
@Override
public void onCallbackFailure(Id from, Throwable failure)
{
+ // TODO (required): handle exception
}
- public void applyMinimal(Topologies participates, Topologies executes, Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
+ public void start(Apply.Factory factory, Apply.Kind kind, Topologies all, Writes writes, Result result)
{
- registerClientCallback(writes, result, clientCallback);
// applyMinimal is used for transaction execution by the original coordinator so it's important to use
// Node's Apply factory in case the factory has to do synchronous Apply.
- node.send(participates.nodes(), to -> Apply.applyMinimal(node.applyFactory(), to, participates, executes, txnId, route, txn, executeAt, stableDeps, writes, result), this);
+ node.send(all.nodes(), to -> factory.create(kind, to, all, txnId, route, txn, executeAt, stableDeps, writes, result), this);
}
- public void applyMaximal(Topologies participates, Topologies executes, Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
- {
- registerClientCallback(writes, result, clientCallback);
- node.send(participates.nodes(), to -> Apply.applyMaximal(Apply.FACTORY, to, participates, executes, txnId, route, txn, executeAt, stableDeps, writes, result), this);
- }
-
- public abstract void registerClientCallback(Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback);
}
diff --git a/accord-core/src/main/java/accord/coordinate/PersistTxn.java b/accord-core/src/main/java/accord/coordinate/PersistTxn.java
index c55777a..b2db464 100644
--- a/accord-core/src/main/java/accord/coordinate/PersistTxn.java
+++ b/accord-core/src/main/java/accord/coordinate/PersistTxn.java
@@ -18,8 +18,6 @@
package accord.coordinate;
-import java.util.function.BiConsumer;
-
import accord.api.Result;
import accord.local.Node;
import accord.primitives.Deps;
@@ -32,17 +30,8 @@
public class PersistTxn extends Persist
{
- public static final Persist.Factory FACTORY = PersistTxn::new;
-
- private PersistTxn(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+ public PersistTxn(Node node, Topologies topologies, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
{
super(node, topologies, txnId, route, txn, executeAt, deps, writes, result);
}
-
- @Override
- public void registerClientCallback(Writes writes, Result result, BiConsumer<? super Result, Throwable> clientCallback)
- {
- if (clientCallback != null)
- clientCallback.accept(result, null);
- }
}
diff --git a/accord-core/src/main/java/accord/coordinate/Propose.java b/accord-core/src/main/java/accord/coordinate/Propose.java
index 0d27f75..b72408e 100644
--- a/accord-core/src/main/java/accord/coordinate/Propose.java
+++ b/accord-core/src/main/java/accord/coordinate/Propose.java
@@ -62,10 +62,10 @@
private final Map<Id, AcceptReply> debug = debug() ? new HashMap<>() : null;
final Timestamp executeAt;
final QuorumTracker acceptTracker;
- final BiConsumer<R, Throwable> callback;
+ final BiConsumer<? super R, Throwable> callback;
private boolean isDone;
- Propose(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Timestamp executeAt, Deps deps, BiConsumer<R, Throwable> callback)
+ Propose(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Timestamp executeAt, Deps deps, BiConsumer<? super R, Throwable> callback)
{
this.node = node;
this.ballot = ballot;
diff --git a/accord-core/src/main/java/accord/coordinate/ProposeAndExecute.java b/accord-core/src/main/java/accord/coordinate/ProposeAndExecute.java
deleted file mode 100644
index 2e7e464..0000000
--- a/accord-core/src/main/java/accord/coordinate/ProposeAndExecute.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.coordinate;
-
-import java.util.function.BiConsumer;
-
-import accord.api.Result;
-import accord.local.Node;
-import accord.primitives.Ballot;
-import accord.primitives.Deps;
-import accord.primitives.FullRoute;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn;
-import accord.primitives.TxnId;
-import accord.topology.Topologies;
-import accord.utils.Faults;
-
-class ProposeAndExecute extends Propose<Result>
-{
- ProposeAndExecute(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, Timestamp executeAt, BiConsumer<Result, Throwable> callback)
- {
- super(node, topologies, ballot, txnId, txn, route, executeAt, deps, callback);
- }
-
- public static void proposeAndExecute(Node node, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route,
- Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
- {
- Topologies topologies = node.topology().withUnsyncedEpochs(route, txnId, executeAt);
- proposeAndExecute(node, topologies, ballot, txnId, txn, route, executeAt, deps, callback);
- }
-
- public static void proposeAndExecute(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route,
- Timestamp executeAt, Deps deps, BiConsumer<Result, Throwable> callback)
- {
- ProposeAndExecute propose = new ProposeAndExecute(node, topologies, ballot, txnId, txn, route, deps, executeAt, callback);
- propose.start();
- }
-
- @Override
- void onAccepted()
- {
- Deps deps = this.deps;
- if (!Faults.TRANSACTION_UNMERGED_DEPS)
- deps = deps.with(Deps.merge(acceptOks, ok -> ok.deps));
-
- Stabilise.stabilise(node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback);
- }
-}
diff --git a/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java b/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java
index 1e8add5..6d3aab2 100644
--- a/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java
+++ b/accord-core/src/main/java/accord/coordinate/ProposeSyncPoint.java
@@ -18,19 +18,15 @@
package accord.coordinate;
-import java.util.Set;
import java.util.function.BiConsumer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import accord.local.Node;
-import accord.local.Node.Id;
-import accord.messages.Apply;
import accord.primitives.Ballot;
import accord.primitives.Deps;
import accord.primitives.FullRoute;
-import accord.primitives.Ranges;
import accord.primitives.Seekables;
import accord.primitives.SyncPoint;
import accord.primitives.Timestamp;
@@ -39,50 +35,24 @@
import accord.topology.Topologies;
import accord.utils.Faults;
-import static accord.primitives.Txn.Kind.ExclusiveSyncPoint;
+import static accord.coordinate.CoordinationAdapter.Invoke.stabilise;
public class ProposeSyncPoint<S extends Seekables<?, ?>> extends Propose<SyncPoint<S>>
{
@SuppressWarnings("unused")
private static final Logger logger = LoggerFactory.getLogger(ProposeSyncPoint.class);
+ private final CoordinationAdapter<SyncPoint<S>> adapter;
- // Whether to wait on the dependencies applying globally before returning a result
- final boolean async;
- final Set<Id> fastPathNodes;
-
- final S keysOrRanges;
-
- ProposeSyncPoint(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, Timestamp executeAt, BiConsumer<SyncPoint<S>, Throwable> callback, boolean async, Set<Id> fastPathNodes, S keysOrRanges)
+ ProposeSyncPoint(CoordinationAdapter<SyncPoint<S>> adapter, Node node, Topologies topologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
{
super(node, topologies, ballot, txnId, txn, route, executeAt, deps, callback);
- this.async = async;
- this.fastPathNodes = fastPathNodes;
- this.keysOrRanges = keysOrRanges;
- }
-
- public static <S extends Seekables<?, ?>> Propose<SyncPoint<S>> proposeSyncPoint(Node node, Topologies topologies, Ballot ballot, TxnId txnId, Txn txn, FullRoute<?> route, Deps deps, BiConsumer<SyncPoint<S>, Throwable> callback, boolean async, Set<Id> fastPathNodes, S keysOrRanges)
- {
- ProposeSyncPoint proposeSyncPoint = new ProposeSyncPoint(node, topologies, ballot, txnId, txn, route, deps, txnId, callback, async, fastPathNodes, keysOrRanges);
- proposeSyncPoint.start();
- return proposeSyncPoint;
+ this.adapter = adapter;
}
@Override
void onAccepted()
{
- Deps deps = this.deps;
- if (!Faults.SYNCPOINT_UNMERGED_DEPS)
- deps = deps.with(Deps.merge(acceptOks, ok -> ok.deps));
-
- if (txnId.kind() == ExclusiveSyncPoint)
- {
- Apply.sendMaximal(node, txnId, route, txn, executeAt, deps, txn.execute(txnId, executeAt, null), txn.result(txnId, executeAt, null));
- node.configService().reportEpochClosed((Ranges)keysOrRanges, txnId.epoch());
- callback.accept(new SyncPoint<S>(txnId, deps, keysOrRanges, route, true), null);
- }
- else
- {
- CoordinateSyncPoint.blockOnDeps(node, txn, txnId, route, keysOrRanges, deps, callback, async);
- }
+ Deps deps = Faults.SYNCPOINT_UNMERGED_DEPS ? this.deps : this.deps.with(Deps.merge(acceptOks, ok -> ok.deps));
+ stabilise(adapter, node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback);
}
}
diff --git a/accord-core/src/main/java/accord/coordinate/ProposeTxn.java b/accord-core/src/main/java/accord/coordinate/ProposeTxn.java
new file mode 100644
index 0000000..075839d
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/ProposeTxn.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.coordinate;
+
+import java.util.function.BiConsumer;
+
+import accord.api.Result;
+import accord.local.Node;
+import accord.primitives.Ballot;
+import accord.primitives.Deps;
+import accord.primitives.FullRoute;
+import accord.primitives.Timestamp;
+import accord.primitives.Txn;
+import accord.primitives.TxnId;
+import accord.topology.Topologies;
+import accord.utils.Faults;
+
+import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue;
+import static accord.coordinate.CoordinationAdapter.Invoke.stabilise;
+
+class ProposeTxn extends Propose<Result>
+{
+ ProposeTxn(Node node, Topologies topologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
+ {
+ super(node, topologies, ballot, txnId, txn, route, executeAt, deps, callback);
+ }
+
+ @Override
+ void onAccepted()
+ {
+ Deps deps = this.deps;
+ if (!Faults.TRANSACTION_UNMERGED_DEPS)
+ deps = deps.with(Deps.merge(acceptOks, ok -> ok.deps));
+
+ stabilise(adapter(), node, acceptTracker.topologies(), route, ballot, txnId, txn, executeAt, deps, callback);
+ }
+
+ protected CoordinationAdapter<Result> adapter()
+ {
+ return node.coordinationAdapter(txnId, Continue);
+ }
+}
diff --git a/accord-core/src/main/java/accord/coordinate/Recover.java b/accord-core/src/main/java/accord/coordinate/Recover.java
index e523169..0455dfa 100644
--- a/accord-core/src/main/java/accord/coordinate/Recover.java
+++ b/accord-core/src/main/java/accord/coordinate/Recover.java
@@ -28,6 +28,7 @@
import java.util.function.Consumer;
import accord.api.Result;
+import accord.coordinate.CoordinationAdapter.Invoke;
import accord.coordinate.tracking.QuorumTracker;
import accord.coordinate.tracking.RecoveryTracker;
import accord.local.Node;
@@ -56,10 +57,13 @@
import accord.utils.async.AsyncResult;
import accord.utils.async.AsyncResults;
-import static accord.coordinate.Execute.Path.RECOVER;
+import static accord.coordinate.CoordinationAdapter.Factory.Step.InitiateRecovery;
+import static accord.coordinate.CoordinationAdapter.Invoke.execute;
+import static accord.coordinate.CoordinationAdapter.Invoke.persist;
+import static accord.coordinate.CoordinationAdapter.Invoke.stabilise;
+import static accord.coordinate.ExecutePath.RECOVER;
import static accord.coordinate.Infer.InvalidateAndCallback.locallyInvalidateAndCallback;
import static accord.coordinate.Propose.Invalidate.proposeInvalidate;
-import static accord.coordinate.ProposeAndExecute.proposeAndExecute;
import static accord.coordinate.tracking.RequestStatus.Failed;
import static accord.coordinate.tracking.RequestStatus.Success;
import static accord.messages.BeginRecovery.RecoverOk.maxAcceptedOrLater;
@@ -68,6 +72,7 @@
// TODO (low priority, cleanup): rename to Recover (verb); rename Recover message to not clash
// TODO (expected): do not recover transactions that are known to be Stable and waiting to execute.
+// TODO (expected): separate out recovery of sync points from standard transactions
public class Recover implements Callback<RecoverReply>, BiConsumer<Result, Throwable>
{
class AwaitCommit extends AsyncResults.SettableResult<Timestamp> implements Callback<WaitOnCommitOk>
@@ -128,6 +133,7 @@
return result;
}
+ private final CoordinationAdapter<Result> adapter;
private final Node node;
private final Ballot ballot;
private final TxnId txnId;
@@ -147,6 +153,7 @@
// TODO (required, correctness): we may have to contact all epochs to ensure we spot any future transaction that might not have taken us as dependency?
// or we need an exclusive sync point covering us and closing out the old epoch before recovering;
// or we need to manage dependencies for ranges we don't own in future epochs; this might be simplest
+ this.adapter = node.coordinationAdapter(txnId, InitiateRecovery);
this.node = node;
this.ballot = ballot;
this.txnId = txnId;
@@ -257,7 +264,7 @@
{
withCommittedDeps(executeAt, stableDeps -> {
// TODO (future development correctness): when writes/result are partially replicated, need to confirm we have quorum of these
- Persist.persistMaximal(node, txnId, route, txn, executeAt, stableDeps, acceptOrCommit.writes, acceptOrCommit.result);
+ persist(adapter, node, tracker.topologies(), route, txnId, txn, executeAt, stableDeps, acceptOrCommit.writes, acceptOrCommit.result, null);
});
accept(acceptOrCommit.result, null);
return;
@@ -267,7 +274,7 @@
case Stable:
{
withCommittedDeps(executeAt, stableDeps -> {
- Execute.execute(node, tracker.topologies(), route, RECOVER, txnId, txn, executeAt, stableDeps, this);
+ execute(adapter, node, tracker.topologies(), route, RECOVER, txnId, txn, executeAt, stableDeps, this);
});
return;
}
@@ -276,7 +283,7 @@
case Committed:
{
withCommittedDeps(executeAt, committedDeps -> {
- Stabilise.stabilise(node, tracker.topologies(), route, ballot, txnId, txn, executeAt, committedDeps, this);
+ stabilise(adapter, node, tracker.topologies(), route, ballot, txnId, txn, executeAt, committedDeps, this);
});
return;
}
@@ -320,7 +327,6 @@
// 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 finish committing before retrying recovery.
- // TODO (required): check paper: do we assume that witnessing in PreAccept implies witnessing in Accept? Not guaranteed.
// See whitepaper for more details
awaitCommits(node, earlierAcceptedNoWitness).addCallback((success, failure) -> {
if (failure != null) accept(null, failure);
@@ -373,7 +379,7 @@
private void propose(Timestamp executeAt, Deps deps)
{
- node.withEpoch(executeAt.epoch(), () -> proposeAndExecute(node, ballot, txnId, txn, route, executeAt, deps, this));
+ node.withEpoch(executeAt.epoch(), () -> Invoke.propose(adapter, node, route, ballot, txnId, txn, executeAt, deps, this));
}
private void retry()
diff --git a/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java b/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java
index 257893a..6987163 100644
--- a/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java
+++ b/accord-core/src/main/java/accord/coordinate/RecoverWithRoute.java
@@ -42,6 +42,8 @@
import accord.utils.Invariants;
import javax.annotation.Nullable;
+import static accord.coordinate.CoordinationAdapter.Factory.Step.InitiateRecovery;
+import static accord.coordinate.CoordinationAdapter.Invoke.persist;
import static accord.local.Status.Durability.Majority;
import static accord.local.Status.KnownDeps.DepsKnown;
import static accord.local.Status.KnownExecuteAt.ExecuteAtKnown;
@@ -186,7 +188,7 @@
{
Invariants.checkState(full.stableDeps.covering.containsAll(sendTo));
Invariants.checkState(full.partialTxn.covering().containsAll(sendTo));
- Persist.persistPartialMaximal(node, txnId, sendTo, route, full.partialTxn, full.executeAt, full.stableDeps, full.writes, full.result);
+ persist(node.coordinationAdapter(txnId, InitiateRecovery), node, route, sendTo, txnId, full.partialTxn, full.executeAt, full.stableDeps, full.writes, full.result, null);
}
propagate = full;
}
@@ -205,13 +207,12 @@
break;
}
- // TODO (required): might not be able to fully recover transaction - may only have enough for local shard
Txn txn = full.partialTxn.reconstitute(route);
if (known.executeAt.isDecidedAndKnownToExecute() && known.deps.hasDecidedDeps() && known.outcome == Apply)
{
Deps deps = full.stableDeps.reconstitute(route());
node.withEpoch(full.executeAt.epoch(), () -> {
- Persist.persistMaximal(node, txnId, route(), txn, full.executeAt, deps, full.writes, full.result);
+ persist(node.coordinationAdapter(txnId, InitiateRecovery), node, topologies, route(), txnId, txn, full.executeAt, deps, full.writes, full.result, null);
});
callback.accept(APPLIED, null);
}
diff --git a/accord-core/src/main/java/accord/coordinate/Stabilise.java b/accord-core/src/main/java/accord/coordinate/Stabilise.java
index e48ee7d..d204fab 100644
--- a/accord-core/src/main/java/accord/coordinate/Stabilise.java
+++ b/accord-core/src/main/java/accord/coordinate/Stabilise.java
@@ -22,7 +22,6 @@
import java.util.Map;
import java.util.function.BiConsumer;
-import accord.api.Result;
import accord.coordinate.tracking.QuorumTracker;
import accord.coordinate.tracking.RequestStatus;
import accord.local.Node;
@@ -38,14 +37,14 @@
import accord.primitives.Txn;
import accord.primitives.TxnId;
import accord.topology.Topologies;
-import accord.utils.Faults;
-import static accord.coordinate.Execute.Path.SLOW;
+import static accord.coordinate.CoordinationAdapter.Invoke.execute;
+import static accord.coordinate.ExecutePath.SLOW;
import static accord.coordinate.tracking.RequestStatus.Failed;
import static accord.messages.Commit.Kind.CommitWithTxn;
import static accord.utils.Invariants.debug;
-public abstract class Stabilise implements Callback<ReadReply>
+public abstract class Stabilise<R> implements Callback<ReadReply>
{
final Node node;
final Txn txn;
@@ -58,10 +57,10 @@
private final Map<Node.Id, Object> debug = debug() ? new HashMap<>() : null;
final QuorumTracker stableTracker;
final Topologies allTopologies;
- final BiConsumer<? super Result, Throwable> callback;
+ final BiConsumer<? super R, Throwable> callback;
private boolean isDone;
- public Stabilise(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, TxnId txnId, Ballot ballot, Txn txn, Timestamp executeAt, Deps stabiliseDeps, BiConsumer<? super Result, Throwable> callback)
+ public Stabilise(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, TxnId txnId, Ballot ballot, Txn txn, Timestamp executeAt, Deps stabiliseDeps, BiConsumer<? super R, Throwable> callback)
{
this.node = node;
this.txn = txn;
@@ -76,40 +75,6 @@
this.callback = callback;
}
- static void stabilise(Node node, Topologies anyTopologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
- {
- Topologies coordinates = anyTopologies.forEpochs(txnId.epoch(), txnId.epoch());
- Topologies allTopologies;
- if (txnId.epoch() == executeAt.epoch()) allTopologies = coordinates;
- else if (anyTopologies.currentEpoch() >= executeAt.epoch() && anyTopologies.oldestEpoch() <= txnId.epoch()) allTopologies = anyTopologies.forEpochs(txnId.epoch(), executeAt.epoch());
- else allTopologies = node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
-
- stabilise(node, coordinates, allTopologies, route, ballot, txnId, txn, executeAt, deps, callback);
- }
-
- public static void stabilise(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps deps, BiConsumer<? super Result, Throwable> callback)
- {
- switch (txnId.kind())
- {
- default:
- throw new AssertionError("Unhandled Txn.Kind: " + txnId.kind());
- case LocalOnly:
- throw new AssertionError("Invalid Txn.Kind to stabilise: " + txnId.kind());
- case SyncPoint:
- case ExclusiveSyncPoint:
- // TODO (expected): merge with branch below, as identical besides fault condition
- if (Faults.SYNCPOINT_INSTABILITY) Execute.execute(node, allTopologies, route, SLOW, txnId, txn, executeAt, deps, callback);
- else new StabiliseTxn(node, coordinates, allTopologies, route, txnId, ballot, txn, executeAt, deps, callback).start();
- break;
- case NoOp:
- case Read:
- case Write:
- if (Faults.TRANSACTION_INSTABILITY) Execute.execute(node, allTopologies, route, SLOW, txnId, txn, executeAt, deps, callback);
- else new StabiliseTxn(node, coordinates, allTopologies, route,
- txnId, ballot, txn, executeAt, deps, callback).start();
- }
- }
-
void start()
{
Commit.commitMinimal(node, stableTracker.topologies(), ballot, txnId, txn, route, executeAt, stabiliseDeps, this);
@@ -175,5 +140,10 @@
callback.accept(null, failure);
}
- abstract void onStabilised();
+ protected void onStabilised()
+ {
+ execute(adapter(), node, allTopologies, route, SLOW, txnId, txn, executeAt, stabiliseDeps, callback);
+ }
+
+ protected abstract CoordinationAdapter<R> adapter();
}
diff --git a/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java b/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java
new file mode 100644
index 0000000..e4936d2
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/StabiliseSyncPoint.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.coordinate;
+
+import java.util.function.BiConsumer;
+
+import accord.local.Node;
+import accord.primitives.Ballot;
+import accord.primitives.Deps;
+import accord.primitives.FullRoute;
+import accord.primitives.Seekables;
+import accord.primitives.SyncPoint;
+import accord.primitives.Timestamp;
+import accord.primitives.Txn;
+import accord.primitives.TxnId;
+import accord.topology.Topologies;
+
+public class StabiliseSyncPoint<S extends Seekables<?, ?>> extends Stabilise<SyncPoint<S>>
+{
+ final CoordinationAdapter<SyncPoint<S>> adapter;
+ StabiliseSyncPoint(CoordinationAdapter<SyncPoint<S>> adapter, Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer<? super SyncPoint<S>, Throwable> callback)
+ {
+ super(node, coordinates, allTopologies, route, txnId, ballot, txn, executeAt, unstableDeps, callback);
+ this.adapter = adapter;
+ }
+
+ @Override
+ protected CoordinationAdapter<SyncPoint<S>> adapter()
+ {
+ return adapter;
+ }
+}
diff --git a/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java b/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java
index edbfd1b..da36f65 100644
--- a/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java
+++ b/accord-core/src/main/java/accord/coordinate/StabiliseTxn.java
@@ -30,18 +30,18 @@
import accord.primitives.TxnId;
import accord.topology.Topologies;
-import static accord.coordinate.Execute.Path.SLOW;
+import static accord.coordinate.CoordinationAdapter.Factory.Step.Continue;
-public class StabiliseTxn extends Stabilise
+public class StabiliseTxn extends Stabilise<Result>
{
- public StabiliseTxn(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, TxnId txnId, Ballot ballot, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer<? super Result, Throwable> callback)
+ StabiliseTxn(Node node, Topologies coordinates, Topologies allTopologies, FullRoute<?> route, Ballot ballot, TxnId txnId, Txn txn, Timestamp executeAt, Deps unstableDeps, BiConsumer<? super Result, Throwable> callback)
{
super(node, coordinates, allTopologies, route, txnId, ballot, txn, executeAt, unstableDeps, callback);
}
@Override
- void onStabilised()
+ protected CoordinationAdapter<Result> adapter()
{
- Execute.execute(node, allTopologies, route, SLOW, txnId, txn, executeAt, stabiliseDeps, callback);
+ return node.coordinationAdapter(txnId, Continue);
}
}
diff --git a/accord-core/src/main/java/accord/coordinate/TopologyMismatch.java b/accord-core/src/main/java/accord/coordinate/TopologyMismatch.java
index 11b9aa4..f8791f8 100644
--- a/accord-core/src/main/java/accord/coordinate/TopologyMismatch.java
+++ b/accord-core/src/main/java/accord/coordinate/TopologyMismatch.java
@@ -34,7 +34,7 @@
private final EnumSet<Reason> reasons;
- private TopologyMismatch(EnumSet<Reason> reasons, Topology topology, TxnId txnId, @Nullable RoutingKey homeKey, Routables<?> keysOrRanges)
+ private TopologyMismatch(EnumSet<Reason> reasons, Topology topology, @Nullable TxnId txnId, @Nullable RoutingKey homeKey, Routables<?> keysOrRanges)
{
super(txnId, homeKey, buildMessage(reasons, topology, homeKey, keysOrRanges));
this.reasons = reasons;
@@ -72,7 +72,7 @@
}
@Nullable
- public static TopologyMismatch checkForMismatch(Topology t, TxnId txnId, RoutingKey homeKey, Routables<?> keysOrRanges)
+ public static TopologyMismatch checkForMismatch(Topology t, @Nullable TxnId txnId, RoutingKey homeKey, Routables<?> keysOrRanges)
{
EnumSet<TopologyMismatch.Reason> reasons = null;
if (!t.ranges().contains(homeKey))
diff --git a/accord-core/src/main/java/accord/coordinate/tracking/AbstractSimpleTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/AbstractSimpleTracker.java
new file mode 100644
index 0000000..845f334
--- /dev/null
+++ b/accord-core/src/main/java/accord/coordinate/tracking/AbstractSimpleTracker.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.coordinate.tracking;
+
+import java.util.function.Function;
+import java.util.function.IntFunction;
+
+import accord.local.Node.Id;
+import accord.topology.Shard;
+import accord.topology.Topologies;
+
+public abstract class AbstractSimpleTracker<ST extends ShardTracker> extends AbstractTracker<ST>
+{
+ AbstractSimpleTracker(Topologies topologies, IntFunction<ST[]> arrayFactory, Function<Shard, ST> trackerFactory)
+ {
+ super(topologies, arrayFactory, trackerFactory);
+ }
+
+ AbstractSimpleTracker(Topologies topologies, IntFunction<ST[]> arrayFactory, ShardFactory<ST> trackerFactory)
+ {
+ super(topologies, arrayFactory, trackerFactory);
+ }
+
+ public abstract RequestStatus recordSuccess(Id from);
+ public abstract RequestStatus recordFailure(Id from);
+}
diff --git a/accord-core/src/main/java/accord/coordinate/tracking/AppliedTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/AppliedTracker.java
index da49dc0..155e4f8 100644
--- a/accord-core/src/main/java/accord/coordinate/tracking/AppliedTracker.java
+++ b/accord-core/src/main/java/accord/coordinate/tracking/AppliedTracker.java
@@ -26,7 +26,7 @@
import static accord.coordinate.tracking.AbstractTracker.ShardOutcomes.NoChange;
import static accord.coordinate.tracking.AbstractTracker.ShardOutcomes.Success;
-public class AppliedTracker extends AbstractTracker<AppliedTracker.AppliedShardTracker> implements ResponseTracker
+public class AppliedTracker extends AbstractSimpleTracker<AppliedTracker.AppliedShardTracker> implements ResponseTracker
{
public static class AppliedShardTracker extends ShardTracker
{
diff --git a/accord-core/src/main/java/accord/coordinate/tracking/QuorumTracker.java b/accord-core/src/main/java/accord/coordinate/tracking/QuorumTracker.java
index c771406..70bc352 100644
--- a/accord-core/src/main/java/accord/coordinate/tracking/QuorumTracker.java
+++ b/accord-core/src/main/java/accord/coordinate/tracking/QuorumTracker.java
@@ -24,7 +24,7 @@
import static accord.coordinate.tracking.AbstractTracker.ShardOutcomes.*;
-public class QuorumTracker extends AbstractTracker<QuorumTracker.QuorumShardTracker> implements ResponseTracker
+public class QuorumTracker extends AbstractSimpleTracker<QuorumTracker.QuorumShardTracker> implements ResponseTracker
{
public static class QuorumShardTracker extends ShardTracker
{
diff --git a/accord-core/src/main/java/accord/impl/AbstractConfigurationService.java b/accord-core/src/main/java/accord/impl/AbstractConfigurationService.java
index d33c123..2f93a99 100644
--- a/accord-core/src/main/java/accord/impl/AbstractConfigurationService.java
+++ b/accord-core/src/main/java/accord/impl/AbstractConfigurationService.java
@@ -269,8 +269,6 @@
}
long lastAcked = epochs.lastAcknowledged;
- // TODO (now, review): lastAcked == 0, lastReceived = 2
- // if we wait for epoch=1.acknowledge the test seems to wait forever... looks like burn test doesn't ack epoch=1
if (lastAcked == 0 && lastReceived > 0)
{
epochs.acknowledgeFuture(epochs.minEpoch()).addCallback(() -> reportTopology(topology, startSync));
diff --git a/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java b/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java
index a3b3d5e..23de1e2 100644
--- a/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java
+++ b/accord-core/src/main/java/accord/impl/AbstractFetchCoordinator.java
@@ -122,9 +122,9 @@
protected abstract void onReadOk(Node.Id from, CommandStore commandStore, Data data, Ranges ranges);
- protected boolean collectMaxApplied()
+ protected FetchRequest newFetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn)
{
- return false;
+ return new FetchRequest(sourceEpoch, syncId, ranges, partialDeps, rangeReadTxn(ranges));
}
@Override
@@ -135,7 +135,7 @@
Ranges ownedRanges = ownedRangesForNode(to);
Invariants.checkArgument(ownedRanges.containsAll(ranges), "Got a reply from %s for ranges %s, but owned ranges %s does not contain all the ranges", to, ranges, ownedRanges);
PartialDeps partialDeps = syncPoint.waitFor.slice(ownedRanges, ranges);
- node.send(to, new FetchRequest(syncPoint.sourceEpoch(), syncPoint.syncId, ranges, partialDeps, rangeReadTxn(ranges), collectMaxApplied()), new Callback<ReadReply>()
+ node.send(to, newFetchRequest(syncPoint.sourceEpoch(), syncPoint.syncId, ranges, partialDeps, rangeReadTxn(ranges)), new Callback<ReadReply>()
{
@Override
public void onSuccess(Node.Id from, ReadReply reply)
@@ -228,50 +228,24 @@
void abort(Ranges abort)
{
- // TODO (required, later): implement abort
+ // TODO (expected): implement abort
}
public static class FetchRequest extends WaitUntilAppliedAndReadData
{
public final PartialDeps partialDeps;
- public final boolean collectMaxApplied;
- private transient Timestamp maxApplied;
- public FetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn, boolean collectMaxApplied)
+ public FetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn)
{
super(syncId, ranges, sourceEpoch, partialTxn);
this.partialDeps = partialDeps;
- this.collectMaxApplied = collectMaxApplied;
}
@Override
protected void readComplete(CommandStore commandStore, Data result, Ranges unavailable)
{
- Ranges slice = commandStore.unsafeRangesForEpoch().allAt(txnId).subtract(unavailable);
- if (collectMaxApplied)
- {
- commandStore.maxAppliedFor((Ranges)readScope, slice).begin((newMaxApplied, failure) -> {
- if (failure != null)
- {
- commandStore.agent().onUncaughtException(failure);
- }
- else
- {
- synchronized (this)
- {
- if (maxApplied == null) maxApplied = newMaxApplied;
- else maxApplied = Timestamp.max(maxApplied, newMaxApplied);
- Ranges reportUnavailable = unavailable.slice((Ranges)this.readScope, Minimal);
- super.readComplete(commandStore, result, reportUnavailable);
- }
- }
- });
- }
- else
- {
- Ranges reportUnavailable = unavailable.slice((Ranges)this.readScope, Minimal);
- super.readComplete(commandStore, result, reportUnavailable);
- }
+ Ranges reportUnavailable = unavailable.slice((Ranges)this.readScope, Minimal);
+ super.readComplete(commandStore, result, reportUnavailable);
}
@Override
@@ -279,7 +253,12 @@
{
// TODO (review): If the fetch response actually does some streaming, but we send back the error
// it is a lot of work and data that might move and be unaccounted for at the coordinator
- node.reply(replyTo, replyContext, fail == null ? new FetchResponse(unavailable, data, maxApplied) : null, fail);
+ node.reply(replyTo, replyContext, fail == null ? new FetchResponse(unavailable, data, maxApplied()) : null, fail);
+ }
+
+ protected Timestamp maxApplied()
+ {
+ return null;
}
@Override
diff --git a/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java b/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java
index 89058bf..0de56e7 100644
--- a/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java
+++ b/accord-core/src/main/java/accord/impl/AbstractSafeCommandStore.java
@@ -18,16 +18,14 @@
package accord.impl;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
import java.util.function.Consumer;
import java.util.function.Function;
-import accord.api.Key;
+import javax.annotation.Nullable;
+
+import com.google.common.annotations.VisibleForTesting;
+
import accord.api.VisibleForImplementation;
-import accord.impl.CommandTimeseries.CommandLoader;
import accord.local.*;
import accord.primitives.*;
import accord.utils.Invariants;
@@ -37,27 +35,10 @@
public abstract class AbstractSafeCommandStore<CommandType extends SafeCommand,
TimestampsForKeyType extends SafeTimestampsForKey,
- CommandsForKeyType extends SafeCommandsForKey,
- CommandsForKeyUpdateType extends SafeCommandsForKey.Update> extends SafeCommandStore
+ CommandsForKeyType extends SafeCommandsForKey> extends SafeCommandStore
{
- private static class PendingRegistration<T>
- {
- final T value;
- final Ranges slice;
- final TxnId txnId;
-
- public PendingRegistration(T value, Ranges slice, TxnId txnId)
- {
- this.value = value;
- this.slice = slice;
- this.txnId = txnId;
- }
- }
protected final PreLoadContext context;
- private List<PendingRegistration<Seekable>> pendingSeekableRegistrations = null;
- private List<PendingRegistration<Seekables<?, ?>>> pendingSeekablesRegistrations = null;
-
public AbstractSafeCommandStore(PreLoadContext context)
{
this.context = context;
@@ -84,18 +65,9 @@
protected abstract void addTimestampsForKeyInternal(TimestampsForKeyType cfk);
protected abstract TimestampsForKeyType getTimestampsForKeyIfLoaded(RoutableKey key);
- protected abstract CommandLoader<?> cfkLoader(RoutableKey key);
- protected abstract CommandsForKeyType getDepsCommandsForKeyInternal(RoutableKey key);
- protected abstract void addDepsCommandsForKeyInternal(CommandsForKeyType cfk);
- protected abstract CommandsForKeyType getDepsCommandsForKeyIfLoaded(RoutableKey key);
-
- protected abstract CommandsForKeyType getAllCommandsForKeyInternal(RoutableKey key);
- protected abstract void addAllCommandsForKeyInternal(CommandsForKeyType cfk);
- protected abstract CommandsForKeyType getAllCommandsForKeyIfLoaded(RoutableKey key);
-
- protected abstract CommandsForKeyUpdateType getCommandsForKeyUpdateInternal(RoutableKey key);
- protected abstract CommandsForKeyUpdateType createCommandsForKeyUpdateInternal(RoutableKey key);
- protected abstract void addCommandsForKeyUpdateInternal(CommandsForKeyUpdateType update);
+ protected abstract CommandsForKeyType getCommandsForKeyInternal(RoutableKey key);
+ protected abstract void addCommandsForKeyInternal(CommandsForKeyType cfk);
+ protected abstract CommandsForKeyType getCommandsForKeyIfLoaded(RoutableKey key);
@Override
protected CommandType getInternalIfLoadedAndInitialised(TxnId txnId)
@@ -117,87 +89,44 @@
return command;
}
- private CommandsForKeyType getCommandsIfLoaded(RoutableKey key, KeyHistory keyHistory)
+ private CommandsForKeyType getCommandsIfLoaded(RoutableKey key)
{
- switch (keyHistory)
- {
- case DEPS:
- return getIfLoaded(key, this::getDepsCommandsForKeyInternal, this::addDepsCommandsForKeyInternal, this::getDepsCommandsForKeyIfLoaded);
- case ALL:
- return getIfLoaded(key, this::getAllCommandsForKeyInternal, this::addAllCommandsForKeyInternal, this::getAllCommandsForKeyIfLoaded);
- default:
- throw new IllegalArgumentException("CommandsForKey not available for " + keyHistory);
- }
+ return getIfLoaded(key, this::getCommandsForKeyInternal, this::addCommandsForKeyInternal, this::getCommandsForKeyIfLoaded);
}
- private CommandsForKeyType commandsIfLoadedAndInitialised(RoutableKey key, KeyHistory keyHistory)
+ CommandsForKeyType commandsIfLoadedAndInitialised(RoutableKey key)
{
- CommandsForKeyType cfk = getCommandsIfLoaded(key, keyHistory);;
+ CommandsForKeyType cfk = getCommandsIfLoaded(key);
if (cfk == null)
return null;
if (cfk.isEmpty())
- {
- cfk.initialize(cfkLoader(key));
- }
- else
- {
- RedundantBefore.Entry entry = commandStore().redundantBefore().get(key.toUnseekable());
- if (entry != null && cfk.current().hasRedundant(entry.shardRedundantBefore()))
- cfk.set(cfk.current().withoutRedundant(entry.shardRedundantBefore()));
- }
+ cfk.initialize();
+
+ RedundantBefore.Entry entry = commandStore().redundantBefore().get(key.toUnseekable());
+ if (entry != null)
+ cfk.updateRedundantBefore(entry.shardRedundantBefore());
return cfk;
}
- protected CommandsForKeyType commandsForKey(RoutableKey key, KeyHistory keyHistory)
+ @VisibleForTesting
+ public CommandsForKeyType commandsForKey(RoutableKey key)
{
- CommandsForKeyType cfk = getCommandsIfLoaded(key, keyHistory);
+ CommandsForKeyType cfk = getCommandsIfLoaded(key);
Invariants.checkState(cfk != null, "%s was not specified in PreLoadContext", key);
if (cfk.isEmpty())
- cfk.initialize(cfkLoader(key));
+ cfk.initialize();
return cfk;
}
@VisibleForImplementation
- private CommandsForKeyType maybeCommandsForKey(RoutableKey key, KeyHistory keyHistory)
+ public CommandsForKeyType maybeCommandsForKey(RoutableKey key)
{
- CommandsForKeyType cfk = getCommandsIfLoaded(key, keyHistory);
+ CommandsForKeyType cfk = getCommandsIfLoaded(key);
if (cfk == null || cfk.isEmpty())
return null;
return cfk;
}
- public CommandsForKeyType depsCommandsIfLoadedAndInitialised(RoutableKey key)
- {
- return commandsIfLoadedAndInitialised(key, KeyHistory.DEPS);
- }
-
- public CommandsForKeyType depsCommandsForKey(RoutableKey key)
- {
- return commandsForKey(key, KeyHistory.DEPS);
- }
-
- @VisibleForImplementation
- public CommandsForKeyType maybeDepsCommandsForKey(RoutableKey key)
- {
- return maybeCommandsForKey(key, KeyHistory.DEPS);
- }
-
- public CommandsForKeyType allCommandsIfLoadedAndInitialised(RoutableKey key)
- {
- return commandsIfLoadedAndInitialised(key, KeyHistory.ALL);
- }
-
- public CommandsForKeyType allCommandsForKey(RoutableKey key)
- {
- return commandsForKey(key, KeyHistory.ALL);
- }
-
- @VisibleForImplementation
- public CommandsForKeyType maybeAllCommandsForKey(RoutableKey key)
- {
- return maybeCommandsForKey(key, KeyHistory.ALL);
- }
-
public TimestampsForKeyType timestampsIfLoadedAndInitialised(RoutableKey key)
{
TimestampsForKeyType cfk = getIfLoaded(key, this::getTimestampsForKeyInternal, this::addTimestampsForKeyInternal, this::getTimestampsForKeyIfLoaded);
@@ -229,108 +158,59 @@
return tfk;
}
- public CommandsForKeyUpdateType getOrCreateCommandsForKeyUpdate(RoutableKey key)
- {
- CommandsForKeyUpdateType update = getIfLoaded(key, this::getCommandsForKeyUpdateInternal, this::addCommandsForKeyUpdateInternal, this::createCommandsForKeyUpdateInternal);
- if (update == null)
- {
- update = createCommandsForKeyUpdateInternal(key);
- addCommandsForKeyUpdateInternal(update);
- }
-
- if (update.isEmpty())
- update.initialize();
-
- return update;
- }
-
@Override
- public void removeCommandFromSeekableDeps(Seekable seekable, TxnId txnId, Timestamp executeAt, Status status)
+ protected void update(Command prev, Command updated, @Nullable Seekables<?, ?> keysOrRanges)
{
- // the cfk listener doesn't know if it can remove the given command from the deps set without loading
- // the deps set, so we don't actually remove it until it becomes applied
- if (!status.hasBeen(Status.Applied))
+ super.update(prev, updated, keysOrRanges);
+
+ if (!CommandsForKey.needsUpdate(prev, updated))
return;
- switch (seekable.domain())
- {
- case Key:
- Key key = seekable.asKey();
- CommandsForKeyUpdater.Mutable<?> updater = getOrCreateCommandsForKeyUpdate(key).deps();
- updater.commands().remove(txnId);
- break;
- case Range:
- break;
- default:
- throw new IllegalArgumentException();
- }
+ TxnId txnId = updated.txnId();
+ if (!txnId.kind().isGloballyVisible() || !txnId.domain().isKey())
+ return;
+
+ // TODO (required): consider carefully epoch overlaps for dependencies;
+ // here we're limiting our registration with CFK to the coordination epoch only
+ // if we permit coordination+execution we have to do a very careful dance (or relax validation)
+ // because for some keys we can expect e.g. PreAccept and Accept states to have been processed
+ // and for other keys Committed onwards will appear suddenly (or, if we permit Accept to process
+ // on its executeAt ranges, it could go either way).
+ Ranges ranges = ranges().allAt(txnId);
+ Keys keys;
+ if (keysOrRanges != null) keys = (Keys) keysOrRanges;
+ else if (updated.known().isDefinitionKnown()) keys = (Keys)updated.partialTxn().keys();
+ else if (prev.known().isDefinitionKnown()) keys = (Keys)prev.partialTxn().keys();
+ else if (updated.saveStatus().hasBeen(Status.Truncated)) return; // TODO (required): we may have transaction registered via Accept, and still want to expunge. we shouldn't special case: should ensure we have everything loaded, or permit asynchronous application
+ else if (updated.saveStatus().is(Status.AcceptedInvalidate)) return; // TODO (required): we may have transaction registered via Accept, and still want to expunge. we shouldn't special case: should ensure we have everything loaded, or permit asynchronous application
+ else throw illegalState("No keys to update CommandsForKey with");
+
+ Routables.foldl(keys, ranges, (self, p, key, u, i) -> {
+ SafeCommandsForKey cfk = self.commandsIfLoadedAndInitialised(key);
+ // TODO (required): we shouldn't special case invalidations, truncations or topology changes: should ensure we have everything loaded, or permit asynchronous application
+ Invariants.checkState(cfk != null || u.saveStatus().hasBeen(Status.Truncated) || u.saveStatus().is(Status.AcceptedInvalidate));
+ if (cfk != null)
+ cfk.update(p, u);
+ return u;
+ }, this, prev, updated, i->false);
}
@Override
public boolean canExecuteWith(PreLoadContext context)
{
+ // TODO (required): check if data is in cache, and if so simply add it to our context
return context.isSubsetOf(this.context);
}
- @Override
- public void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command)
- {
- if (pendingSeekablesRegistrations == null)
- pendingSeekablesRegistrations = new ArrayList<>();
- pendingSeekablesRegistrations.add(new PendingRegistration<>(keysOrRanges, slice, command.txnId()));
- }
-
- @Override
- public void register(Seekable keyOrRange, Ranges slice, Command command)
- {
- if (pendingSeekableRegistrations == null)
- pendingSeekableRegistrations = new ArrayList<>();
- pendingSeekableRegistrations.add(new PendingRegistration<>(keyOrRange, slice, command.txnId()));
- }
-
- public abstract CommonAttributes completeRegistration(Seekables<?, ?> keysOrRanges, Ranges slice, CommandType command, CommonAttributes attrs);
-
- public abstract CommonAttributes completeRegistration(Seekable keyOrRange, Ranges slice, CommandType command, CommonAttributes attrs);
-
- private interface RegistrationCompleter<T, CommandType extends SafeCommand>
- {
- CommonAttributes complete(T value, Ranges ranges, CommandType command, CommonAttributes attrs);
- }
-
- private <T> void completeRegistrations(Map<TxnId, CommonAttributes> updates, List<PendingRegistration<T>> pendingRegistrations, RegistrationCompleter<T, CommandType> completer)
- {
- if (pendingRegistrations == null)
- return;
-
- for (PendingRegistration<T> pendingRegistration : pendingRegistrations)
- {
- TxnId txnId = pendingRegistration.txnId;
- CommandType safeCommand = getInternal(pendingRegistration.txnId);
- Command command = safeCommand.current();
- CommonAttributes attrs = updates.getOrDefault(txnId, command);
- attrs = completer.complete(pendingRegistration.value, pendingRegistration.slice, safeCommand, attrs);
- if (attrs != command)
- updates.put(txnId, attrs);
- }
- }
-
protected abstract void invalidateSafeState();
- protected abstract void applyCommandForKeyUpdates();
public void postExecute()
{
- if (pendingSeekableRegistrations != null || pendingSeekablesRegistrations != null)
- {
- Map<TxnId, CommonAttributes> attributeUpdates = new HashMap<>();
- completeRegistrations(attributeUpdates, pendingSeekablesRegistrations, this::completeRegistration);
- completeRegistrations(attributeUpdates, pendingSeekableRegistrations, this::completeRegistration);
- attributeUpdates.forEach(((txnId, attributes) -> get(txnId).updateAttributes(attributes)));
- }
- applyCommandForKeyUpdates();
}
public void complete()
{
+ postExecute();
invalidateSafeState();
}
}
diff --git a/accord-core/src/main/java/accord/impl/CommandTimeseries.java b/accord-core/src/main/java/accord/impl/CommandTimeseries.java
deleted file mode 100644
index 0c0c727..0000000
--- a/accord-core/src/main/java/accord/impl/CommandTimeseries.java
+++ /dev/null
@@ -1,537 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.impl;
-
-import java.util.*;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-import java.util.stream.Stream;
-import javax.annotation.Nullable;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.ImmutableSortedMap;
-
-import accord.api.Key;
-import accord.local.Command;
-import accord.local.SafeCommandStore;
-import accord.local.SaveStatus;
-import accord.local.Status;
-import accord.primitives.Seekable;
-import accord.primitives.Timestamp;
-import accord.primitives.Txn.Kind.Kinds;
-import accord.primitives.TxnId;
-
-import static accord.local.SafeCommandStore.TestDep.ANY_DEPS;
-import static accord.local.SafeCommandStore.TestDep.WITH;
-import static accord.utils.Utils.ensureSortedImmutable;
-import static accord.utils.Utils.ensureSortedMutable;
-
-public class CommandTimeseries<D>
-{
- public enum TestTimestamp { BEFORE, AFTER }
- public enum TimestampType { TXN_ID, EXECUTE_AT }
-
- private final Seekable keyOrRange;
- protected final CommandLoader<D> loader;
- public final ImmutableSortedMap<Timestamp, D> commands;
-
- public CommandTimeseries(Builder<D> builder)
- {
- this.keyOrRange = builder.keyOrRange;
- this.loader = builder.loader;
- this.commands = ensureSortedImmutable(builder.commands);
- }
-
- CommandTimeseries(Seekable keyOrRange, CommandLoader<D> loader, ImmutableSortedMap<Timestamp, D> commands)
- {
- this.keyOrRange = keyOrRange;
- this.loader = loader;
- this.commands = commands;
- }
-
- public CommandTimeseries(Key keyOrRange, CommandLoader<D> loader)
- {
- this(keyOrRange, loader, ImmutableSortedMap.of());
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- CommandTimeseries<?> that = (CommandTimeseries<?>) o;
- return keyOrRange.equals(that.keyOrRange) && loader.equals(that.loader) && commands.equals(that.commands);
- }
-
- @Override
- public int hashCode()
- {
- int hash = 1;
- hash = 31 * hash + Objects.hashCode(keyOrRange);
- hash = 31 * hash + Objects.hashCode(loader);
- hash = 31 * hash + Objects.hashCode(commands);
- return hash;
- }
-
- public D get(Timestamp key)
- {
- return commands.get(key);
- }
-
- public boolean isEmpty()
- {
- return commands.isEmpty();
- }
-
- public Timestamp maxTimestamp()
- {
- if (commands.isEmpty())
- return Timestamp.NONE;
- Timestamp result = null;
- for (D data : commands.values())
- {
- if (result == null)
- {
- result = Timestamp.max(loader.txnId(data), loader.executeAt(data));
- }
- else
- {
- result = Timestamp.max(result, loader.txnId(data));
- result = Timestamp.max(result, loader.executeAt(data));
- }
- }
- return result;
- }
-
- public Timestamp minTimestamp()
- {
- if (commands.isEmpty())
- return Timestamp.NONE;
- Timestamp result = null;
- for (D data : commands.values())
- {
- if (result == null)
- {
- result = Timestamp.nonNullOrMin(loader.txnId(data), loader.executeAt(data));
- }
- else
- {
- result = Timestamp.min(result, loader.txnId(data));
- result = Timestamp.nonNullOrMin(result, loader.executeAt(data));
- }
- }
- return result;
- }
-
- /**
- * All commands before/after (exclusive of) the given timestamp
- * <p>
- * Note that {@code testDep} applies only to commands that know at least proposed deps; if specified any
- * commands that do not know any deps will be ignored.
- * <p>
- * TODO (expected, efficiency): TestDep should be asynchronous; data should not be kept memory-resident as only used for recovery
- */
- public <P1, T> T mapReduce(Kinds testKind, TimestampType timestampType, TestTimestamp testTimestamp, Timestamp timestamp,
- SafeCommandStore.TestDep testDep, @Nullable TxnId depId,
- @Nullable Status minStatus, @Nullable Status maxStatus,
- SafeCommandStore.CommandFunction<P1, T, T> map, P1 p1, T initialValue, Predicate<? super T> terminatePredicate)
- {
- Iterable<D> dataIterable;
- Predicate<Timestamp> executeAtPredicate;
- if (timestampType == TimestampType.TXN_ID)
- {
- dataIterable = testTimestamp == TestTimestamp.BEFORE ? commands.headMap(timestamp, false).values() : commands.tailMap(timestamp, false).values();
- executeAtPredicate = ts -> true;
- }
- else
- {
- dataIterable = commands.values();
- executeAtPredicate = testTimestamp == TestTimestamp.BEFORE ? ts -> ts.compareTo(timestamp) < 0 : ts -> ts.compareTo(timestamp) > 0;
- }
-
-
- for (D data : dataIterable)
- {
- TxnId txnId = loader.txnId(data);
- if (!testKind.test(txnId.kind())) continue;
- SaveStatus status = loader.saveStatus(data);
- if (minStatus != null && minStatus.compareTo(status.status) > 0)
- continue;
- if (maxStatus != null && maxStatus.compareTo(status.status) < 0)
- continue;
-
- Timestamp executeAt = loader.executeAt(data);
- if (!executeAtPredicate.test(executeAt))
- continue;
-
- List<TxnId> deps = loader.depsIds(data);
- // If we don't have any dependencies, we treat a dependency filter as a mismatch
- if (testDep != ANY_DEPS && (!status.known.deps.hasProposedOrDecidedDeps() || (deps.contains(depId) != (testDep == WITH))))
- continue;
- initialValue = map.apply(p1, keyOrRange, txnId, executeAt, status.status, () -> deps, initialValue);
- if (terminatePredicate.test(initialValue))
- break;
- }
- return initialValue;
- }
-
- public Timestamp maxExecuteAtBefore(Timestamp before)
- {
- return commands.headMap(before).values().stream().map(loader::executeAt)
- .filter(Objects::nonNull).reduce(Timestamp::max).orElse(before);
- }
-
- Stream<TxnId> between(Timestamp min, Timestamp max, Predicate<Status> statusPredicate)
- {
- return commands.subMap(min, true, max, true).values().stream()
- .filter(d -> statusPredicate.test(loader.status(d))).map(loader::txnId);
- }
-
- public Stream<D> all()
- {
- return commands.values().stream();
- }
-
- Builder<D> unbuild()
- {
- return new Builder<>(this);
- }
-
- public CommandLoader<D> loader()
- {
- return loader;
- }
-
- public interface CommandLoader<D>
- {
- D saveForCFK(Command command);
-
- TxnId txnId(D data);
- Timestamp executeAt(D data);
- SaveStatus saveStatus(D data);
- List<TxnId> depsIds(D data);
-
- default Status status(D data)
- {
- return saveStatus(data).status;
- }
-
- default Status.Known known(D data)
- {
- return saveStatus(data).known;
- }
- }
-
- public static class Builder<D>
- {
- private final Seekable keyOrRange;
- protected CommandLoader<D> loader;
- protected NavigableMap<Timestamp, D> commands;
-
- public Builder(Seekable keyOrRange, CommandLoader<D> loader)
- {
- this.keyOrRange = keyOrRange;
- this.loader = loader;
- this.commands = new TreeMap<>();
- }
-
- public Builder(CommandTimeseries<D> timeseries)
- {
- this.keyOrRange = timeseries.keyOrRange;
- this.loader = timeseries.loader;
- this.commands = timeseries.commands;
- }
-
- public Builder<D> add(Timestamp timestamp, Command command)
- {
- commands = ensureSortedMutable(commands);
- commands.put(timestamp, loader.saveForCFK(command));
- return this;
- }
-
- public Builder<D> add(Timestamp timestamp, D value)
- {
- commands = ensureSortedMutable(commands);
- commands.put(timestamp, value);
- return this;
- }
-
- public Builder<D> remove(Timestamp timestamp)
- {
- commands = ensureSortedMutable(commands);
- commands.remove(timestamp);
- return this;
- }
-
- public Builder<D> removeBefore(Timestamp timestamp)
- {
- commands = ensureSortedMutable(commands);
- commands.headMap(timestamp, false).clear();
- return this;
- }
-
- public CommandTimeseries<D> build()
- {
- return new CommandTimeseries<>(this);
- }
- }
-
- public interface Update<T extends Timestamp, D>
- {
- interface Mutable<T extends Timestamp, D> extends Update<T, D>
- {
- void add(T ts, Command command);
- void remove(T ts);
- }
-
- int numWrites();
- int numDeletes();
-
- boolean contains(T key);
-
- default int numChanges()
- {
- return numWrites() + numDeletes();
- }
-
- default boolean isEmpty()
- {
- return numChanges() == 0;
- }
-
- void forEachWrite(BiConsumer<T, D> consumer);
- void forEachDelete(Consumer<T> consumer);
-
- default CommandTimeseries<D> apply(CommandTimeseries<D> current)
- {
- if (isEmpty())
- return current;
-
- CommandTimeseries.Builder<D> builder = current.unbuild();
- forEachWrite(builder::add);
- forEachDelete(builder::remove);
- return builder.build();
- }
-
- }
-
- public static class ImmutableUpdate<T extends Timestamp, D> implements Update<T, D>
- {
- private static final ImmutableUpdate<?, ?> EMPTY = new ImmutableUpdate<>(ImmutableMap.of(), ImmutableSet.of());
-
- public final ImmutableMap<T, D> writes;
- public final ImmutableSet<T> deletes;
-
- public ImmutableUpdate(ImmutableMap<T, D> writes, ImmutableSet<T> deletes)
- {
- this.writes = writes;
- this.deletes = deletes;
- }
-
- public boolean equals(Object o)
- {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- ImmutableUpdate<?, ?> immutable = (ImmutableUpdate<?, ?>) o;
- return Objects.equals(writes, immutable.writes) && Objects.equals(deletes, immutable.deletes);
- }
-
- public int hashCode()
- {
- throw new UnsupportedOperationException();
- }
-
- public String toString()
- {
- return "Update.Immutable{" +
- "writes=" + writes +
- ", deletes=" + deletes +
- '}';
- }
-
- public static <T extends Timestamp, D> ImmutableUpdate<T, D> empty()
- {
- return (ImmutableUpdate<T, D>) EMPTY;
- }
-
- @Override
- public int numWrites()
- {
- return writes.size();
- }
-
- @Override
- public void forEachWrite(BiConsumer<T, D> consumer)
- {
- writes.forEach(consumer);
- }
-
- @Override
- public int numDeletes()
- {
- return deletes.size();
- }
-
- @Override
- public void forEachDelete(Consumer<T> consumer)
- {
- deletes.forEach(consumer);
- }
-
- public boolean contains(T key)
- {
- return writes.containsKey(key) || deletes.contains(key);
- }
-
- public ImmutableUpdate<T, D> apply(Update<T, D> next)
- {
- if (next.isEmpty())
- return this;
-
- Map<T, D> writes = new HashMap<>(this.writes);
- Set<T> deletes = new HashSet<>(this.deletes);
-
- next.forEachDelete(k -> {
- writes.remove(k);
- deletes.add(k);
- });
-
- next.forEachWrite((k, v) -> {
- writes.put(k , v);
- deletes.remove(k);
- });
-
- return new ImmutableUpdate<>(ImmutableMap.copyOf(writes), ImmutableSet.copyOf(deletes));
- }
-
- public MutableUpdate<T, D> toMutable(CommandLoader<D> loader)
- {
- return new MutableUpdate<>(loader, new HashMap<>(writes), new HashSet<>(deletes));
- }
- }
-
- public static class MutableUpdate<T extends Timestamp, D> implements Update.Mutable<T, D>
- {
- private final CommandLoader<D> loader;
- private final Map<T, D> writes;
- private final Set<T> deletes;
-
- public MutableUpdate(CommandLoader<D> loader, Map<T, D> writes, Set<T> deletes)
- {
- this.loader = loader;
- this.writes = writes;
- this.deletes = deletes;
- }
-
- public MutableUpdate(CommandLoader<D> loader)
- {
- this(loader, new HashMap<>(), new HashSet<>());
- }
-
- @Override
- public void add(T ts, Command command)
- {
- writes.put(ts, loader.saveForCFK(command));
- deletes.remove(ts);
- }
-
- @Override
- public void remove(T ts)
- {
- deletes.add(ts);
- writes.remove(ts);
- }
-
- @Override
- public int numWrites()
- {
- return writes.size();
- }
-
- @Override
- public void forEachWrite(BiConsumer<T, D> consumer)
- {
- writes.forEach(consumer);
- }
-
- @Override
- public int numDeletes()
- {
- return deletes.size();
- }
-
- @Override
- public void forEachDelete(Consumer<T> consumer)
- {
- deletes.forEach(consumer);
- }
-
- public boolean contains(T key)
- {
- return writes.containsKey(key) || deletes.contains(key);
- }
-
- /**
- * remove the given key from both the writes and deletes set
- * @param key
- */
- void removeKeyUnsafe(T key)
- {
- writes.remove(key);
- deletes.remove(key);
- }
-
- /**
- * add the given write data directly, without using the command loader
- * @param key
- * @param value
- */
- void addWriteUnsafe(T key, D value)
- {
- writes.put(key, value);
- deletes.remove(key);
- }
-
- /**
- * if this update has a write or delete for the given key, remove
- * it from this update and put it in the destination update
- */
- boolean transferKeyTo(T key, MutableUpdate<T, D> dst)
- {
- if (writes.containsKey(key))
- {
- dst.writes.put(key, this.writes.remove(key));
- return true;
- }
- else if (deletes.remove(key))
- {
- dst.deletes.add(key);
- return true;
- }
- return false;
- }
-
- public ImmutableUpdate<T, D> toImmutable()
- {
- return new ImmutableUpdate<>(ImmutableMap.copyOf(writes), ImmutableSet.copyOf(deletes));
- }
- }
-}
diff --git a/accord-core/src/main/java/accord/impl/CommandsForKey.java b/accord-core/src/main/java/accord/impl/CommandsForKey.java
index 59be3bd..185c143 100644
--- a/accord-core/src/main/java/accord/impl/CommandsForKey.java
+++ b/accord-core/src/main/java/accord/impl/CommandsForKey.java
@@ -18,44 +18,251 @@
package accord.impl;
-import accord.api.Key;
-import accord.impl.CommandTimeseries.CommandLoader;
-import com.google.common.collect.ImmutableSortedMap;
-
+import java.util.Arrays;
+import java.util.EnumMap;
import java.util.Objects;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import accord.api.Key;
import accord.local.Command;
-import accord.local.PreLoadContext;
-import accord.local.SafeCommand;
-import accord.local.SafeCommandStore;
-import accord.primitives.Keys;
+import accord.local.SafeCommandStore.CommandFunction;
+import accord.local.SafeCommandStore.TestDep;
+import accord.local.SafeCommandStore.TestStartedAt;
+import accord.local.SafeCommandStore.TestStatus;
+import accord.local.SaveStatus;
+import accord.local.Status;
+import accord.primitives.Ballot;
import accord.primitives.Timestamp;
+import accord.primitives.Txn.Kind.Kinds;
import accord.primitives.TxnId;
+import accord.utils.Invariants;
+import accord.utils.SortedArrays;
+import accord.utils.SortedList;
-public class CommandsForKey implements DomainCommands
+import static accord.impl.CommandsForKey.InternalStatus.ACCEPTED;
+import static accord.impl.CommandsForKey.InternalStatus.COMMITTED;
+import static accord.impl.CommandsForKey.InternalStatus.STABLE;
+import static accord.impl.CommandsForKey.InternalStatus.HISTORICAL;
+import static accord.impl.CommandsForKey.InternalStatus.INVALID_OR_TRUNCATED;
+import static accord.impl.CommandsForKey.InternalStatus.TRANSITIVELY_KNOWN;
+import static accord.local.SafeCommandStore.TestDep.ANY_DEPS;
+import static accord.local.SafeCommandStore.TestDep.WITH;
+import static accord.primitives.Txn.Kind.Write;
+import static accord.utils.ArrayBuffers.cachedTxnIds;
+import static accord.utils.Invariants.illegalState;
+import static accord.utils.Invariants.isParanoid;
+import static accord.utils.SortedArrays.linearUnion;
+
+/**
+ * A specialised collection for efficiently representing and querying everything we need for making coordination
+ * and recovery decisions about a key's command conflicts.
+ *
+ * Every command we know about that is not shard-redundant is listed in the TxnId[] collection, which is sorted by TxnId.
+ * This list implies the contents of the deps of all commands in the collection - it is assumed that in the normal course
+ * of events every transaction will include the full set of TxnId we know. We only encode divergences from this, stored
+ * in each command's {@code missing} collection.
+ *
+ * We then go one step further, exploiting the fact that the missing collection exists solely to implement recovery,
+ * and so we elide from this missing collection any TxnId we have recorded as Committed or higher.
+ * Any recovery coordinator that contacts this replica will report that the command has been agreed to execute,
+ * and so will not need to decipher any fast-path decisions. So the missing collection is redundant, as no command's deps
+ * will not be queried for this TxnId's presence/absence.
+ * TODO (expected) this logic applies equally well to Accepted
+ *
+ * The goal with these behaviours is that this missing collection will ordinarily be empty, and represented by the exact
+ * same NO_TXNIDS array instance as every other command.
+ *
+ * We also exploit the property that most commands will also agree to execute at their proposed TxnId. If we have
+ * no missing collection to encode, and no modified executeAt, we store a global NoInfo object that takes up no
+ * space on heap. These NoInfo objects permit further efficiencies, as we may perform class-pointer comparisons
+ * before querying any contents to skip uninteresting contents, permitting fast iteration of the collection's contents.
+ *
+ * We also impose the condition that every TxnId is uniquely represented in the collection, so any executeAt and missing
+ * collection that represents the same value as the TxnId must be the same object present in the main TxnId[].
+ *
+ * This collection also implements transitive dependency elision.
+ * When evaluating mapReduceActive, we first establish the last-executing Stable write command (i.e. those whose deps
+ * are considered durably decided, and so must wait for all commands Committed with a lower executeAt).
+ * We then elide any Committed command that has a lower executeAt than this command.
+ *
+ * Both commands must be known at a majority, but neither might be Committed at any other replica.
+ * Either command may therefore be recovered.
+ * If the later command is recovered, this replica will report its Stable deps thereby recovering them.
+ * If this replica is not contacted, some other replica must participate that either has taken the same action as this replica,
+ * or else does not know the later command is Stable, and so will report the earlier command as a dependency again.
+ * If the earlier command is recovered, this replica will report that it is Committed, and so will not consult
+ * this replica's collection to decipher any fast path decision. Any other replica must either do the same, or else
+ * will correctly record this transaction as present in any relevant deps of later transactions.
+ *
+ * TODO (expected): optimisations:
+ * 1) we probably can rely only on COMMITTED status rather than STABLE for filtering active, but require STABLE still for filtering on status
+ * 2) do we need PREAPPLIED state?
+ * 3) consider storing a prefix of TxnId that are all NoInfo PreApplied encoded as a BitStream as only required for computing missing collection
+ * 4) consider storing (or caching) an int[] of records with an executeAt that occurs out of order, sorted by executeAt
+ *
+ * TODO (required): randomised testing
+ */
+public class CommandsForKey implements CommandsSummary
{
+ private static final boolean PRUNE_TRANSITIVE_DEPENDENCIES = true;
+ public static final TxnId[] NO_TXNIDS = new TxnId[0];
+ public static final Info[] NO_INFOS = new Info[0];
+
public static class SerializerSupport
{
- public static Listener listener(Key key)
+ public static CommandsForKey create(Key key, TxnId redundantBefore, TxnId[] txnIds, Info[] infos)
{
- return new Listener(key);
- }
-
- public static <D> CommandsForKey create(Key key,
- CommandLoader<D> loader,
- ImmutableSortedMap<Timestamp, D> commands)
- {
- return new CommandsForKey(key, loader, commands);
+ return new CommandsForKey(key, redundantBefore, txnIds, infos);
}
}
- public static class Listener implements Command.DurableAndIdempotentListener
+ public enum InternalStatus
{
- protected final Key listenerKey;
+ TRANSITIVELY_KNOWN(false, false), // (unwitnessed; no need for mapReduce to witness)
+ HISTORICAL(false, false),
+ PREACCEPTED(false),
+ ACCEPTED(true),
+ COMMITTED(true),
+ STABLE(true),
+ // TODO (required): we can probably retire this status; STABLE is sufficient
+ PREAPPLIED(true),
+ INVALID_OR_TRUNCATED(false);
- public Listener(Key listenerKey)
+ static final EnumMap<SaveStatus, InternalStatus> convert = new EnumMap<>(SaveStatus.class);
+ static final InternalStatus[] VALUES = values();
+ static
{
- this.listenerKey = listenerKey;
+ convert.put(SaveStatus.PreAccepted, PREACCEPTED);
+ convert.put(SaveStatus.AcceptedInvalidateWithDefinition, PREACCEPTED);
+ convert.put(SaveStatus.Accepted, ACCEPTED);
+ convert.put(SaveStatus.AcceptedWithDefinition, ACCEPTED);
+ convert.put(SaveStatus.PreCommittedWithDefinition, PREACCEPTED);
+ convert.put(SaveStatus.PreCommittedWithAcceptedDeps, ACCEPTED);
+ convert.put(SaveStatus.PreCommittedWithDefinitionAndAcceptedDeps, ACCEPTED);
+ convert.put(SaveStatus.Committed, COMMITTED);
+ convert.put(SaveStatus.Stable, STABLE);
+ convert.put(SaveStatus.ReadyToExecute, STABLE);
+ convert.put(SaveStatus.PreApplied, PREAPPLIED);
+ convert.put(SaveStatus.Applying, PREAPPLIED);
+ convert.put(SaveStatus.Applied, PREAPPLIED);
+ convert.put(SaveStatus.TruncatedApplyWithDeps, INVALID_OR_TRUNCATED);
+ convert.put(SaveStatus.TruncatedApplyWithOutcome, INVALID_OR_TRUNCATED);
+ convert.put(SaveStatus.TruncatedApply, INVALID_OR_TRUNCATED);
+ convert.put(SaveStatus.ErasedOrInvalidated, INVALID_OR_TRUNCATED);
+ convert.put(SaveStatus.Erased, INVALID_OR_TRUNCATED);
+ convert.put(SaveStatus.Invalidated, INVALID_OR_TRUNCATED);
+ }
+
+ public final boolean hasInfo;
+ public final NoInfo asNoInfo;
+ public final InfoWithAdditions asNoInfoOrAdditions;
+ final InternalStatus expectMatch;
+
+ InternalStatus(boolean hasInfo)
+ {
+ this(hasInfo, true);
+ }
+
+ InternalStatus(boolean hasInfo, boolean expectMatch)
+ {
+ this.hasInfo = hasInfo;
+ this.asNoInfo = new NoInfo(this);
+ this.asNoInfoOrAdditions = new InfoWithAdditions(asNoInfo, NO_TXNIDS, 0);
+ this.expectMatch = expectMatch ? this : null;
+ }
+
+ boolean hasExecuteAt()
+ {
+ return hasInfo;
+ }
+
+ boolean hasDeps()
+ {
+ return hasInfo;
+ }
+
+ boolean hasStableDeps()
+ {
+ return this == STABLE || this == PREAPPLIED;
+ }
+
+ public Timestamp depsKnownBefore(TxnId txnId, @Nullable Timestamp executeAt)
+ {
+ switch (this)
+ {
+ default: throw new AssertionError("Unhandled InternalStatus: " + this);
+ case TRANSITIVELY_KNOWN:
+ case INVALID_OR_TRUNCATED:
+ case HISTORICAL:
+ throw new AssertionError("Invalid InternalStatus to know deps");
+
+ case PREACCEPTED:
+ case ACCEPTED:
+ return txnId;
+
+ case PREAPPLIED:
+ case STABLE:
+ case COMMITTED:
+ return executeAt == null ? txnId : executeAt;
+ }
+ }
+
+ @VisibleForTesting
+ public static InternalStatus from(SaveStatus status)
+ {
+ return convert.get(status);
+ }
+
+ public static InternalStatus get(int ordinal)
+ {
+ return VALUES[ordinal];
+ }
+ }
+
+ public static class Info
+ {
+ public final InternalStatus status;
+ // ACCESS DIRECTLY WITH CARE: if null, TxnId is implied; use accessor method to ensure correct value is returned
+ public final @Nullable Timestamp executeAt;
+ public final TxnId[] missing; // those TxnId we know of that would be expected to be found in the provided deps, but aren't
+
+ private Info(InternalStatus status, @Nullable Timestamp executeAt, TxnId[] missing)
+ {
+ this.status = status;
+ this.executeAt = executeAt;
+ this.missing = missing;
+ }
+
+ public static Info create(@Nonnull TxnId txnId, InternalStatus status, @Nonnull Timestamp executeAt, @Nonnull TxnId[] missing)
+ {
+ return new Info(status,
+ Invariants.checkArgument(executeAt, executeAt == txnId || executeAt.compareTo(txnId) > 0),
+ Invariants.checkArgument(missing, missing == NO_TXNIDS || missing.length > 0));
+ }
+
+ public static Info createMock(InternalStatus status, @Nullable Timestamp executeAt, TxnId[] missing)
+ {
+ return new Info(status, executeAt, Invariants.checkArgument(missing, missing == null || missing == NO_TXNIDS));
+ }
+
+ @VisibleForTesting
+ public Timestamp executeAt(TxnId txnId)
+ {
+ return executeAt;
+ }
+
+ Timestamp depsKnownBefore(TxnId txnId)
+ {
+ return status.depsKnownBefore(txnId, executeAt);
+ }
+
+ Info update(TxnId txnId, TxnId[] newMissing)
+ {
+ return newMissing == NO_TXNIDS && executeAt == txnId ? status.asNoInfo : Info.create(txnId, status, executeAt(txnId), newMissing);
}
@Override
@@ -63,59 +270,69 @@
{
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
- Listener that = (Listener) o;
- return listenerKey.equals(that.listenerKey);
+ Info info = (Info) o;
+ return status == info.status && Objects.equals(executeAt, info.executeAt) && Arrays.equals(missing, info.missing);
}
@Override
public int hashCode()
{
- return Objects.hash(listenerKey);
+ throw new UnsupportedOperationException();
}
@Override
public String toString()
{
- return "ListenerProxy{" + listenerKey + '}';
+ return "Info{" +
+ "status=" + status +
+ ", executeAt=" + executeAt +
+ ", missing=" + Arrays.toString(missing) +
+ '}';
+ }
+ }
+
+ public static class NoInfo extends Info
+ {
+ NoInfo(InternalStatus status)
+ {
+ super(status, null, NO_TXNIDS);
}
- public Key key()
+ public Timestamp executeAt(TxnId txnId)
{
- return listenerKey;
- }
-
- @Override
- public void onChange(SafeCommandStore safeStore, SafeCommand safeCommand)
- {
- CommandsForKeys.listenerUpdate((AbstractSafeCommandStore<?,?,?,?>) safeStore, listenerKey, safeCommand.current());
- }
-
- @Override
- public PreLoadContext listenerPreLoadContext(TxnId caller)
- {
- return PreLoadContext.contextFor(caller, Keys.of(listenerKey));
+ return txnId;
}
}
private final Key key;
- private final CommandTimeseries<?> commands;
+ private final TxnId redundantBefore;
+ // any transactions that are durably decided (i.e. STABLE) and execute before this are durably dependent, and can be elided from mapReduceActive
+ private final @Nullable Timestamp maxStableWrite;
+ private final TxnId[] txnIds;
+ private final Info[] infos;
- <D> CommandsForKey(Key key,
- CommandTimeseries<D> commands)
+ CommandsForKey(Key key, TxnId redundantBefore, TxnId[] txnIds, Info[] infos)
{
- this.key = key;
- this.commands = commands;
+ this(key, redundantBefore, maxStableWriteBefore(Timestamp.MAX, txnIds, infos), txnIds, infos);
}
- <D> CommandsForKey(Key key, CommandLoader<D> loader, ImmutableSortedMap<Timestamp, D> commands)
- {
- this(key, new CommandTimeseries<>(key, loader, commands));
- }
-
- public <D> CommandsForKey(Key key, CommandLoader<D> loader)
+ CommandsForKey(Key key, TxnId redundantBefore, @Nullable Timestamp maxStableWrite, TxnId[] txnIds, Info[] infos)
{
this.key = key;
- this.commands = new CommandTimeseries<>(key, loader);
+ this.redundantBefore = redundantBefore;
+ this.maxStableWrite = maxStableWrite;
+ this.txnIds = txnIds;
+ this.infos = infos;
+ if (isParanoid()) Invariants.checkArgument(SortedArrays.isSortedUnique(txnIds));
+ }
+
+ public CommandsForKey(Key key)
+ {
+ this.key = key;
+ this.redundantBefore = TxnId.NONE;
+ this.maxStableWrite = null;
+ this.txnIds = NO_TXNIDS;
+ this.infos = NO_INFOS;
}
@Override
@@ -124,50 +341,718 @@
return "CommandsForKey@" + System.identityHashCode(this) + '{' + key + '}';
}
+ public Key key()
+ {
+ return key;
+ }
+
+ public int size()
+ {
+ return txnIds.length;
+ }
+
+ public int indexOf(TxnId txnId)
+ {
+ return Arrays.binarySearch(txnIds, txnId);
+ }
+
+ public TxnId txnId(int i)
+ {
+ return txnIds[i];
+ }
+
+ public Info info(int i)
+ {
+ return infos[i];
+ }
+
+ public TxnId redundantBefore()
+ {
+ return redundantBefore;
+ }
+
+ /**
+ * All commands before/after (exclusive of) the given timestamp
+ * <p>
+ * Note that {@code testDep} applies only to commands that MAY have the command in their deps; if specified any
+ * commands that do not know any deps will be ignored, as will any with an executeAt prior to the txnId.
+ * <p>
+ */
+ public <P1, T> T mapReduceFull(TxnId testTxnId,
+ Kinds testKind,
+ TestStartedAt testStartedAt,
+ TestDep testDep,
+ TestStatus testStatus,
+ CommandFunction<P1, T, T> map, P1 p1, T initialValue)
+ {
+ int start, end;
+ boolean isKnown;
+ {
+ int insertPos = Arrays.binarySearch(txnIds, testTxnId);
+ isKnown = insertPos >= 0;
+ if (!isKnown && testDep == WITH) return initialValue;
+ if (!isKnown) insertPos = -1 - insertPos;
+ switch (testStartedAt)
+ {
+ default: throw new AssertionError("Unhandled TestStartedAt: " + testTxnId);
+ case STARTED_BEFORE: start = 0; end = insertPos; break;
+ case STARTED_AFTER: start = insertPos; end = txnIds.length; break;
+ case ANY: start = 0; end = txnIds.length;
+ }
+ }
+
+ for (int i = start; i < end ; ++i)
+ {
+ TxnId txnId = txnIds[i];
+ if (!testKind.test(txnId.kind())) continue;
+
+ Info info = infos[i];
+ InternalStatus status = info.status;
+ switch (testStatus)
+ {
+ default: throw new AssertionError("Unhandled TestStatus: " + testStatus);
+ case IS_PROPOSED:
+ if (status == ACCEPTED || status == COMMITTED) break;
+ else continue;
+ case IS_STABLE:
+ if (status.compareTo(STABLE) >= 0 && status.compareTo(INVALID_OR_TRUNCATED) < 0) break;
+ else continue;
+ case ANY_STATUS:
+ if (status == TRANSITIVELY_KNOWN)
+ continue;
+ }
+
+ Timestamp executeAt = info.executeAt(txnId);
+ if (testDep != ANY_DEPS)
+ {
+ if (!status.hasInfo)
+ continue;
+
+ if (executeAt.compareTo(testTxnId) <= 0)
+ continue;
+
+ boolean hasAsDep = Arrays.binarySearch(info.missing, testTxnId) < 0;
+ if (hasAsDep != (testDep == WITH))
+ continue;
+ }
+
+ initialValue = map.apply(p1, key, txnId, executeAt, initialValue);
+ }
+ return initialValue;
+ }
+
+ public <P1, T> T mapReduceActive(Timestamp startedBefore,
+ Kinds testKind,
+ CommandFunction<P1, T, T> map, P1 p1, T initialValue)
+ {
+ Timestamp maxStableWrite = maxStableWriteBefore(startedBefore);
+ int start = 0, end = insertPos(startedBefore);
+
+ for (int i = start; i < end ; ++i)
+ {
+ TxnId txnId = txnIds[i];
+ if (!testKind.test(txnId.kind()))
+ continue;
+
+ Info info = infos[i];
+ switch (info.status)
+ {
+ case COMMITTED:
+ case STABLE:
+ case PREAPPLIED:
+ // TODO (expected): prove the correctness of this approach
+ if (!PRUNE_TRANSITIVE_DEPENDENCIES || maxStableWrite == null || info.executeAt(txnId).compareTo(maxStableWrite) >= 0)
+ break;
+ case TRANSITIVELY_KNOWN:
+ case INVALID_OR_TRUNCATED:
+ continue;
+ }
+
+ initialValue = map.apply(p1, key, txnId, info.executeAt(txnId), initialValue);
+ }
+ return initialValue;
+ }
+
+ public CommandsForKey update(Command prev, Command next)
+ {
+ InternalStatus newStatus = InternalStatus.from(next.saveStatus());
+ if (newStatus == null)
+ return this;
+
+ TxnId txnId = next.txnId();
+ int pos = Arrays.binarySearch(txnIds, txnId);
+ if (pos < 0)
+ {
+ pos = -1 - pos;
+ if (!newStatus.hasInfo)
+ return insert(pos, txnId, newStatus.asNoInfo);
+
+ return insert(pos, txnId, newStatus, next);
+ }
+ else
+ {
+ // we do not permit multiple versions of the same TxnId, so use the existing one
+ txnId = txnIds[pos];
+
+ // update
+ InternalStatus prevStatus = prev == null ? null : InternalStatus.from(prev.saveStatus());
+ if (newStatus == prevStatus && (!newStatus.hasInfo || next.acceptedOrCommitted().equals(prev.acceptedOrCommitted())))
+ return this;
+
+ @Nonnull Info cur = Invariants.nonNull(infos[pos]);
+ // TODO (required): HACK to permit prev.saveStatus() == SaveStatus.AcceptedInvalidateWithDefinition as we don't always update as keys aren't guaranteed to be provided
+ // fix as soon as we support async updates
+ Invariants.checkState(cur.status.expectMatch == prevStatus || (prev != null && prev.status() == Status.AcceptedInvalidate));
+
+ if (newStatus.hasInfo) return update(pos, txnId, cur, newStatus, next);
+ else return update(pos, cur, newStatus.asNoInfo);
+ }
+ }
+
+ public static boolean needsUpdate(Command prev, Command updated)
+ {
+ if (!updated.txnId().kind().isGloballyVisible())
+ return false;
+
+ SaveStatus prevStatus;
+ Ballot prevAcceptedOrCommitted;
+ if (prev == null)
+ {
+ prevStatus = SaveStatus.NotDefined;
+ prevAcceptedOrCommitted = Ballot.ZERO;
+ }
+ else
+ {
+ prevStatus = prev.saveStatus();
+ prevAcceptedOrCommitted = prev.acceptedOrCommitted();
+ }
+
+ return needsUpdate(prevStatus, prevAcceptedOrCommitted, updated.saveStatus(), updated.acceptedOrCommitted());
+ }
+
+ public static boolean needsUpdate(SaveStatus prevStatus, Ballot prevAcceptedOrCommitted, SaveStatus updatedStatus, Ballot updatedAcceptedOrCommitted)
+ {
+ InternalStatus prev = InternalStatus.from(prevStatus);
+ InternalStatus updated = InternalStatus.from(updatedStatus);
+ return updated != prev || (updated != null && updated.hasInfo && !prevAcceptedOrCommitted.equals(updatedAcceptedOrCommitted));
+ }
+
+ private Timestamp maxStableWriteBefore(Timestamp before)
+ {
+ if (maxStableWrite == null)
+ return null;
+
+ if (maxStableWrite.compareTo(before) < 0)
+ return maxStableWrite;
+
+ return maxStableWriteBefore(before, txnIds, infos);
+ }
+
+ private static Timestamp maxStableWriteBefore(Timestamp before, TxnId[] txnIds, Info[] infos)
+ {
+ int i = Arrays.binarySearch(txnIds, before);
+ if (i < 0) i = -1 -i;
+ Timestamp max = null;
+ while (--i >= 0)
+ {
+ TxnId txnId = txnIds[i];
+ if (txnId.kind() != Write) continue;
+
+ Info info = infos[i];
+ if (!info.status.hasStableDeps()) continue;
+
+ if (info.executeAt == null) max = txnId;
+ else if (info.executeAt.compareTo(before) < 0) max = info.executeAt;
+ else continue;
+
+ break;
+ }
+
+ while (--i >= 0)
+ {
+ Info info = infos[i];
+ TxnId txnId = txnIds[i];
+
+ if (info.getClass() == NoInfo.class) continue;
+ if (!info.status.hasStableDeps()) continue;
+ if (txnId.kind() != Write) continue;
+ if (info.executeAt.compareTo(max) <= 0 || info.executeAt.compareTo(before) >= 0) continue;
+
+ max = info.executeAt;
+ }
+ return max;
+ }
+
+ private CommandsForKey insert(int insertPos, TxnId insertTxnId, InternalStatus newStatus, Command command)
+ {
+ InfoWithAdditions newInfo = computeInsert(insertPos, insertTxnId, newStatus, command);
+ if (newInfo.additionCount == 0)
+ return insert(insertPos, insertTxnId, newInfo.info);
+
+ TxnId[] newTxnIds = new TxnId[txnIds.length + newInfo.additionCount + 1];
+ Info[] newInfos = new Info[newTxnIds.length];
+ insertWithAdditions(insertPos, insertTxnId, newInfo, newTxnIds, newInfos);
+ return update(newTxnIds, newInfos, insertTxnId, newInfo.info);
+ }
+
+ private CommandsForKey update(int updatePos, TxnId txnId, Info prevInfo, InternalStatus newStatus, Command command)
+ {
+ InfoWithAdditions newInfo = computeUpdate(updatePos, txnId, newStatus, command);
+ if (newInfo.additionCount == 0)
+ return update(updatePos, prevInfo, newInfo.info);
+
+ TxnId[] newTxnIds = new TxnId[txnIds.length + newInfo.additionCount];
+ Info[] newInfos = new Info[newTxnIds.length];
+ TxnId updateTxnId = txnIds[updatePos]; // want to reuse the existing TxnId for object identity
+ int newPos = updateWithAdditions(updatePos, txnIds[updatePos], newInfo, newTxnIds, newInfos);
+ if (prevInfo.status.compareTo(COMMITTED) < 0 && newStatus.compareTo(COMMITTED) >= 0)
+ removeMissing(newTxnIds, newInfos, newPos);
+ return update(newTxnIds, newInfos, updateTxnId, newInfo.info);
+ }
+
+ private int updateWithAdditions(int updatePos, TxnId updateTxnId, InfoWithAdditions withInfo, TxnId[] newTxnIds, Info[] newInfos)
+ {
+ return updateOrInsertWithAdditions(updatePos, updatePos, updateTxnId, withInfo, newTxnIds, newInfos);
+ }
+
+ private void insertWithAdditions(int pos, TxnId updateTxnId, InfoWithAdditions withInfo, TxnId[] newTxnIds, Info[] newInfos)
+ {
+ updateOrInsertWithAdditions(pos, -1, updateTxnId, withInfo, newTxnIds, newInfos);
+ }
+
+ private int updateOrInsertWithAdditions(int sourceInsertPos, int sourceUpdatePos, TxnId updateTxnId, InfoWithAdditions withInfo, TxnId[] newTxnIds, Info[] newInfos)
+ {
+ TxnId[] additions = withInfo.additions;
+ int additionCount = withInfo.additionCount;
+ int additionInsertPos = Arrays.binarySearch(additions, 0, additionCount, updateTxnId);
+ additionInsertPos = Invariants.checkArgument(-1 - additionInsertPos, additionInsertPos < 0);
+ int targetInsertPos = sourceInsertPos + additionInsertPos;
+
+ // additions plus the updateTxnId when necessary
+ TxnId[] missingSource = additions;
+ boolean insertSelfMissing = sourceUpdatePos < 0 && withInfo.info.status.compareTo(COMMITTED) < 0;
+
+ // the most recently constructed pure insert missing array, so that it may be reused if possible
+ TxnId[] cachedMissing = null;
+ int i = 0, j = 0, missingCount = 0, missingLimit = additionCount, count = 0;
+ while (i < infos.length)
+ {
+ if (count == targetInsertPos)
+ {
+ newTxnIds[count] = updateTxnId;
+ newInfos[count] = withInfo.info;
+ if (i == sourceUpdatePos) ++i;
+ else if (insertSelfMissing) ++missingCount;
+ ++count;
+ continue;
+ }
+
+ int c = j == additionCount ? -1 : txnIds[i].compareTo(additions[j]);
+ if (c < 0)
+ {
+ TxnId txnId = txnIds[i];
+ Info info = infos[i];
+ if (i == sourceUpdatePos)
+ {
+ info = withInfo.info;
+ }
+ else if (info.status.hasDeps())
+ {
+ Timestamp depsKnownBefore = info.status.depsKnownBefore(txnId, info.executeAt);
+ if (insertSelfMissing && missingSource == additions && (missingCount != j || (depsKnownBefore != txnId && depsKnownBefore.compareTo(updateTxnId) > 0)))
+ {
+ missingSource = mergeMissing(additions, additionCount, updateTxnId, additionInsertPos);
+ ++missingLimit;
+ }
+
+ int to = to(txnId, depsKnownBefore, missingSource, missingCount, missingLimit);
+ if (to > 0)
+ {
+ TxnId[] missing = info.missing == NO_TXNIDS && to == missingCount
+ ? cachedMissing = ensureCachedMissing(missingSource, to, cachedMissing)
+ : linearUnion(info.missing, info.missing.length, missingSource, to, cachedTxnIds());
+ info = info.update(txnId, missing);
+ }
+ }
+ newTxnIds[count] = txnId;
+ newInfos[count] = info;
+ i++;
+ }
+ else if (c > 0)
+ {
+ newTxnIds[count] = additions[j++];
+ ++missingCount;
+ newInfos[count] = TRANSITIVELY_KNOWN.asNoInfo;
+ }
+ else
+ {
+ throw illegalState(txnIds[i] + " should be an insertion, but found match when merging with origin");
+ }
+ count++;
+ }
+
+ if (j < additionCount)
+ {
+ if (count <= targetInsertPos)
+ {
+ int length = targetInsertPos - count;
+ System.arraycopy(additions, j, newTxnIds, count, length);
+ Arrays.fill(newInfos, count, targetInsertPos, TRANSITIVELY_KNOWN.asNoInfo);
+ newTxnIds[targetInsertPos] = updateTxnId;
+ newInfos[targetInsertPos] = withInfo.info;
+ count = targetInsertPos + 1;
+ j = additionInsertPos;
+ }
+ System.arraycopy(additions, j, newTxnIds, count, additionCount - j);
+ Arrays.fill(newInfos, count, count + additionCount - j, TRANSITIVELY_KNOWN.asNoInfo);
+ }
+ else if (count == targetInsertPos)
+ {
+ newTxnIds[targetInsertPos] = updateTxnId;
+ newInfos[targetInsertPos] = withInfo.info;
+ }
+
+ cachedTxnIds().forceDiscard(additions, additionCount);
+ return targetInsertPos;
+ }
+
+ private CommandsForKey update(int pos, Info curInfo, Info newInfo)
+ {
+ if (curInfo == newInfo)
+ return this;
+
+ Info[] newInfos = infos.clone();
+ newInfos[pos] = newInfo;
+ if (curInfo.status.compareTo(COMMITTED) < 0 && newInfo.status.compareTo(COMMITTED) >= 0)
+ removeMissing(txnIds, newInfos, pos);
+ return update(txnIds, newInfos, txnIds[pos], newInfo);
+ }
+
+ /**
+ * Insert a new txnId and info
+ */
+ private CommandsForKey insert(int pos, TxnId newTxnId, Info newInfo)
+ {
+ TxnId[] newTxnIds = new TxnId[txnIds.length + 1];
+ System.arraycopy(txnIds, 0, newTxnIds, 0, pos);
+ newTxnIds[pos] = newTxnId;
+ System.arraycopy(txnIds, pos, newTxnIds, pos + 1, txnIds.length - pos);
+
+ Info[] newInfos = new Info[infos.length + 1];
+ if (newInfo.status.compareTo(COMMITTED) >= 0)
+ {
+ System.arraycopy(infos, 0, newInfos, 0, pos);
+ newInfos[pos] = newInfo;
+ System.arraycopy(infos, pos, newInfos, pos + 1, infos.length - pos);
+ }
+ else
+ {
+ insertInfoAndOneMissing(pos, newTxnId, newInfo, infos, newInfos, newTxnIds, 1);
+ }
+ return update(newTxnIds, newInfos, newTxnId, newInfo);
+ }
+
+ /**
+ * Insert a new txnId and info, then insert the txnId into the missing collection of any command that should have already caused us to witness it
+ */
+ private static void insertInfoAndOneMissing(int insertPos, TxnId txnId, Info newInfo, Info[] oldInfos, Info[] newInfos, TxnId[] newTxnIds, int offsetAfterInsertPos)
+ {
+ TxnId[] oneMissing = null;
+ for (int i = 0 ; i < insertPos ; ++i)
+ {
+ Info oldInfo = oldInfos[i];
+ if (oldInfo.getClass() != NoInfo.class)
+ {
+ Timestamp depsKnownBefore = oldInfo.depsKnownBefore(null);
+ if (depsKnownBefore != null && depsKnownBefore.compareTo(txnId) > 0)
+ {
+ TxnId[] missing;
+ if (oldInfo.missing == NO_TXNIDS)
+ missing = oneMissing = ensureOneMissing(txnId, oneMissing);
+ else
+ missing = SortedArrays.insert(oldInfo.missing, txnId, TxnId[]::new);
+
+ newInfos[i] = Info.create(txnId, oldInfo.status, oldInfo.executeAt, missing);
+ continue;
+ }
+ }
+ newInfos[i] = oldInfo;
+ }
+
+ newInfos[insertPos] = newInfo;
+
+ for (int i = insertPos; i < oldInfos.length ; ++i)
+ {
+ Info oldInfo = oldInfos[i];
+ if (!oldInfo.status.hasDeps())
+ {
+ newInfos[i + offsetAfterInsertPos] = oldInfo;
+ continue;
+ }
+
+ TxnId[] missing;
+ if (oldInfo.missing == NO_TXNIDS)
+ missing = oneMissing = ensureOneMissing(txnId, oneMissing);
+ else
+ missing = SortedArrays.insert(oldInfo.missing, txnId, TxnId[]::new);
+
+ int newIndex = i + offsetAfterInsertPos;
+ newInfos[newIndex] = Info.create(newTxnIds[newIndex], oldInfo.status, oldInfo.executeAt(newTxnIds[newIndex]), missing);
+ }
+ }
+
+ private static void removeMissing(TxnId[] txnIds, Info[] infos, int pos)
+ {
+ TxnId removeTxnId = txnIds[pos];
+ for (int i = 0 ; i < infos.length ; ++i)
+ {
+ Info info = infos[i];
+ if (info.getClass() == NoInfo.class) continue;
+
+ TxnId[] missing = info.missing;
+ if (missing == NO_TXNIDS) continue;
+
+ // linear scan on object identity going to be faster in practice than binary search almost every time
+ int j = 0;
+ while (j < missing.length && missing[j] != removeTxnId) ++j;
+ if (j == missing.length) continue;
+
+ if (missing.length == 1)
+ {
+ missing = NO_TXNIDS;
+ }
+ else
+ {
+ int length = missing.length;
+ TxnId[] newMissing = new TxnId[length - 1];
+ System.arraycopy(missing, 0, newMissing, 0, j);
+ System.arraycopy(missing, j + 1, newMissing, j, length - (1 + j));
+ missing = newMissing;
+ }
+
+ infos[i] = info.update(txnIds[i], missing);
+ }
+ }
+
+ private static TxnId[] ensureOneMissing(TxnId txnId, TxnId[] oneMissing)
+ {
+ return oneMissing != null ? oneMissing : new TxnId[] { txnId };
+ }
+
+ private static TxnId[] ensureCachedMissing(TxnId[] missingIds, int missingIdCount, TxnId[] cachedMissing)
+ {
+ return cachedMissing != null && cachedMissing.length == missingIdCount ? cachedMissing : Arrays.copyOf(missingIds, missingIdCount);
+ }
+
+ private static TxnId[] mergeMissing(TxnId[] additions, int additionCount, TxnId updateTxnId, int additionInsertPos)
+ {
+ TxnId[] missingSource = new TxnId[additionCount + 1];
+ System.arraycopy(additions, 0, missingSource, 0, additionInsertPos);
+ System.arraycopy(additions, additionInsertPos, missingSource, additionInsertPos + 1, additionCount - additionInsertPos);
+ missingSource[additionInsertPos] = updateTxnId;
+ return missingSource;
+ }
+
+ private static int to(TxnId txnId, Timestamp depsKnownBefore, TxnId[] missingSource, int missingCount, int missingLimit)
+ {
+ if (depsKnownBefore == txnId) return missingCount;
+ int to = Arrays.binarySearch(missingSource, 0, missingLimit, depsKnownBefore);
+ if (to < 0) to = -1 - to;
+ return to;
+ }
+
+ static class InfoWithAdditions
+ {
+ final Info info;
+ final TxnId[] additions;
+ final int additionCount;
+
+ InfoWithAdditions(Info info, TxnId[] additions, int additionCount)
+ {
+ this.info = info;
+ this.additions = additions;
+ this.additionCount = additionCount;
+ }
+ }
+
+ private InfoWithAdditions computeInsert(int insertPos, TxnId txnId, InternalStatus newStatus, Command command)
+ {
+ return computeInfoAndAdditions(insertPos, -1, txnId, newStatus, command);
+ }
+
+ private InfoWithAdditions computeUpdate(int updatePos, TxnId txnId, InternalStatus newStatus, Command command)
+ {
+ return computeInfoAndAdditions(updatePos, updatePos, txnId, newStatus, command);
+ }
+
+ private InfoWithAdditions computeInfoAndAdditions(int insertPos, int updatePos, TxnId txnId, InternalStatus newStatus, Command command)
+ {
+ Timestamp executeAt = txnId;
+ if (newStatus.hasInfo)
+ {
+ executeAt = command.executeAt();
+ if (executeAt.equals(txnId)) executeAt = txnId;
+ }
+ Timestamp depsKnownBefore = newStatus.depsKnownBefore(txnId, executeAt);
+ return computeInfoAndAdditions(insertPos, updatePos, txnId, newStatus, executeAt, depsKnownBefore, command.partialDeps().keyDeps.txnIds(key));
+ }
+
+ private InfoWithAdditions computeInfoAndAdditions(int insertPos, int updatePos, TxnId txnId, InternalStatus newStatus, Timestamp executeAt, Timestamp depsKnownBefore, SortedList<TxnId> deps)
+ {
+ int depsKnownBeforePos;
+ if (depsKnownBefore == txnId)
+ {
+ depsKnownBeforePos = insertPos;
+ }
+ else
+ {
+ depsKnownBeforePos = Arrays.binarySearch(txnIds, insertPos, txnIds.length, depsKnownBefore);
+ Invariants.checkState(depsKnownBeforePos < 0);
+ depsKnownBeforePos = -1 - depsKnownBeforePos;
+ }
+
+ TxnId[] additions = NO_TXNIDS, missing = NO_TXNIDS;
+ int additionCount = 0, missingCount = 0;
+
+ int depsIndex = deps.find(redundantBefore);
+ if (depsIndex < 0) depsIndex = -1 - depsIndex;
+ int txnIdsIndex = 0;
+ while (txnIdsIndex < depsKnownBeforePos && depsIndex < deps.size())
+ {
+ TxnId t = txnIds[txnIdsIndex];
+ TxnId d = deps.get(depsIndex);
+ int c = t.compareTo(d);
+ if (c == 0)
+ {
+ ++txnIdsIndex;
+ ++depsIndex;
+ }
+ else if (c < 0)
+ {
+ // we expect to be missing ourselves
+ // we also permit any transaction we have recorded as COMMITTED or later to be missing, as recovery will not need to consult our information
+ if (txnIdsIndex != updatePos && infos[txnIdsIndex].status.compareTo(COMMITTED) < 0)
+ {
+ if (missingCount == missing.length)
+ missing = cachedTxnIds().resize(missing, missingCount, Math.max(8, missingCount * 2));
+ missing[missingCount++] = t;
+ }
+ txnIdsIndex++;
+ }
+ else
+ {
+ if (additionCount >= additions.length)
+ additions = cachedTxnIds().resize(additions, additionCount, Math.max(8, additionCount * 2));
+
+ additions[additionCount++] = d;
+ depsIndex++;
+ }
+ }
+
+ while (txnIdsIndex < depsKnownBeforePos)
+ {
+ if (txnIdsIndex != updatePos && infos[txnIdsIndex].status.compareTo(COMMITTED) < 0)
+ {
+ TxnId t = txnIds[txnIdsIndex];
+ if (missingCount == missing.length)
+ missing = cachedTxnIds().resize(missing, missingCount, Math.max(8, missingCount * 2));
+ missing[missingCount++] = t;
+ }
+ txnIdsIndex++;
+ }
+
+ while (depsIndex < deps.size())
+ {
+ if (additionCount >= additions.length)
+ additions = cachedTxnIds().resize(additions, additionCount, Math.max(8, additionCount * 2));
+ additions[additionCount++] = deps.get(depsIndex++);
+ }
+
+ if (missingCount == 0 && executeAt == txnId)
+ return additionCount == 0 ? newStatus.asNoInfoOrAdditions : new InfoWithAdditions(newStatus.asNoInfo, additions, additionCount);
+
+ return new InfoWithAdditions(Info.create(txnId, newStatus, executeAt, cachedTxnIds().completeAndDiscard(missing, missingCount)), additions, additionCount);
+ }
+
+ private CommandsForKey update(TxnId[] newTxnIds, Info[] newInfos, TxnId updatedTxnId, Info updatedInfo)
+ {
+ Timestamp maxStableWrite = maybeUpdateMaxStableWrite(updatedTxnId, updatedInfo, this.maxStableWrite);
+ return new CommandsForKey(key, redundantBefore, maxStableWrite, newTxnIds, newInfos);
+ }
+
+ private static Timestamp maybeUpdateMaxStableWrite(TxnId txnId, Info info, Timestamp maxStableWrite)
+ {
+ if (!info.status.hasStableDeps() || txnId.kind() != Write)
+ return maxStableWrite;
+
+ Timestamp executeAt = info.executeAt(txnId);
+ if (maxStableWrite != null && maxStableWrite.compareTo(executeAt) >= 0)
+ return maxStableWrite;
+
+ return executeAt;
+ }
+
+ public CommandsForKey withoutRedundant(TxnId redundantBefore)
+ {
+ if (this.redundantBefore.compareTo(redundantBefore) >= 0)
+ return this;
+
+ int pos = insertPos(redundantBefore);
+ if (pos == 0)
+ return new CommandsForKey(key, redundantBefore, maxStableWrite, txnIds, infos);
+
+ TxnId[] newTxnIds = Arrays.copyOfRange(txnIds, pos, txnIds.length);
+ Info[] newInfos = Arrays.copyOfRange(infos, pos, infos.length);
+ for (int i = 0 ; i < newInfos.length ; ++i)
+ {
+ Info info = newInfos[i];
+ if (info.getClass() == NoInfo.class) continue;
+ if (info.missing == NO_TXNIDS) continue;
+ int j = Arrays.binarySearch(info.missing, redundantBefore);
+ if (j < 0) j = -1 - j;
+ if (j <= 0) continue;
+ TxnId[] newMissing = j == info.missing.length ? NO_TXNIDS : Arrays.copyOfRange(info.missing, j, info.missing.length);
+ newInfos[i] = info.update(txnIds[i], newMissing);
+ }
+ return new CommandsForKey(key, redundantBefore, maxStableWrite, newTxnIds, newInfos);
+ }
+
+ public CommandsForKey registerHistorical(TxnId txnId)
+ {
+ int i = Arrays.binarySearch(txnIds, txnId);
+ if (i >= 0)
+ return infos[i].status.compareTo(HISTORICAL) >= 0 ? this : update(i, infos[i], HISTORICAL.asNoInfo);
+
+ return insert(-1 - i, txnId, HISTORICAL.asNoInfo);
+ }
+
+ private int insertPos(Timestamp timestamp)
+ {
+ int i = Arrays.binarySearch(txnIds, timestamp);
+ if (i < 0) i = -1 -i;
+ return i;
+ }
+
+ public TxnId findFirst()
+ {
+ return txnIds.length > 0 ? txnIds[0] : null;
+ }
+
@Override
public boolean equals(Object o)
{
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
CommandsForKey that = (CommandsForKey) o;
- return key.equals(that.key) && commands.equals(that.commands);
+ return Objects.equals(key, that.key)
+ && Objects.equals(redundantBefore, that.redundantBefore)
+ && Arrays.equals(txnIds, that.txnIds)
+ && Arrays.equals(infos, that.infos);
}
@Override
- public final int hashCode()
+ public int hashCode()
{
throw new UnsupportedOperationException();
}
-
- public Key key()
- {
- return key;
- }
-
- @Override
- public CommandTimeseries<?> commands()
- {
- return commands;
- }
-
- public <D> CommandsForKey update(CommandTimeseries.Update<TxnId, D> commands)
- {
- if (commands.isEmpty())
- return this;
-
- return new CommandsForKey(key, commands.apply((CommandTimeseries<D>) this.commands));
- }
-
- public boolean hasRedundant(TxnId redundantBefore)
- {
- return commands.minTimestamp().compareTo(redundantBefore) < 0;
- }
-
- public CommandsForKey withoutRedundant(TxnId redundantBefore)
- {
- return new CommandsForKey(key, (CommandTimeseries) commands.unbuild().removeBefore(redundantBefore).build());
- }
-
- public interface Update {}
-
}
diff --git a/accord-core/src/main/java/accord/impl/CommandsForKeyGroupUpdater.java b/accord-core/src/main/java/accord/impl/CommandsForKeyGroupUpdater.java
deleted file mode 100644
index 63a3fe8..0000000
--- a/accord-core/src/main/java/accord/impl/CommandsForKeyGroupUpdater.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.impl;
-
-import accord.impl.CommandTimeseries.CommandLoader;
-import accord.primitives.Timestamp;
-
-import java.util.Objects;
-
-/**
- * Contains updates for both commands for key sets. One for deps, one for all,
- * and a common set affecting both
- */
-public abstract class CommandsForKeyGroupUpdater<D>
-{
- public static class Mutable<D> extends CommandsForKeyGroupUpdater<D>
- {
- private final CommandsForKeyUpdater.Mutable<D> deps;
- private final CommandsForKeyUpdater.Mutable<D> all;
- private final CommandsForKeyUpdater.Mutable<D> common;
-
- public Mutable(CommandsForKeyUpdater.Mutable<D> deps, CommandsForKeyUpdater.Mutable<D> all, CommandsForKeyUpdater.Mutable<D> common)
- {
- this.deps = deps;
- this.all = all;
- this.common = common;
- }
-
- public Mutable(CommandLoader<D> loader)
- {
- this(new CommandsForKeyUpdater.Mutable<>(loader),
- new CommandsForKeyUpdater.Mutable<>(loader),
- new CommandsForKeyUpdater.Mutable<>(loader));
- }
-
- @Override
- public CommandsForKeyUpdater.Mutable<D> deps()
- {
- return deps;
- }
-
- @Override
- public CommandsForKeyUpdater.Mutable<D> all()
- {
- return all;
- }
-
- @Override
- public CommandsForKeyUpdater.Mutable<D> common()
- {
- return common;
- }
-
- public Immutable<D> toImmutable()
- {
- return new Immutable<>(deps.toImmutable(), all.toImmutable(), common.toImmutable());
- }
- }
-
- public static class Immutable<D> extends CommandsForKeyGroupUpdater<D>
- {
- private final CommandsForKeyUpdater.Immutable<D> deps;
- private final CommandsForKeyUpdater.Immutable<D> all;
- private final CommandsForKeyUpdater.Immutable<D> common;
-
- public interface Factory<D, T extends Immutable<D>>
- {
- T create(CommandsForKeyUpdater.Immutable<D> deps, CommandsForKeyUpdater.Immutable<D> all, CommandsForKeyUpdater.Immutable<D> common);
- }
-
- public static <T> Factory<T, Immutable<T>> getFactory()
- {
- return Immutable::new;
- }
-
- public Immutable(CommandsForKeyUpdater.Immutable<D> deps, CommandsForKeyUpdater.Immutable<D> all, CommandsForKeyUpdater.Immutable<D> common)
- {
- this.deps = deps;
- this.all = all;
- this.common = common;
- }
-
- @Override
- public String toString()
- {
- return "Immutable{" +
- "deps=" + deps +
- ", all=" + all +
- ", common=" + common +
- '}';
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- Immutable<?> immutable = (Immutable<?>) o;
- return Objects.equals(deps, immutable.deps) && Objects.equals(all, immutable.all) && Objects.equals(common, immutable.common);
- }
-
- @Override
- public int hashCode()
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public CommandsForKeyUpdater.Immutable<D> deps()
- {
- return deps;
- }
-
- @Override
- public CommandsForKeyUpdater.Immutable<D> all()
- {
- return all;
- }
-
- @Override
- public CommandsForKeyUpdater.Immutable<D> common()
- {
- return common;
- }
-
- public Mutable<D> toMutable(CommandLoader<D> loader)
- {
- return new Mutable<>(deps.toMutable(loader), all.toMutable(loader), common.toMutable(loader));
- }
-
- private static <T extends Timestamp, D> void mergeCommon(CommandTimeseries.ImmutableUpdate<T, D> newCommon,
- CommandTimeseries.MutableUpdate<T, D> mergedCommon,
- CommandTimeseries.MutableUpdate<T, D> mergedDeps,
- CommandTimeseries.MutableUpdate<T, D> mergedAll)
- {
- newCommon.forEachWrite((k, v) -> {
- mergedCommon.addWriteUnsafe(k, v);
- mergedDeps.removeKeyUnsafe(k);
- mergedAll.removeKeyUnsafe(k);
- });
- newCommon.forEachDelete(k -> {
- mergedCommon.remove(k);
- mergedDeps.removeKeyUnsafe(k);
- mergedAll.removeKeyUnsafe(k);
- });
- }
-
- private static <T extends Timestamp, D> void mergeSpecific(CommandTimeseries.ImmutableUpdate<T, D> newSpecific,
- CommandTimeseries.MutableUpdate<T, D> mergedSpecific, CommandTimeseries.MutableUpdate<T, D> mergedOther, CommandTimeseries.MutableUpdate<T, D> mergedCommon)
- {
- newSpecific.forEachWrite((k, v) -> {
- mergedSpecific.addWriteUnsafe(k, v);
- mergedCommon.transferKeyTo(k, mergedOther);
- });
-
- newSpecific.forEachDelete(k -> {
- mergedSpecific.remove(k);
- mergedCommon.transferKeyTo(k, mergedOther);
- });
- }
-
- public static <D, T extends Immutable<D>> T merge(T current, T update, Factory<D, T> factory)
- {
- if (current == null || current.isEmpty())
- return update;
-
- if (update == null || update.isEmpty())
- return current;
-
- Mutable<D> merged = current.toMutable(null);
-
- mergeCommon(update.common().commands(), merged.common.commands(), merged.deps.commands(), merged.all.commands());
-
- mergeSpecific(update.deps().commands(), merged.deps.commands(), merged.all.commands(), merged.common.commands());
- mergeSpecific(update.all().commands(), merged.all.commands(), merged.deps.commands(), merged.common.commands());
-
- return factory.create(merged.deps.toImmutable(), merged.all.toImmutable(), merged.common.toImmutable());
- }
- }
-
- public abstract CommandsForKeyUpdater<D> deps();
-
- public abstract CommandsForKeyUpdater<D> all();
-
- /**
- * Update the deps and all CFK groups.
- * In case conflicting updates have been applied to the common udpater and a specific one, the specific one should take precedence
- */
- public abstract CommandsForKeyUpdater<D> common();
-
- public boolean isEmpty()
- {
- return deps().isEmpty() && all().isEmpty() && common().isEmpty();
- }
-
- public CommandsForKey applyToDeps(CommandsForKey current)
- {
- current = common().apply(current);
- current = deps().apply(current);
-
- return current;
- }
-
- public CommandsForKey applyToAll(CommandsForKey current)
- {
- current = common().apply(current);
- current = all().apply(current);
-
- return current;
- }
-}
diff --git a/accord-core/src/main/java/accord/impl/CommandsForKeyUpdater.java b/accord-core/src/main/java/accord/impl/CommandsForKeyUpdater.java
deleted file mode 100644
index c46c327..0000000
--- a/accord-core/src/main/java/accord/impl/CommandsForKeyUpdater.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.impl;
-
-import accord.impl.CommandTimeseries.CommandLoader;
-import accord.primitives.TxnId;
-
-import java.util.Objects;
-
-/**
- * Contains updates for a single commands for key dataset (ie: deps OR all)
- * @param <D>
- */
-public abstract class CommandsForKeyUpdater<D>
-{
- public abstract CommandTimeseries.Update<TxnId, D> commands();
-
- public static class Mutable<D> extends CommandsForKeyUpdater<D>
- {
- private final CommandTimeseries.MutableUpdate<TxnId, D> commands;
-
- Mutable(CommandTimeseries.MutableUpdate<TxnId, D> commands)
- {
- this.commands = commands;
- }
-
- public Mutable(CommandLoader<D> loader)
- {
- this(new CommandTimeseries.MutableUpdate<>(loader));
- }
-
- @Override public CommandTimeseries.MutableUpdate<TxnId, D> commands() { return commands; }
- public Immutable<D> toImmutable()
- {
- return new Immutable<>(commands.toImmutable());
- }
- }
-
- public static class Immutable<D> extends CommandsForKeyUpdater<D>
- {
- private static final Immutable<Object> EMPTY = new Immutable<>(CommandTimeseries.ImmutableUpdate.empty());
- private final CommandTimeseries.ImmutableUpdate<TxnId, D> commands;
-
- public Immutable(CommandTimeseries.ImmutableUpdate<TxnId, D> commands)
- {
- this.commands = commands;
- }
-
- @Override
- public String toString()
- {
- return "Immutable{" +
- "commands=" + commands +
- '}';
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- Immutable<?> immutable = (Immutable<?>) o;
- return Objects.equals(commands, immutable.commands);
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(commands);
- }
-
- public static <D> Immutable<D> empty()
- {
- return (Immutable<D>) EMPTY;
- }
-
- @Override public CommandTimeseries.ImmutableUpdate<TxnId, D> commands() { return commands; }
-
- public Mutable<D> toMutable(CommandLoader<D> loader)
- {
- return new Mutable<>(commands.toMutable(loader));
- }
- }
-
- public int totalChanges()
- {
- return commands().numChanges();
- }
-
- public boolean isEmpty()
- {
- return commands().isEmpty();
- }
-
- public CommandsForKey apply(CommandsForKey current)
- {
- if (commands().isEmpty())
- return current;
- return current.update(commands());
- }
-}
diff --git a/accord-core/src/main/java/accord/impl/CommandsForKeys.java b/accord-core/src/main/java/accord/impl/CommandsForKeys.java
index 56d5b9b..3ab2b7b 100644
--- a/accord-core/src/main/java/accord/impl/CommandsForKeys.java
+++ b/accord-core/src/main/java/accord/impl/CommandsForKeys.java
@@ -19,16 +19,13 @@
package accord.impl;
import accord.api.VisibleForImplementation;
-import accord.local.Command;
import accord.local.CommandStore;
import accord.primitives.RoutableKey;
import accord.primitives.Timestamp;
import accord.primitives.TxnId;
-import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static accord.local.Command.NotDefined.uninitialised;
public class CommandsForKeys
{
@@ -36,77 +33,10 @@
private CommandsForKeys() {}
- @VisibleForTesting
- @VisibleForImplementation
- public static TimestampsForKey updateMax(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey key, Timestamp timestamp)
+ public static void registerNotWitnessed(AbstractSafeCommandStore<?,?,?> safeStore, RoutableKey key, TxnId txnId)
{
- SafeTimestampsForKey tfk = safeStore.timestampsForKey(key);
- return tfk.updateMax(timestamp);
- }
-
- public static Command.DurableAndIdempotentListener registerCommand(SafeTimestampsForKey timestamps, SafeCommandsForKey.Update<?,?> update, Command command)
- {
-
- update.common().commands().add(command.txnId(), command);
-
- timestamps.updateMax(command.executeAt());
-
- return new CommandsForKey.Listener(timestamps.key());
- }
-
- public static Command.DurableAndIdempotentListener registerCommand(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey key, Command command)
- {
- return registerCommand(safeStore.timestampsForKey(key), safeStore.getOrCreateCommandsForKeyUpdate(key), command);
- }
-
- public static void registerNotWitnessed(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey key, TxnId txnId)
- {
- SafeTimestampsForKey tfk = safeStore.timestampsForKey(key);
- SafeCommandsForKey cfk = safeStore.depsCommandsForKey(key);
-
- // FIXME: should be the recovery commands
- if (cfk.current().commands().commands.containsKey(txnId))
- return;
-
- tfk.updateMax(txnId);
- SafeCommandsForKey.Update<?,?> update = safeStore.getOrCreateCommandsForKeyUpdate(key);
-
- update.common().commands().add(txnId, uninitialised(txnId));
- }
-
- public static void listenerUpdate(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey listenerKey, Command command)
- {
- if (logger.isTraceEnabled())
- logger.trace("[{}]: updating as listener in response to change on {} with status {} ({})",
- listenerKey, command.txnId(), command.status(), command);
-
- SafeTimestampsForKey tfk = safeStore.timestampsForKey(listenerKey);
- SafeCommandsForKey.Update<?,?> update = safeStore.getOrCreateCommandsForKeyUpdate(listenerKey);
-
- // add/remove the command on every listener update to avoid
- // special denormalization handling in Cassandra
- switch (command.status())
- {
- default: throw new AssertionError();
- case PreAccepted:
- case NotDefined:
- case Accepted:
- case AcceptedInvalidate:
- case PreCommitted:
- case Applied:
- case PreApplied:
- case Committed:
- case Stable:
- case ReadyToExecute:
- update.common().commands().add(command.txnId(), command);
- break;
- case Invalidated:
- update.common().commands().remove(command.txnId());
- case Truncated:
- break;
- }
-
- tfk.updateMax(command.executeAt());
+ SafeCommandsForKey cfk = safeStore.commandsForKey(key);
+ cfk.registerHistorical(txnId);
}
public static TimestampsForKey updateLastExecutionTimestamps(CommandStore commandStore, SafeTimestampsForKey tfk, Timestamp executeAt, boolean isForWriteTxn)
@@ -146,7 +76,7 @@
}
@VisibleForImplementation
- public static <D> TimestampsForKey updateLastExecutionTimestamps(AbstractSafeCommandStore<?,?,?,?> safeStore, RoutableKey key, Timestamp executeAt, boolean isForWriteTxn)
+ public static <D> TimestampsForKey updateLastExecutionTimestamps(AbstractSafeCommandStore<?,?,?> safeStore, RoutableKey key, Timestamp executeAt, boolean isForWriteTxn)
{
return updateLastExecutionTimestamps(safeStore.commandStore(), safeStore.timestampsForKey(key), executeAt, isForWriteTxn);
}
diff --git a/accord-core/src/main/java/accord/impl/CommandsSummary.java b/accord-core/src/main/java/accord/impl/CommandsSummary.java
new file mode 100644
index 0000000..7793c6e
--- /dev/null
+++ b/accord-core/src/main/java/accord/impl/CommandsSummary.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.impl;
+
+import accord.local.SafeCommandStore.CommandFunction;
+import accord.local.SafeCommandStore.TestDep;
+import accord.local.SafeCommandStore.TestStartedAt;
+import accord.local.SafeCommandStore.TestStatus;
+import accord.primitives.Timestamp;
+import accord.primitives.Txn.Kind.Kinds;
+import accord.primitives.TxnId;
+
+public interface CommandsSummary
+{
+ <P1, T> T mapReduceFull(TxnId testTxnId,
+ Kinds testKind,
+ TestStartedAt testStartedAt,
+ TestDep testDep,
+ TestStatus testStatus,
+ CommandFunction<P1, T, T> map, P1 p1, T initialValue);
+
+ <P1, T> T mapReduceActive(Timestamp startedBefore,
+ Kinds testKind,
+ CommandFunction<P1, T, T> map, P1 p1, T initialValue);
+}
diff --git a/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java b/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java
index e4db999..df39788 100644
--- a/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java
+++ b/accord-core/src/main/java/accord/impl/CoordinateDurabilityScheduling.java
@@ -29,9 +29,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import accord.api.Scheduler;
import accord.coordinate.CoordinateGloballyDurable;
import accord.coordinate.CoordinateShardDurable;
import accord.coordinate.CoordinateSyncPoint;
+import accord.coordinate.ExecuteSyncPoint.SyncPointErased;
import accord.local.Node;
import accord.local.ShardDistributor;
import accord.primitives.Range;
@@ -69,7 +71,6 @@
* The work for CoordinateShardDurable is further subdivided where each subrange a node operates on is divided a fixed
* number of times and then processed one at a time with a fixed wait between them.
*
- * // TODO review will the scheduler shut down on its own or do the individual tasks need to be canceled manually?
* // TODO (expected): cap number of coordinations we can have in flight at once
* Didn't go with recurring because it doesn't play well with async execution of these tasks
*/
@@ -78,6 +79,7 @@
private static final Logger logger = LoggerFactory.getLogger(CoordinateDurabilityScheduling.class);
private final Node node;
+ private Scheduler.Scheduled scheduled;
/*
* In each round at each node wait this amount of time between initiating new CoordinateShardDurable
@@ -156,26 +158,22 @@
/**
* Schedule regular invocations of CoordinateShardDurable and CoordinateGloballyDurable
*/
- public void start()
+ public synchronized void start()
{
Invariants.checkState(!stop); // cannot currently restart safely
long nowMicros = node.unix(MICROSECONDS);
prevShardSyncTimeMicros = nowMicros;
setNextGlobalSyncTime(nowMicros);
- requeue();
+ scheduled = node.scheduler().recurring(this::run, frequencyMicros, MICROSECONDS);
}
public void stop()
{
+ if (scheduled != null)
+ scheduled.cancel();
stop = true;
}
- private void requeue()
- {
- if (!stop)
- node.scheduler().once(this::run, frequencyMicros, MICROSECONDS);
- }
-
/**
* Schedule the first CoordinateShardDurable execution for the current round. Sub-steps will be scheduled after
* each sub-step completes, and once all are completed scheduleCoordinateShardDurable is called again.
@@ -185,35 +183,28 @@
if (stop)
return;
- try
+ updateTopology();
+ if (currentGlobalTopology == null || currentGlobalTopology.size() == 0)
+ return;
+
+ long nowMicros = node.unix(MICROSECONDS);
+ if (nextGlobalSyncTimeMicros <= nowMicros)
{
- updateTopology();
- if (currentGlobalTopology.size() == 0)
- return;
-
- long nowMicros = node.unix(MICROSECONDS);
- if (nextGlobalSyncTimeMicros <= nowMicros)
- {
- startGlobalSync();
- setNextGlobalSyncTime(nowMicros);
- }
-
- List<Ranges> coordinate = rangesToShardSync(nowMicros);
- prevShardSyncTimeMicros = nowMicros;
- if (coordinate.isEmpty())
- {
- logger.trace("Nothing pending in schedule for time slot at {}", nowMicros);
- return;
- }
-
- logger.trace("Scheduling CoordinateShardDurable for {} at {}", coordinate, nowMicros);
- for (Ranges ranges : coordinate)
- startShardSync(ranges);
+ startGlobalSync();
+ setNextGlobalSyncTime(nowMicros);
}
- finally
+
+ List<Ranges> coordinate = rangesToShardSync(nowMicros);
+ prevShardSyncTimeMicros = nowMicros;
+ if (coordinate.isEmpty())
{
- requeue();
+ logger.trace("Nothing pending in schedule for time slot at {}", nowMicros);
+ return;
}
+
+ logger.trace("Scheduling CoordinateShardDurable for {} at {}", coordinate, nowMicros);
+ for (Ranges ranges : coordinate)
+ startShardSync(ranges);
}
/**
@@ -238,7 +229,7 @@
node.commandStores().any().execute(() -> {
CoordinateShardDurable.coordinate(node, exclusiveSyncPoint)
.addCallback((success, fail) -> {
- if (fail != null)
+ if (fail != null && fail.getClass() != SyncPointErased.class)
{
logger.trace("Exception coordinating local shard durability, will retry immediately", fail);
coordinateShardDurableAfterExclusiveSyncPoint(node, exclusiveSyncPoint);
@@ -309,7 +300,6 @@
return;
currentGlobalTopology = latestGlobal;
-
List<Node.Id> ids = new ArrayList<>(latestGlobal.nodes());
Collections.sort(ids);
globalIndex = ids.indexOf(node.id());
diff --git a/accord-core/src/main/java/accord/impl/DepsBuilder.java b/accord-core/src/main/java/accord/impl/DepsBuilder.java
deleted file mode 100644
index 75e17d1..0000000
--- a/accord-core/src/main/java/accord/impl/DepsBuilder.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.impl;
-
-import accord.local.SafeCommandStore;
-import accord.local.Status;
-import accord.primitives.*;
-
-import java.util.*;
-import java.util.function.Consumer;
-
-public class DepsBuilder
-{
- private static boolean CAN_PRUNE = Boolean.getBoolean("accord.can_prune_deps");
- private final TxnId subject;
- private final Map<Seekable, Builder> builders = new HashMap<>();
-
- public static void setCanPruneUnsafe(boolean value)
- {
- CAN_PRUNE = value;
- }
-
- public static boolean getCanPrune()
- {
- return CAN_PRUNE;
- }
-
- public DepsBuilder(TxnId subject)
- {
- this.subject = subject;
- }
-
- private static class TxnInfo
- {
- private final TxnId txnId;
- private final Status status;
- private final Timestamp executeAt;
- private final List<TxnId> depsIds;
- private boolean required = false;
-
- public TxnInfo(TxnId txnId, Status status, Timestamp executeAt, List<TxnId> depsIds)
- {
- this.txnId = txnId;
- this.status = status;
- this.executeAt = executeAt;
- this.depsIds = depsIds;
- }
- }
-
- /**
- * To remove transitive dependencies, we filter out commands that exist in the deps list
- * We need commands that are not superseded by other commands with later execution times
- */
- public static class Builder
- {
- private final Map<TxnId, TxnInfo> commands = new HashMap<>();
- private final Map<TxnId, Set<TxnId>> supportingDeps = new HashMap<>();
-
- public void add(TxnId txnId, Status status, Timestamp executeAt, List<TxnId> depsIds)
- {
- commands.put(txnId, new TxnInfo(txnId, status, executeAt, depsIds));
- if (depsIds != null)
- depsIds.forEach(id -> supportingDeps.computeIfAbsent(id, i -> new HashSet<>()).add(txnId));
- }
-
- private void markRequired(TxnInfo info)
- {
- if (info.required)
- return;
-
- info.required = true;
-
- Set<TxnId> supportingIds = supportingDeps.get(info.txnId);
- if (supportingIds == null || supportingIds.isEmpty())
- return;
-
- for (TxnId supportingId : supportingIds)
- {
- TxnInfo supporting = commands.get(supportingId);
- if (supporting == null)
- continue;
-
- markRequired(supporting);
- }
- }
-
- private boolean isRequired(TxnInfo info)
- {
- if (!CAN_PRUNE || info.required)
- return true;
-
- if (!info.status.hasBeen(Status.Applied))
- return true;
-
- Set<TxnId> supportingIds = supportingDeps.get(info.txnId);
- if (supportingIds == null || supportingIds.isEmpty())
- return true;
-
- for (TxnId supportingId : supportingIds)
- {
- TxnInfo supporting = commands.get(supportingId);
-
- // TODO: check for eviction of txnId and skip if it's evicted
-
- if (!supporting.status.hasBeen(Status.Applied))
- return true;
-
- if (supporting.executeAt.compareTo(info.executeAt) < 0)
- return true;
-
- // FIXME: is this actually needed?
- // let execution order strongly connected components
- if (info.depsIds.contains(supportingId))
- return true;
- }
-
- return false;
- }
-
- private void preprocess(TxnInfo info)
- {
- if (isRequired(info))
- markRequired(info);
- }
-
- private void preprocess()
- {
- commands.values().forEach(this::preprocess);
- }
-
- public <T> void build(SafeCommandStore safeStore, Seekable seekable, Consumer<TxnId> consumer)
- {
- preprocess();
- commands.forEach(((txnId, info) -> {
- if (info.required)
- consumer.accept(txnId);
- else
- safeStore.removeCommandFromSeekableDeps(seekable, txnId, info.executeAt, info.status);
- }));
- }
- }
-
- public void add(TxnId txnId, Seekable seekable, Status status, Timestamp executeAt, List<TxnId> depsIds)
- {
- // don't add self to deps
- if (txnId.equals(subject))
- return;
-
- builders.computeIfAbsent(seekable, s -> new Builder()).add(txnId, status, executeAt, depsIds);
- }
-
- public PartialDeps buildPartialDeps(SafeCommandStore safeStore, Ranges ranges)
- {
- try (PartialDeps.Builder result = new PartialDeps.Builder(ranges);)
- {
- builders.forEach((seekable, builder) -> builder.build(safeStore, seekable, txnId -> result.add(seekable, txnId)));
- return result.build();
- }
- }
-}
diff --git a/accord-core/src/main/java/accord/impl/DomainCommands.java b/accord-core/src/main/java/accord/impl/DomainCommands.java
deleted file mode 100644
index ee1423f..0000000
--- a/accord-core/src/main/java/accord/impl/DomainCommands.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.impl;
-
-public interface DomainCommands
-{
- enum Kind { COORDINATION, RECOVERY }
-
- CommandTimeseries<?> commands();
-}
diff --git a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
index 42c043c..9b49cdf 100644
--- a/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
+++ b/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
@@ -26,7 +26,6 @@
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
-import java.util.Objects;
import java.util.Queue;
import java.util.Set;
import java.util.TreeMap;
@@ -35,15 +34,16 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
import java.util.function.Consumer;
import java.util.function.Function;
-import java.util.function.Predicate;
+import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import com.google.common.annotations.VisibleForTesting;
import accord.local.*;
-import accord.utils.TriFunction;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -51,7 +51,6 @@
import accord.api.DataStore;
import accord.api.Key;
import accord.api.ProgressLog;
-import accord.impl.CommandTimeseries.CommandLoader;
import accord.local.CommandStores.RangesForEpoch;
import accord.primitives.AbstractKeys;
import accord.primitives.Deps;
@@ -60,6 +59,7 @@
import accord.primitives.Range;
import accord.primitives.Ranges;
import accord.primitives.Routable;
+import accord.primitives.Routable.Domain;
import accord.primitives.RoutableKey;
import accord.primitives.Routables;
import accord.primitives.Seekable;
@@ -71,12 +71,13 @@
import accord.utils.async.AsyncChain;
import accord.utils.async.AsyncChains;
-import static accord.local.Command.NotDefined.uninitialised;
-
import static accord.local.SafeCommandStore.TestDep.ANY_DEPS;
import static accord.local.SafeCommandStore.TestDep.WITH;
+import static accord.local.SafeCommandStore.TestStatus.ANY_STATUS;
+import static accord.local.SafeCommandStore.TestStartedAt.STARTED_BEFORE;
import static accord.local.SaveStatus.Applying;
import static accord.local.SaveStatus.ReadyToExecute;
+import static accord.local.Status.Applied;
import static accord.local.Status.Stable;
import static accord.local.Status.Truncated;
import static accord.local.Status.NotDefined;
@@ -92,8 +93,7 @@
final NavigableMap<TxnId, GlobalCommand> commands = new TreeMap<>();
final NavigableMap<Timestamp, GlobalCommand> commandsByExecuteAt = new TreeMap<>();
private final NavigableMap<RoutableKey, GlobalTimestampsForKey> timestampsForKey = new TreeMap<>();
- private final NavigableMap<RoutableKey, GlobalCommandsForKey> depsCommandsForKey = new TreeMap<>();
- private final NavigableMap<RoutableKey, GlobalCommandsForKey> allCommandsForKey = new TreeMap<>();
+ private final NavigableMap<RoutableKey, GlobalCommandsForKey> commandsForKey = new TreeMap<>();
// TODO (find library, efficiency): this is obviously super inefficient, need some range map
private final TreeMap<TxnId, RangeCommand> rangeCommands = new TreeMap<>();
@@ -148,7 +148,6 @@
{
if (CHECK_DEPENDENCY_INVARIANTS)
{
-
newGlobalCommand.addListener(new Command.TransientListener()
{
@Override
@@ -197,7 +196,7 @@
if (prev.txnId().isWrite())
participants = participants.subtract(intersectingParticipants);
- };
+ }
}
}
@@ -229,34 +228,19 @@
return commands.containsKey(txnId);
}
- public GlobalCommandsForKey depsCommandsForKeyIfPresent(Key key)
+ public GlobalCommandsForKey commandsForKeyIfPresent(Key key)
{
- return depsCommandsForKey.get(key);
+ return commandsForKey.get(key);
}
- public GlobalCommandsForKey depsCommandsForKey(Key key)
+ public GlobalCommandsForKey commandsForKey(Key key)
{
- return depsCommandsForKey.computeIfAbsent(key, GlobalCommandsForKey::new);
+ return commandsForKey.computeIfAbsent(key, GlobalCommandsForKey::new);
}
- public boolean hasDepsCommandsForKey(Key key)
+ public boolean hasCommandsForKey(Key key)
{
- return depsCommandsForKey.containsKey(key);
- }
-
- public GlobalCommandsForKey allCommandsForKeyIfPresent(Key key)
- {
- return allCommandsForKey.get(key);
- }
-
- public GlobalCommandsForKey allCommandsForKey(Key key)
- {
- return allCommandsForKey.computeIfAbsent(key, GlobalCommandsForKey::new);
- }
-
- public boolean hasAllCommandsForKey(Key key)
- {
- return allCommandsForKey.containsKey(key);
+ return commandsForKey.containsKey(key);
}
public GlobalTimestampsForKey timestampsForKey(Key key)
@@ -269,60 +253,7 @@
return timestampsForKey.get(key);
}
- public CommonAttributes register(InMemorySafeStore safeStore, Seekables<?, ?> keysOrRanges, Ranges slice, SafeCommand command, CommonAttributes attrs)
- {
- switch (keysOrRanges.domain())
- {
- default: throw new AssertionError();
- case Key:
- CommonAttributes.Mutable mutable = attrs.mutable();
- forEach(keysOrRanges, slice, key -> {
- Command.DurableAndIdempotentListener listener = CommandsForKeys.registerCommand(safeStore, key, command.current());
- mutable.addListener(listener);
- });
- return mutable;
- case Range:
- // TODO (required): confirm we slice to owned ranges on C*
- rangeCommands.computeIfAbsent(command.txnId(), ignore -> new RangeCommand(commands.get(command.txnId())))
- .update(((Ranges)keysOrRanges).slice(slice, Minimal));
- }
- return attrs;
- }
-
- public CommonAttributes register(InMemorySafeStore safeStore, Seekable keyOrRange, Ranges slice, SafeCommand command, CommonAttributes attrs)
- {
- switch (keyOrRange.domain())
- {
- default: throw new AssertionError();
- case Key:
- CommonAttributes.Mutable mutable = attrs.mutable();
- forEach(keyOrRange, slice, key -> {
- Command.DurableAndIdempotentListener listener = CommandsForKeys.registerCommand(safeStore, key, command.current());
- mutable.addListener(listener);
- });
- return mutable;
- case Range:
- // TODO (required): confirm we slice to owned ranges on C*
- rangeCommands.computeIfAbsent(command.txnId(), ignore -> new RangeCommand(commands.get(command.txnId())))
- .update(Ranges.of((Range)keyOrRange).slice(slice, Minimal));
- }
- return attrs;
- }
-
- private NavigableMap<RoutableKey, GlobalCommandsForKey> globalCommandsMapFor(KeyHistory keyHistory)
- {
- switch (keyHistory)
- {
- case DEPS:
- return depsCommandsForKey;
- case ALL:
- return allCommandsForKey;
- default:
- throw new IllegalStateException(keyHistory + " does not have a commands map");
- }
- }
-
- private <O> O mapReduceForKey(InMemorySafeStore safeStore, Routables<?> keysOrRanges, Ranges slice, KeyHistory keyHistory, TriFunction<TimestampsForKey, CommandsForKey, O, O> map, O accumulate, Predicate<? super O> terminate)
+ private <O> O mapReduceForKey(InMemorySafeStore safeStore, Routables<?> keysOrRanges, Ranges slice, BiFunction<CommandsForKey, O, O> map, O accumulate)
{
switch (keysOrRanges.domain()) {
default:
@@ -332,27 +263,11 @@
for (Key key : keys)
{
if (!slice.contains(key)) continue;
- SafeTimestampsForKey timestamps = safeStore.timestampsIfLoadedAndInitialised(key);
- if (timestamps.current() == null)
+ CommandsForKey commands = safeStore.commandsIfLoadedAndInitialised(key).current();
+ if (commands == null)
continue;
- CommandsForKey commands = null;
- switch (keyHistory)
- {
- case DEPS:
- commands = safeStore.depsCommandsIfLoadedAndInitialised(key).current();
- break;
- case ALL:
- commands = safeStore.allCommandsIfLoadedAndInitialised(key).current();
- break;
- }
-
- if (commands == null && keyHistory != KeyHistory.NONE)
- continue;
-
- accumulate = map.apply(timestamps.current(), commands, accumulate);
- if (terminate.test(accumulate))
- return accumulate;
+ accumulate = map.apply(commands, accumulate);
}
break;
case Range:
@@ -362,28 +277,13 @@
{
// TODO (required): this method should fail if it requires more info than available
// TODO (required): I don't think this can possibly work in C*, as we don't know which timestampsForKey we need
- NavigableMap<RoutableKey, GlobalCommandsForKey> commandsMap = keyHistory.isNone() ? null : globalCommandsMapFor(keyHistory);
- for (Map.Entry<RoutableKey, GlobalTimestampsForKey> entry : timestampsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive()).entrySet())
+ for (Map.Entry<RoutableKey, GlobalCommandsForKey> entry : commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive()).entrySet())
{
- RoutableKey key = entry.getKey();
- TimestampsForKey timestamps = entry.getValue().value();
-
- GlobalCommandsForKey globalCommands = keyHistory.isNone() ? null : commandsMap.get(key);
- if (timestamps == null)
- {
- Invariants.checkState(globalCommands == null || globalCommands.value() == null);
+ GlobalCommandsForKey globalCommands = entry.getValue();
+ CommandsForKey commands = globalCommands.value();
+ if (commands == null)
continue;
- }
-
- // TODO (required): what if consumer wants timestamps independently? not safe, should declare separately
- if (globalCommands == null && !keyHistory.isNone())
- continue;
-
- CommandsForKey commands = keyHistory.isNone() ? null : Invariants.nonNull(globalCommands.value());
-
- accumulate = map.apply(timestamps, commands, accumulate);
- if (terminate.test(accumulate))
- return accumulate;
+ accumulate = map.apply(commands, accumulate);
}
}
}
@@ -402,10 +302,8 @@
case Range:
Ranges ranges = (Ranges) keysOrRanges;
ranges.slice(slice).forEach(range -> {
- depsCommandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive())
- .keySet().forEach(forEach);
- allCommandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive())
- .keySet().forEach(forEach);
+ commandsForKey.subMap(range.start(), range.startInclusive(), range.end(), range.endInclusive())
+ .keySet().forEach(forEach);
});
}
}
@@ -423,10 +321,8 @@
case Range:
Range range = (Range) keyOrRange;
Ranges.of(range).slice(slice).forEach(r -> {
- depsCommandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive())
- .keySet().forEach(forEach);
- allCommandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive())
- .keySet().forEach(forEach);
+ commandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive())
+ .keySet().forEach(forEach);
});
}
}
@@ -457,11 +353,7 @@
}
});
ranges.forEach(r -> {
- depsCommandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive()).values().forEach(forKey -> {
- if (!forKey.isEmpty())
- forKey.value(forKey.value().withoutRedundant(syncId));
- });
- allCommandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive()).values().forEach(forKey -> {
+ commandsForKey.subMap(r.start(), r.startInclusive(), r.end(), r.endInclusive()).values().forEach(forKey -> {
if (!forKey.isEmpty())
forKey.value(forKey.value().withoutRedundant(syncId));
});
@@ -471,35 +363,24 @@
protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges,
Map<TxnId, InMemorySafeCommand> commands,
Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey,
- Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommandsForKeys,
- Map<RoutableKey, InMemorySafeCommandsForKey> completeCommandsForKeys)
+ Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKeys)
{
- return new InMemorySafeStore(this, ranges, context, commands, timestampsForKey, minimalCommandsForKeys, completeCommandsForKeys);
+ return new InMemorySafeStore(this, ranges, context, commands, timestampsForKey, commandsForKeys);
}
private void loadCommandsForKey(RoutableKey key,
KeyHistory keyHistory,
Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey,
- Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommands,
- Map<RoutableKey, InMemorySafeCommandsForKey> completeCommands)
+ Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey)
{
- switch (keyHistory)
- {
- case ALL:
- // loading recovery commands implicitly loads deps
- completeCommands.put(key, allCommandsForKey((Key) key).createSafeReference());
- case DEPS:
- minimalCommands.put(key, depsCommandsForKey((Key) key).createSafeReference());
- break;
- }
+ commandsForKey.put(key, commandsForKey((Key) key).createSafeReference());
timestampsForKey.put(key, timestampsForKey((Key) key).createSafeReference());
}
protected final InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges)
{
Map<TxnId, InMemorySafeCommand> commands = new HashMap<>();
- Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommands = new HashMap<>();
- Map<RoutableKey, InMemorySafeCommandsForKey> completeCommands = new HashMap<>();
+ Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey = new HashMap<>();
Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey = new HashMap<>();
context.forEachId(txnId -> commands.put(txnId, lazyReference(txnId)));
@@ -510,22 +391,14 @@
{
case Key:
RoutableKey key = (RoutableKey) seekable;
- switch (context.keyHistory())
- {
- case ALL:
- // loading recovery commands implicitly loads deps
- completeCommands.put(key, allCommandsForKey((Key) key).createSafeReference());
- case DEPS:
- minimalCommands.put(key, depsCommandsForKey((Key) key).createSafeReference());
- break;
- }
+ commandsForKey.put(key, commandsForKey((Key) key).createSafeReference());
timestampsForKey.put(key, timestampsForKey((Key) key).createSafeReference());
break;
case Range:
// load range cfks here
}
}
- return createSafeStore(context, ranges, commands, timestampsForKey, minimalCommands, completeCommands);
+ return createSafeStore(context, ranges, commands, timestampsForKey, commandsForKey);
}
public SafeCommandStore beginOperation(PreLoadContext context)
@@ -593,6 +466,22 @@
}
}
+ private static Timestamp maxApplied(SafeCommandStore safeStore, Seekables<?, ?> keysOrRanges, Ranges slice)
+ {
+ Timestamp max = ((InMemoryCommandStore)safeStore.commandStore()).maxRedundant;
+ for (GlobalCommand command : ((InMemoryCommandStore) safeStore.commandStore()).commands.values())
+ {
+ if (command.value().hasBeen(Applied))
+ max = Timestamp.max(command.value().executeAt(), max);
+ }
+ return max;
+ }
+
+ public AsyncChain<Timestamp> maxAppliedFor(Seekables<?, ?> keysOrRanges, Ranges slice)
+ {
+ return submit(PreLoadContext.contextFor(keysOrRanges), safeStore -> maxApplied(safeStore, keysOrRanges, slice));
+ }
+
@Override
public String toString()
{
@@ -618,66 +507,6 @@
}
}
- static class CFKEntry extends TxnId
- {
- final boolean uninitialised;
- public CFKEntry(TxnId copy, boolean uninitialised)
- {
- super(copy);
- this.uninitialised = uninitialised;
- }
- }
-
- class CFKLoader implements CommandLoader<CFKEntry>
- {
- final RoutableKey key;
- CFKLoader(RoutableKey key)
- {
- this.key = key;
- }
-
- private Command loadForCFK(CFKEntry entry)
- {
- GlobalCommand globalCommand = commandIfPresent(entry);
- if (globalCommand != null && globalCommand.value() != null)
- return globalCommand.value();
- if (entry.uninitialised)
- return uninitialised(entry);
- throw illegalState("Could not find command for CFK for " + entry);
- }
-
- @Override
- public TxnId txnId(CFKEntry txnId)
- {
- return loadForCFK(txnId).txnId();
- }
-
- @Override
- public Timestamp executeAt(CFKEntry txnId)
- {
- return loadForCFK(txnId).executeAt();
- }
-
- @Override
- public SaveStatus saveStatus(CFKEntry txnId)
- {
- return loadForCFK(txnId).saveStatus();
- }
-
- @Override
- public List<TxnId> depsIds(CFKEntry data)
- {
- PartialDeps deps = loadForCFK(data).partialDeps();
- return deps != null ? deps.txnIds() : Collections.emptyList();
- }
-
- @Override
- public CFKEntry saveForCFK(Command command)
- {
- return new CFKEntry(command.txnId(), command.saveStatus().isUninitialised());
- }
- }
-
public static abstract class GlobalState<V>
{
private V value;
@@ -790,14 +619,12 @@
}
}
- public static class InMemorySafeStore extends AbstractSafeCommandStore<InMemorySafeCommand, InMemorySafeTimestampsForKey, InMemorySafeCommandsForKey, InMemorySafeCommandsForKeyUpdate>
+ public static class InMemorySafeStore extends AbstractSafeCommandStore<InMemorySafeCommand, InMemorySafeTimestampsForKey, InMemorySafeCommandsForKey>
{
private final InMemoryCommandStore commandStore;
private final Map<TxnId, InMemorySafeCommand> commands;
private final Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey;
- private final Map<RoutableKey, InMemorySafeCommandsForKey> depsCommandsForKey;
- private final Map<RoutableKey, InMemorySafeCommandsForKey> allCommandsForKey;
- private final Map<RoutableKey, InMemorySafeCommandsForKeyUpdate> updatesForKey = new HashMap<>();
+ private final Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey;
private final RangesForEpoch ranges;
public InMemorySafeStore(InMemoryCommandStore commandStore,
@@ -805,14 +632,12 @@
PreLoadContext context,
Map<TxnId, InMemorySafeCommand> commands,
Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey,
- Map<RoutableKey, InMemorySafeCommandsForKey> depsCommandsForKey,
- Map<RoutableKey, InMemorySafeCommandsForKey> allCommandsForKey)
+ Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey)
{
super(context);
this.commandStore = commandStore;
this.commands = commands;
- this.depsCommandsForKey = depsCommandsForKey;
- this.allCommandsForKey = allCommandsForKey;
+ this.commandsForKey = commandsForKey;
this.timestampsForKey = timestampsForKey;
this.ranges = Invariants.nonNull(ranges);
}
@@ -860,107 +685,46 @@
}
@Override
- protected InMemorySafeCommandsForKey getDepsCommandsForKeyInternal(RoutableKey key)
+ protected InMemorySafeCommandsForKey getCommandsForKeyInternal(RoutableKey key)
{
- return depsCommandsForKey.get(key);
+ return commandsForKey.get(key);
}
@Override
- protected void addDepsCommandsForKeyInternal(InMemorySafeCommandsForKey cfk)
+ protected void addCommandsForKeyInternal(InMemorySafeCommandsForKey cfk)
{
- depsCommandsForKey.put(cfk.key(), cfk);
+ commandsForKey.put(cfk.key(), cfk);
}
@Override
- protected InMemorySafeCommandsForKey getDepsCommandsForKeyIfLoaded(RoutableKey key)
+ protected InMemorySafeCommandsForKey getCommandsForKeyIfLoaded(RoutableKey key)
{
if (!commandStore.canExposeUnloaded())
return null;
- GlobalCommandsForKey global = commandStore.depsCommandsForKeyIfPresent((Key) key);
+ GlobalCommandsForKey global = commandStore.commandsForKeyIfPresent((Key) key);
return global != null ? global.createSafeReference() : null;
}
@Override
- protected InMemorySafeCommandsForKey getAllCommandsForKeyInternal(RoutableKey key)
+ protected void update(Command prev, Command updated, @Nullable Seekables<?, ?> keysOrRanges)
{
- return allCommandsForKey.get(key);
- }
+ super.update(prev, updated, keysOrRanges);
- @Override
- protected void addAllCommandsForKeyInternal(InMemorySafeCommandsForKey cfk)
- {
- allCommandsForKey.put(cfk.key(), cfk);
- }
+ TxnId txnId = updated.txnId();
+ if (txnId.domain() != Domain.Range)
+ return;
+ if (keysOrRanges == null && !updated.known().isDefinitionKnown())
+ return;
+ if (prev != null && prev.known().isDefinitionKnown())
+ return;
- @Override
- protected InMemorySafeCommandsForKey getAllCommandsForKeyIfLoaded(RoutableKey key)
- {
- if (!commandStore.canExposeUnloaded())
- return null;
- GlobalCommandsForKey global = commandStore.allCommandsForKeyIfPresent((Key) key);
- return global != null ? global.createSafeReference() : null;
- }
+ if (keysOrRanges == null) keysOrRanges = updated.partialTxn().keys();
- @Override
- protected InMemorySafeCommandsForKeyUpdate getCommandsForKeyUpdateInternal(RoutableKey key)
- {
- return updatesForKey.get(key);
- }
+ Ranges slice = ranges().allBetween(txnId, updated.executeAtOrTxnId());
+ slice = commandStore.redundantBefore().removeShardRedundant(txnId, updated.executeAt(), slice);
+ commandStore.rangeCommands.computeIfAbsent(txnId, ignore -> new RangeCommand(commandStore.commands.get(txnId)))
+ .update(((Ranges)keysOrRanges).slice(slice, Minimal));
- @Override
- protected InMemorySafeCommandsForKeyUpdate createCommandsForKeyUpdateInternal(RoutableKey key)
- {
- return new InMemorySafeCommandsForKeyUpdate((Key) key, (CommandLoader<CFKEntry>) cfkLoader(key));
- }
-
- @Override
- protected void addCommandsForKeyUpdateInternal(InMemorySafeCommandsForKeyUpdate update)
- {
- updatesForKey.put(update.key(), update);
- }
-
- private CommandsForKey applyUpdate(CommandsForKey cfk, InMemorySafeCommandsForKeyUpdate update, KeyHistory keyHistory)
- {
- switch (keyHistory)
- {
- case DEPS:
- return update.applyToDeps(cfk);
- case ALL:
- return update.applyToAll(cfk);
- default:
- throw new IllegalArgumentException("Unhandled KeyHistory " + keyHistory);
- }
- }
-
- private void applyCommandForKeyUpdate(InMemorySafeCommandsForKeyUpdate update, KeyHistory kind, Map<RoutableKey, InMemorySafeCommandsForKey> commandsMap, Function<Key, GlobalCommandsForKey> getter)
- {
- Key key = update.key();
- InMemorySafeCommandsForKey safeCFK = commandsMap.get(key);
- if (safeCFK != null)
- {
- if (safeCFK.isEmpty()) safeCFK.initialize(cfkLoader(key));
- safeCFK.update(applyUpdate(safeCFK.current(), update, kind));
- }
- else
- {
- GlobalCommandsForKey globalCFK = getter.apply(key);
- CommandsForKey cfk = globalCFK.value();
- if (cfk == null)
- cfk = new CommandsForKey(key, cfkLoader(key));
- globalCFK.value(applyUpdate(cfk, update, kind));
- }
- }
-
- private void applyCommandForKeyUpdate(InMemorySafeCommandsForKeyUpdate update)
- {
- applyCommandForKeyUpdate(update, KeyHistory.DEPS, depsCommandsForKey, commandStore::depsCommandsForKey);
- applyCommandForKeyUpdate(update, KeyHistory.ALL, allCommandsForKey, commandStore::allCommandsForKey);
- }
-
- @Override
- protected void applyCommandForKeyUpdates()
- {
- updatesForKey.values().forEach(this::applyCommandForKeyUpdate);
}
@Override
@@ -993,26 +757,6 @@
return ranges;
}
- @Override
- public Timestamp maxConflict(Seekables<?, ?> keysOrRanges, Ranges slice)
- {
- Timestamp timestamp = commandStore.mapReduceForKey(this, keysOrRanges, slice, KeyHistory.NONE, (timestamps, commands, prev) -> Timestamp.nonNullOrMax(timestamps.max(), prev), Timestamp.NONE, Objects::isNull);
- Seekables<?, ?> sliced = keysOrRanges.slice(slice, Minimal);
- for (RangeCommand command : commandStore.rangeCommands.values())
- {
- if (command.ranges.intersects(sliced))
- timestamp = Timestamp.nonNullOrMax(timestamp, command.command.value().executeAt());
- }
- // TODO (required): check we have this on C*-side; accept TxnId instead of Ranges when updating API, to avoid slicing
- return Timestamp.nonNullOrMax(timestamp, commandStore.redundantBefore().minConflict(keysOrRanges.slice(slice)));
- }
-
- @Override
- public void erase(SafeCommand command)
- {
- commands.remove(command.txnId());
- }
-
// TODO (preferable): this can have protected visibility if under CommandStore, and this is perhaps a better place to put it also
@Override
public void registerHistoricalTransactions(Deps deps)
@@ -1080,41 +824,30 @@
this.deps = deps != null ? deps.txnIds() : Collections.emptyList();
}
}
+
+ @Override
+ public <P1, T> T mapReduceActive(Seekables<?, ?> keysOrRanges, Ranges slice, Timestamp startedBefore, Kinds testKind, CommandFunction<P1, T, T> map, P1 p1, T accumulate)
+ {
+ accumulate = commandStore.mapReduceForKey(this, keysOrRanges, slice, (commands, prev) -> {
+ return commands.mapReduceActive(startedBefore, testKind, map, p1, prev);
+ }, accumulate);
+
+ return mapReduceRangesInternal(keysOrRanges, slice, startedBefore, null, testKind, STARTED_BEFORE, ANY_DEPS, ANY_STATUS, map, p1, accumulate);
+ }
+
// TODO (expected): instead of accepting a slice, accept the min/max epoch and let implementation handle it
@Override
- public <P1, T> T mapReduce(Seekables<?, ?> keysOrRanges, Ranges slice, KeyHistory keyHistory, Kinds testKind, TestTimestamp testTimestamp, Timestamp timestamp, TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus, CommandFunction<P1, T, T> map, P1 p1, T accumulate, Predicate<? super T> terminate)
+ public <P1, T> T mapReduceFull(Seekables<?, ?> keysOrRanges, Ranges slice, TxnId testTxnId, Kinds testKind, TestStartedAt testStartedAt, TestDep testDep, TestStatus testStatus, CommandFunction<P1, T, T> map, P1 p1, T accumulate)
{
- accumulate = commandStore.mapReduceForKey(this, keysOrRanges, slice, keyHistory, (timestamps, commands, prev) -> {
- CommandTimeseries.TimestampType timestampType;
- switch (testTimestamp)
- {
- default: throw new AssertionError();
- case STARTED_AFTER:
- case STARTED_BEFORE:
- timestampType = CommandTimeseries.TimestampType.TXN_ID;
- break;
- case EXECUTES_AFTER:
- case MAY_EXECUTE_BEFORE:
- timestampType = CommandTimeseries.TimestampType.EXECUTE_AT;
- }
- CommandTimeseries.TestTimestamp remapTestTimestamp;
- switch (testTimestamp)
- {
- default: throw new AssertionError();
- case STARTED_AFTER:
- case EXECUTES_AFTER:
- remapTestTimestamp = CommandTimeseries.TestTimestamp.AFTER;
- break;
- case STARTED_BEFORE:
- case MAY_EXECUTE_BEFORE:
- remapTestTimestamp = CommandTimeseries.TestTimestamp.BEFORE;
- }
- return commands.commands().mapReduce(testKind, timestampType, remapTestTimestamp, timestamp, testDep, depId, minStatus, maxStatus, map, p1, prev, terminate);
- }, accumulate, terminate);
+ accumulate = commandStore.mapReduceForKey(this, keysOrRanges, slice, (commands, prev) -> {
+ return commands.mapReduceFull(testTxnId, testKind, testStartedAt, testDep, testStatus, map, p1, prev);
+ }, accumulate);
- if (terminate.test(accumulate))
- return accumulate;
+ return mapReduceRangesInternal(keysOrRanges, slice, testTxnId, testTxnId, testKind, testStartedAt, testDep, testStatus, map, p1, accumulate);
+ }
+ private <P1, T> T mapReduceRangesInternal(Seekables<?, ?> keysOrRanges, Ranges slice, @Nonnull Timestamp testTimestamp, @Nullable TxnId testTxnId, Kinds testKind, TestStartedAt testStartedAt, TestDep testDep, TestStatus testStatus, CommandFunction<P1, T, T> map, P1 p1, T accumulate)
+ {
// TODO (find lib, efficiency): this is super inefficient, need to store Command in something queryable
Seekables<?, ?> sliced = keysOrRanges.slice(slice, Minimal);
Map<Range, List<TxnInfo>> collect = new TreeMap<>(Range::compare);
@@ -1125,29 +858,37 @@
return;
Invariants.nonNull(command);
- switch (testTimestamp)
+ switch (testStartedAt)
{
default: throw new AssertionError();
case STARTED_AFTER:
- if (command.txnId().compareTo(timestamp) < 0) return;
+ if (command.txnId().compareTo(testTimestamp) <= 0) return;
else break;
case STARTED_BEFORE:
- if (command.txnId().compareTo(timestamp) > 0) return;
- else break;
- case EXECUTES_AFTER:
- if (command.executeAt().compareTo(timestamp) < 0) return;
- else break;
- case MAY_EXECUTE_BEFORE:
- Timestamp compareTo = command.executeAtIfKnownElseTxnId();
- if (compareTo.compareTo(timestamp) > 0) return;
- else break;
+ if (command.txnId().compareTo(testTimestamp) >= 0) return;
+ case ANY:
+ if (testDep != ANY_DEPS && command.executeAtOrTxnId().compareTo(testTxnId) < 0)
+ return;
}
- if (minStatus != null && command.status().compareTo(minStatus) < 0)
- return;
-
- if (maxStatus != null && command.status().compareTo(maxStatus) > 0)
- return;
+ switch (testStatus)
+ {
+ default: throw new AssertionError("Unhandled TestStatus: " + testStatus);
+ case ANY_STATUS:
+ break;
+ case IS_PROPOSED:
+ switch (command.status())
+ {
+ default: return;
+ case PreCommitted:
+ case Committed:
+ case Accepted:
+ }
+ break;
+ case IS_STABLE:
+ if (command.status().compareTo(Stable) < 0 || command.status().compareTo(Truncated) >= 0)
+ return;
+ }
if (!testKind.test(command.txnId().kind()))
return;
@@ -1168,7 +909,7 @@
// and so it is safe to execute, when in fact it is only a dependency on a different shard
// (and that other shard, perhaps, does not know that it is a dependency - and so it is not durably known)
// TODO (required): consider this some more
- if ((testDep == WITH) == !command.partialDeps().intersects(depId, rangeCommand.ranges))
+ if ((testDep == WITH) == !command.partialDeps().intersects(testTxnId, rangeCommand.ranges))
return;
}
@@ -1184,20 +925,19 @@
}, collect);
}));
- if (minStatus == null && testDep == ANY_DEPS)
+ if (testStatus == ANY_STATUS && testDep == ANY_DEPS)
{
commandStore.historicalRangeCommands.forEach(((txnId, ranges) -> {
- switch (testTimestamp)
+ switch (testStartedAt)
{
default: throw new AssertionError();
case STARTED_AFTER:
- case EXECUTES_AFTER:
- if (txnId.compareTo(timestamp) < 0) return;
+ if (txnId.compareTo(testTimestamp) <= 0) return;
else break;
case STARTED_BEFORE:
- case MAY_EXECUTE_BEFORE:
- if (txnId.compareTo(timestamp) > 0) return;
+ if (txnId.compareTo(testTimestamp) >= 0) return;
else break;
+ case ANY:
}
if (!testKind.test(txnId.kind()))
@@ -1234,9 +974,7 @@
for (TxnInfo command : e.getValue())
{
T initial = accumulate;
- accumulate = map.apply(p1, e.getKey(), command.txnId, command.executeAt, command.status, () -> command.deps, initial);
- if (terminate.test(accumulate))
- return accumulate;
+ accumulate = map.apply(p1, e.getKey(), command.txnId, command.executeAt, initial);
}
}
@@ -1244,24 +982,6 @@
}
@Override
- public CommonAttributes completeRegistration(Seekables<?, ?> keysOrRanges, Ranges slice, InMemorySafeCommand command, CommonAttributes attrs)
- {
- return commandStore.register(this, keysOrRanges, slice, command, attrs);
- }
-
- @Override
- public CommonAttributes completeRegistration(Seekable keyOrRange, Ranges slice, InMemorySafeCommand command, CommonAttributes attrs)
- {
- return commandStore.register(this, keyOrRange, slice, command, attrs);
- }
-
- @Override
- public CommandLoader<?> cfkLoader(RoutableKey key)
- {
- return commandStore.new CFKLoader(key);
- }
-
- @Override
protected void invalidateSafeState()
{
commands.values().forEach(c -> {
@@ -1277,16 +997,7 @@
});
commands.values().forEach(SafeState::invalidate);
timestampsForKey.values().forEach(SafeState::invalidate);
- depsCommandsForKey.values().forEach(SafeState::invalidate);
- allCommandsForKey.values().forEach(SafeState::invalidate);
- updatesForKey.values().forEach(SafeState::invalidate);
- }
-
- @Override
- public void complete()
- {
- postExecute();
- super.complete();
+ commandsForKey.values().forEach(SafeState::invalidate);
}
}
@@ -1451,10 +1162,9 @@
PreLoadContext context,
Map<TxnId, InMemorySafeCommand> commands,
Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKey,
- Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommandsForKey,
- Map<RoutableKey, InMemorySafeCommandsForKey> completeCommandsForKey)
+ Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKey)
{
- super(commandStore, ranges, context, commands, timestampsForKey, minimalCommandsForKey, completeCommandsForKey);
+ super(commandStore, ranges, context, commands, timestampsForKey, commandsForKey);
}
@Override
@@ -1470,20 +1180,6 @@
assertThread();
return super.getInternal(txnId);
}
-
- @Override
- public void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command)
- {
- assertThread();
- super.register(keysOrRanges, slice, command);
- }
-
- @Override
- public void register(Seekable keyOrRange, Ranges slice, Command command)
- {
- assertThread();
- super.register(keyOrRange, slice, command);
- }
}
public Debug(int id, NodeTimeService time, Agent agent, DataStore store, ProgressLog.Factory progressLogFactory, EpochUpdateHolder epochUpdateHolder)
@@ -1492,9 +1188,9 @@
}
@Override
- protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges, Map<TxnId, InMemorySafeCommand> commands, Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKeyMap, Map<RoutableKey, InMemorySafeCommandsForKey> minimalCommandsForKeys, Map<RoutableKey, InMemorySafeCommandsForKey> completeCommandsForKeys)
+ protected InMemorySafeStore createSafeStore(PreLoadContext context, RangesForEpoch ranges, Map<TxnId, InMemorySafeCommand> commands, Map<RoutableKey, InMemorySafeTimestampsForKey> timestampsForKeyMap, Map<RoutableKey, InMemorySafeCommandsForKey> commandsForKeys)
{
- return new DebugSafeStore(this, ranges, context, commands, timestampsForKeyMap, minimalCommandsForKeys, completeCommandsForKeys);
+ return new DebugSafeStore(this, ranges, context, commands, timestampsForKeyMap, commandsForKeys);
}
}
diff --git a/accord-core/src/main/java/accord/impl/InMemorySafeCommandsForKeyUpdate.java b/accord-core/src/main/java/accord/impl/InMemorySafeCommandsForKeyUpdate.java
deleted file mode 100644
index 2c7f955..0000000
--- a/accord-core/src/main/java/accord/impl/InMemorySafeCommandsForKeyUpdate.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.impl;
-
-import accord.api.Key;
-import accord.impl.CommandTimeseries.CommandLoader;
-import accord.impl.InMemoryCommandStore.CFKEntry;
-import accord.utils.Invariants;
-
-public class InMemorySafeCommandsForKeyUpdate extends SafeCommandsForKey.Update<CommandsForKey.Update, CFKEntry>
-{
- private static final CommandsForKey.Update VALUE = new CommandsForKey.Update() {};
-
- private boolean invalidated = false;
-
- public InMemorySafeCommandsForKeyUpdate(Key key, CommandLoader<CFKEntry> loader)
- {
- super(key, loader);
- }
-
- @Override
- public void initialize()
- {
- Invariants.checkState(isEmpty());
- }
-
- @Override
- public CommandsForKey.Update current()
- {
- return VALUE;
- }
-
- @Override
- public void invalidate()
- {
- invalidated = true;
- }
-
- @Override
- public boolean invalidated()
- {
- return invalidated;
- }
-}
diff --git a/accord-core/src/main/java/accord/impl/SafeCommandsForKey.java b/accord-core/src/main/java/accord/impl/SafeCommandsForKey.java
index df1a749..aea0dbc 100644
--- a/accord-core/src/main/java/accord/impl/SafeCommandsForKey.java
+++ b/accord-core/src/main/java/accord/impl/SafeCommandsForKey.java
@@ -19,7 +19,8 @@
package accord.impl;
import accord.api.Key;
-import accord.impl.CommandTimeseries.CommandLoader;
+import accord.local.Command;
+import accord.primitives.TxnId;
public abstract class SafeCommandsForKey implements SafeState<CommandsForKey>
{
@@ -43,25 +44,35 @@
return update;
}
- public CommandsForKey initialize(CommandLoader<?> loader)
+ CommandsForKey update(Command prev, Command update)
{
- return update(new CommandsForKey(key, loader));
+ CommandsForKey current = current();
+ CommandsForKey next = current.update(prev, update);
+ if (next != current)
+ set(next);
+ return next;
}
- public static abstract class Update<U extends CommandsForKey.Update, D> extends CommandsForKeyGroupUpdater.Mutable<D> implements SafeState<U>
+ CommandsForKey registerHistorical(TxnId txnId)
{
- private final Key key;
+ CommandsForKey current = current();
+ CommandsForKey next = current.registerHistorical(txnId);
+ if (next != current)
+ set(next);
+ return next;
+ }
- public Update(Key key, CommandLoader<D> loader)
- {
- super(loader);
- this.key = key;
- }
+ CommandsForKey updateRedundantBefore(TxnId redundantBefore)
+ {
+ CommandsForKey current = current();
+ CommandsForKey next = current.withoutRedundant(redundantBefore);
+ if (next != current)
+ set(next);
+ return next;
+ }
- public Key key()
- {
- return key;
- }
- public abstract void initialize();
+ public void initialize()
+ {
+ set(new CommandsForKey(key));
}
}
diff --git a/accord-core/src/main/java/accord/impl/SafeTimestampsForKey.java b/accord-core/src/main/java/accord/impl/SafeTimestampsForKey.java
index ee5d58c..42eb3c5 100644
--- a/accord-core/src/main/java/accord/impl/SafeTimestampsForKey.java
+++ b/accord-core/src/main/java/accord/impl/SafeTimestampsForKey.java
@@ -19,10 +19,7 @@
package accord.impl;
import accord.api.Key;
-import accord.api.VisibleForImplementation;
import accord.primitives.Timestamp;
-import accord.utils.Invariants;
-import com.google.common.annotations.VisibleForTesting;
public abstract class SafeTimestampsForKey implements SafeState<TimestampsForKey>
{
@@ -51,35 +48,10 @@
return update(new TimestampsForKey(key));
}
- @VisibleForTesting
- @VisibleForImplementation
- public static Timestamp updateMax(TimestampsForKey tfk, Timestamp timestamp)
- {
- Invariants.checkArgument(tfk != null || timestamp != null);
- if (tfk == null)
- return timestamp;
- if (timestamp == null)
- return tfk.max();
- return Timestamp.max(tfk.max(), timestamp);
- }
-
- @VisibleForTesting
- @VisibleForImplementation
- public <D> TimestampsForKey updateMax(Timestamp timestamp)
+ TimestampsForKey updateLastExecutionTimestamps(Timestamp lastExecutedTimestamp, long lastExecutedHlc, Timestamp lastWriteTimestamp)
{
TimestampsForKey current = current();
return update(new TimestampsForKey(current.key(),
- updateMax(current, timestamp),
- current.lastExecutedTimestamp(),
- current.rawLastExecutedHlc(),
- current.lastWriteTimestamp()));
- }
-
- <D> TimestampsForKey updateLastExecutionTimestamps(Timestamp lastExecutedTimestamp, long lastExecutedHlc, Timestamp lastWriteTimestamp)
- {
- TimestampsForKey current = current();
- return update(new TimestampsForKey(current.key(),
- current.max(),
lastExecutedTimestamp,
lastExecutedHlc,
lastWriteTimestamp));
diff --git a/accord-core/src/main/java/accord/impl/TimestampsForKey.java b/accord-core/src/main/java/accord/impl/TimestampsForKey.java
index 65fc094..32494b6 100644
--- a/accord-core/src/main/java/accord/impl/TimestampsForKey.java
+++ b/accord-core/src/main/java/accord/impl/TimestampsForKey.java
@@ -24,33 +24,30 @@
import accord.primitives.Timestamp;
import accord.primitives.TxnId;
-public class TimestampsForKey implements DomainTimestamps
+public class TimestampsForKey
{
public static final long NO_LAST_EXECUTED_HLC = Long.MIN_VALUE;
public static class SerializerSupport
{
- public static <D> TimestampsForKey create(Key key,
- Timestamp max,
+ public static TimestampsForKey create(Key key,
Timestamp lastExecutedTimestamp,
long lastExecutedHlc,
Timestamp lastWriteTimestamp)
{
- return new TimestampsForKey(key, max, lastExecutedTimestamp, lastExecutedHlc, lastWriteTimestamp);
+ return new TimestampsForKey(key, lastExecutedTimestamp, lastExecutedHlc, lastWriteTimestamp);
}
}
private final Key key;
- private final Timestamp max;
private final Timestamp lastExecutedTimestamp;
// TODO (desired): we have leaked C* implementation details here
private final long rawLastExecutedHlc;
private final Timestamp lastWriteTimestamp;
- public TimestampsForKey(Key key, Timestamp max, Timestamp lastExecutedTimestamp, long rawLastExecutedHlc, Timestamp lastWriteTimestamp)
+ public TimestampsForKey(Key key, Timestamp lastExecutedTimestamp, long rawLastExecutedHlc, Timestamp lastWriteTimestamp)
{
this.key = key;
- this.max = max;
this.lastExecutedTimestamp = lastExecutedTimestamp;
this.rawLastExecutedHlc = rawLastExecutedHlc;
this.lastWriteTimestamp = lastWriteTimestamp;
@@ -59,7 +56,6 @@
public TimestampsForKey(Key key)
{
this.key = key;
- this.max = Timestamp.NONE;
this.lastExecutedTimestamp = Timestamp.NONE;
this.rawLastExecutedHlc = 0;
this.lastWriteTimestamp = Timestamp.NONE;
@@ -70,7 +66,7 @@
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
TimestampsForKey that = (TimestampsForKey) o;
- return rawLastExecutedHlc == that.rawLastExecutedHlc && Objects.equals(key, that.key) && Objects.equals(max, that.max) && Objects.equals(lastExecutedTimestamp, that.lastExecutedTimestamp) && Objects.equals(lastWriteTimestamp, that.lastWriteTimestamp);
+ return rawLastExecutedHlc == that.rawLastExecutedHlc && Objects.equals(key, that.key) && Objects.equals(lastExecutedTimestamp, that.lastExecutedTimestamp) && Objects.equals(lastWriteTimestamp, that.lastWriteTimestamp);
}
public int hashCode()
@@ -83,11 +79,6 @@
return key;
}
- public Timestamp max()
- {
- return max;
- }
-
public Timestamp lastExecutedTimestamp()
{
return lastExecutedTimestamp;
@@ -110,7 +101,7 @@
public TimestampsForKey withoutRedundant(TxnId redundantBefore)
{
- return new TimestampsForKey(key, max.compareTo(redundantBefore) < 0 ? Timestamp.NONE : max,
+ return new TimestampsForKey(key,
lastExecutedTimestamp.compareTo(redundantBefore) < 0 ? Timestamp.NONE : lastExecutedTimestamp,
rawLastExecutedHlc < redundantBefore.hlc() ? NO_LAST_EXECUTED_HLC : rawLastExecutedHlc,
lastWriteTimestamp.compareTo(redundantBefore) < 0 ? Timestamp.NONE : lastWriteTimestamp);
@@ -141,7 +132,6 @@
{
return "TimestampsForKey@" + System.identityHashCode(this) + '{' +
"key=" + key +
- ", max=" + max +
", lastExecutedTimestamp=" + lastExecutedTimestamp +
", rawLastExecutedHlc=" + rawLastExecutedHlc +
", lastWriteTimestamp=" + lastWriteTimestamp +
diff --git a/accord-core/src/main/java/accord/local/Bootstrap.java b/accord-core/src/main/java/accord/local/Bootstrap.java
index 3d685a0..d268812 100644
--- a/accord-core/src/main/java/accord/local/Bootstrap.java
+++ b/accord-core/src/main/java/accord/local/Bootstrap.java
@@ -28,7 +28,7 @@
import accord.api.DataStore.FetchRanges;
import accord.api.DataStore.FetchResult;
import accord.api.DataStore.StartingRangeFetch;
-import accord.coordinate.CoordinateNoOp;
+import accord.coordinate.FetchMaxConflict;
import accord.coordinate.CoordinateSyncPoint;
import accord.primitives.Ranges;
import accord.primitives.Routable;
@@ -45,7 +45,6 @@
import static accord.primitives.Routables.Slice.Minimal;
import static accord.primitives.Txn.Kind.ExclusiveSyncPoint;
import static accord.primitives.Txn.Kind.LocalOnly;
-import static accord.primitives.Txn.Kind.NoOp;
import static accord.utils.Invariants.illegalState;
/**
@@ -137,7 +136,7 @@
// TODO (required, correcness) : PreLoadContext only works with Seekables, which doesn't allow mixing Keys and Ranges... But Deps has both Keys AND Ranges!
// TODO (required): is localSyncId even being used anymore
// ATM all known implementations store ranges in-memory, but this will not be true soon, so this will need to be addressed
- .flatMap(syncPoint -> node.withEpoch(epoch, () -> store.submit(contextFor(localSyncId, syncPoint.waitFor.keyDeps.keys(), KeyHistory.DEPS), safeStore1 -> {
+ .flatMap(syncPoint -> node.withEpoch(epoch, () -> store.submit(contextFor(localSyncId, syncPoint.waitFor.keyDeps.keys(), KeyHistory.COMMANDS), safeStore1 -> {
if (valid.isEmpty()) // we've lost ownership of the range
return AsyncResults.success(Ranges.EMPTY);
@@ -226,18 +225,14 @@
{
if (maxApplied == null)
{
- TxnId txnId;
synchronized (this)
{
if (state.startedAt == Integer.MAX_VALUE)
state.startedAt = logicalClock++;
- txnId = node.nextTxnId(NoOp, Routable.Domain.Range);
}
// TODO (expected): associate callbacks with this CommandStore, to remove synchronization
- node.withEpoch(txnId.epoch(), () -> {
- CoordinateNoOp.coordinate(node, txnId, state.ranges)
- .begin((executeAt, failure) -> safeToReadCallback(state, executeAt, failure));
- });
+ FetchMaxConflict.fetchMaxConflict(node, state.ranges)
+ .begin((executeAt, failure) -> safeToReadCallback(state, executeAt, failure));
}
else
{
diff --git a/accord-core/src/main/java/accord/local/Cleanup.java b/accord-core/src/main/java/accord/local/Cleanup.java
index 97191dc..8c2dd18 100644
--- a/accord-core/src/main/java/accord/local/Cleanup.java
+++ b/accord-core/src/main/java/accord/local/Cleanup.java
@@ -36,6 +36,7 @@
import static accord.local.Status.Durability.Universal;
import static accord.local.Status.Durability.UniversalOrInvalidated;
import static accord.local.Status.PreCommitted;
+import static accord.primitives.Txn.Kind.EphemeralRead;
import static accord.utils.Invariants.illegalState;
/**
@@ -100,6 +101,9 @@
public static Cleanup shouldCleanup(TxnId txnId, Status status, Durability durability, EpochSupplier toEpoch, Route<?> route, RedundantBefore redundantBefore, DurableBefore durableBefore, boolean enforceInvariants)
{
+ if (txnId.kind() == EphemeralRead)
+ return Cleanup.NO; // TODO (required): clean-up based on timeout
+
if (durableBefore.min(txnId) == Universal)
{
if (status.hasBeen(PreCommitted) && !status.hasBeen(Applied)) // TODO (expected): may be stale
diff --git a/accord-core/src/main/java/accord/local/Command.java b/accord-core/src/main/java/accord/local/Command.java
index fd18291..edb3310 100644
--- a/accord-core/src/main/java/accord/local/Command.java
+++ b/accord-core/src/main/java/accord/local/Command.java
@@ -128,12 +128,6 @@
}
}
- static PreLoadContext contextForCommand(Command command)
- {
- Invariants.checkState(command.hasBeen(Status.PreAccepted) && command.partialTxn() != null);
- return command instanceof PreLoadContext ? (PreLoadContext) command : PreLoadContext.contextFor(command.txnId(), command.partialTxn().keys());
- }
-
private static Durability durability(Durability durability, SaveStatus status)
{
if (status.compareTo(SaveStatus.PreApplied) >= 0 && !status.hasBeen(Invalidated) && durability == NotDurable)
@@ -174,10 +168,15 @@
return Truncated.invalidated(txnId, durableListeners);
}
- public static Truncated truncatedApply(CommonAttributes common,SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result)
+ public static Truncated truncatedApply(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result)
{
return Truncated.truncatedApply(common, saveStatus, executeAt, writes, result);
}
+
+ public static Truncated truncatedApply(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result, Timestamp executesAtLeast)
+ {
+ return Truncated.truncatedApply(common, saveStatus, executeAt, writes, result, executesAtLeast);
+ }
}
private static SaveStatus validateCommandClass(SaveStatus status, Class<?> expected, Class<?> actual)
@@ -190,27 +189,40 @@
return status;
}
- private static SaveStatus validateCommandClass(SaveStatus status, Class<?> klass)
+ private static SaveStatus validateCommandClass(TxnId txnId, SaveStatus status, Class<?> klass)
{
- switch (status.status)
+ switch (status)
{
+ case Uninitialised:
case NotDefined:
return validateCommandClass(status, NotDefined.class, klass);
case PreAccepted:
return validateCommandClass(status, PreAccepted.class, klass);
- case AcceptedInvalidate:
case Accepted:
+ case AcceptedWithDefinition:
+ case AcceptedInvalidate:
+ case AcceptedInvalidateWithDefinition:
case PreCommitted:
+ case PreCommittedWithAcceptedDeps:
+ case PreCommittedWithDefinition:
+ case PreCommittedWithDefinitionAndAcceptedDeps:
return validateCommandClass(status, Accepted.class, klass);
case Committed:
case ReadyToExecute:
case Stable:
return validateCommandClass(status, Committed.class, klass);
case PreApplied:
+ case Applying:
case Applied:
return validateCommandClass(status, Executed.class, klass);
+ case TruncatedApply:
+ case TruncatedApplyWithDeps:
+ case TruncatedApplyWithOutcome:
+ if (txnId.kind().awaitsOnlyDeps())
+ return validateCommandClass(status, TruncatedAwaitsOnlyDeps.class, klass);
+ case Erased:
+ case ErasedOrInvalidated:
case Invalidated:
- case Truncated:
return validateCommandClass(status, Truncated.class, klass);
default:
throw illegalState("Unhandled status " + status);
@@ -228,15 +240,14 @@
private final TxnId txnId;
private final SaveStatus status;
private final Durability durability;
- @Nullable
- private final Route<?> route;
+ private final @Nullable Route<?> route;
private final Ballot promised;
private final Listeners.Immutable listeners;
private AbstractCommand(TxnId txnId, SaveStatus status, Durability durability, @Nullable Route<?> route, Ballot promised, Listeners.Immutable listeners)
{
this.txnId = txnId;
- this.status = validateCommandClass(status, getClass());
+ this.status = validateCommandClass(txnId, status, getClass());
this.durability = durability;
this.route = route;
this.promised = promised;
@@ -246,7 +257,7 @@
private AbstractCommand(CommonAttributes common, SaveStatus status, Ballot promised)
{
this.txnId = common.txnId();
- this.status = validateCommandClass(status, getClass());
+ this.status = validateCommandClass(txnId, status, getClass());
this.durability = common.durability();
this.route = common.route();
this.promised = promised;
@@ -352,8 +363,6 @@
switch (known.executeAt)
{
default: throw new AssertionError("Unhandled KnownExecuteAt: " + known.executeAt);
- case ExecuteAtNotWitnessed:
- Invariants.checkState(executeAt == null);
case ExecuteAtErased:
case ExecuteAtUnknown:
break;
@@ -410,7 +419,7 @@
case CleaningUp:
break;
case ReadyToExclude:
- Invariants.checkState(!validate.saveStatus().hasBeen(Status.Committed) || validate.asCommitted().waitingOn == null);
+ Invariants.checkState(validate.saveStatus() != SaveStatus.Committed || validate.asCommitted().waitingOn == null);
break;
case WaitingToExecute:
case ReadyToExecute:
@@ -475,6 +484,11 @@
public abstract Listeners.Immutable<DurableAndIdempotentListener> durableListeners();
public abstract SaveStatus saveStatus();
+ /**
+ * Only meaningful when txnId.kind().awaitsOnlyDeps()
+ */
+ public Timestamp executesAtLeast() { return executeAt(); }
+
static boolean isSameClass(Command command, Class<? extends Command> klass)
{
return command.getClass() == klass;
@@ -502,11 +516,6 @@
throw new IllegalArgumentException(errorMsg + format(" expected %s got %s", klass.getSimpleName(), command.getClass().getSimpleName()));
}
- public PreLoadContext contextForSelf()
- {
- return contextForCommand(this);
- }
-
public abstract Timestamp executeAt();
public abstract Ballot acceptedOrCommitted();
@@ -715,7 +724,7 @@
}
}
- public static final class Truncated extends AbstractCommand
+ public static class Truncated extends AbstractCommand
{
@Nullable final Timestamp executeAt;
@Nullable final Writes writes;
@@ -787,22 +796,41 @@
Invariants.checkArgument(command.known().executeAt.isDecidedAndKnownToExecute());
if (route == null) route = Route.castToNonNullFullRoute(command.route());
Durability durability = Durability.mergeAtLeast(command.durability(), ShardUniversal);
+ if (command.txnId().kind().awaitsOnlyDeps())
+ {
+ Timestamp executesAtLeast = command.hasBeen(Stable) ? command.executesAtLeast() : null;
+ return validate(new TruncatedAwaitsOnlyDeps(command.txnId(), SaveStatus.TruncatedApply, durability, route, command.executeAt(), EMPTY, null, null, executesAtLeast));
+ }
return validate(new Truncated(command.txnId(), SaveStatus.TruncatedApply, durability, route, command.executeAt(), EMPTY, null, null));
}
public static Truncated truncatedApplyWithOutcome(Executed command)
{
Durability durability = Durability.mergeAtLeast(command.durability(), ShardUniversal);
+ if (command.txnId().kind().awaitsOnlyDeps())
+ return validate(new TruncatedAwaitsOnlyDeps(command.txnId(), SaveStatus.TruncatedApplyWithOutcome, durability, command.route(), command.executeAt(), EMPTY, command.writes, command.result, command.executesAtLeast()));
return validate(new Truncated(command.txnId(), SaveStatus.TruncatedApplyWithOutcome, durability, command.route(), command.executeAt(), EMPTY, command.writes, command.result));
}
public static Truncated truncatedApply(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result)
{
+ Invariants.checkArgument(!common.txnId().kind().awaitsOnlyDeps());
+ Durability durability = checkTruncatedApplyInvariants(common, saveStatus, executeAt);
+ return validate(new Truncated(common.txnId(), saveStatus, durability, common.route(), executeAt, EMPTY, writes, result));
+ }
+
+ public static Truncated truncatedApply(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt, Writes writes, Result result, Timestamp dependencyExecutesAt)
+ {
+ Invariants.checkArgument(common.txnId().kind().awaitsOnlyDeps());
+ Durability durability = checkTruncatedApplyInvariants(common, saveStatus, executeAt);
+ return validate(new TruncatedAwaitsOnlyDeps(common.txnId(), saveStatus, durability, common.route(), executeAt, EMPTY, writes, result, dependencyExecutesAt));
+ }
+
+ private static Durability checkTruncatedApplyInvariants(CommonAttributes common, SaveStatus saveStatus, Timestamp executeAt)
+ {
Invariants.checkArgument(executeAt != null);
Invariants.checkArgument(saveStatus == SaveStatus.TruncatedApply || saveStatus == SaveStatus.TruncatedApplyWithDeps || saveStatus == SaveStatus.TruncatedApplyWithOutcome);
- // TODO review Is this correctly handling all three versions of truncatedApplyStatus? Should writes be null some of the time?
- Durability durability = Durability.mergeAtLeast(common.durability(), ShardUniversal);
- return validate(new Truncated(common.txnId(), saveStatus, durability, common.route(), executeAt, EMPTY, writes, result));
+ return Durability.mergeAtLeast(common.durability(), ShardUniversal);
}
public static Truncated invalidated(Command command)
@@ -863,6 +891,43 @@
}
}
+ public static class TruncatedAwaitsOnlyDeps extends Truncated
+ {
+ @Nullable final Timestamp executesAtLeast;
+
+ public TruncatedAwaitsOnlyDeps(CommonAttributes commonAttributes, SaveStatus saveStatus, @Nullable Timestamp executeAt, @Nullable Writes writes, @Nullable Result result, @Nullable Timestamp executesAtLeast)
+ {
+ super(commonAttributes, saveStatus, executeAt, writes, result);
+ this.executesAtLeast = executesAtLeast;
+ }
+
+ public TruncatedAwaitsOnlyDeps(TxnId txnId, SaveStatus saveStatus, Durability durability, @Nullable Route<?> route, @Nullable Timestamp executeAt, Listeners.Immutable listeners, @Nullable Writes writes, @Nullable Result result, @Nullable Timestamp executesAtLeast)
+ {
+ super(txnId, saveStatus, durability, route, executeAt, listeners, writes, result);
+ this.executesAtLeast = executesAtLeast;
+ }
+
+ public Timestamp executesAtLeast()
+ {
+ return executesAtLeast;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (!super.equals(o)) return false;
+ return Objects.equals(executesAtLeast, ((TruncatedAwaitsOnlyDeps)o).executesAtLeast);
+ }
+
+ @Override
+ public boolean isEqualOrFuller(Command command)
+ {
+ if (!super.isEqualOrFuller(command)) return false;
+ return Objects.equals(executesAtLeast, ((TruncatedAwaitsOnlyDeps)command).executesAtLeast);
+ }
+
+ }
+
public static class PreAccepted extends AbstractCommand
{
private final Timestamp executeAt;
@@ -1030,6 +1095,14 @@
}
@Override
+ public Timestamp executesAtLeast()
+ {
+ if (!txnId().kind().awaitsOnlyDeps()) return executeAt();
+ if (status().hasBeen(Stable)) return waitingOn.executeAtLeast(executeAt());
+ return null;
+ }
+
+ @Override
public Command updateAttributes(CommonAttributes attrs, Ballot promised)
{
return validate(new Committed(attrs, saveStatus(), executeAt(), promised, acceptedOrCommitted(), waitingOn()));
@@ -1208,6 +1281,11 @@
return null;
}
+ public Timestamp executeAtLeast(Timestamp ifNull)
+ {
+ return ifNull;
+ }
+
public static WaitingOn none(Deps deps)
{
ImmutableBitSet empty = new ImmutableBitSet(deps.txnIdCount());
@@ -1585,6 +1663,12 @@
}
@Override
+ public Timestamp executeAtLeast(Timestamp ifNull)
+ {
+ return executeAtLeast != null ? executeAtLeast : ifNull;
+ }
+
+ @Override
public boolean equals(Object other)
{
return other.getClass() == WaitingOnWithExecuteAt.class && this.equals((WaitingOnWithExecuteAt) other);
diff --git a/accord-core/src/main/java/accord/local/CommandStore.java b/accord-core/src/main/java/accord/local/CommandStore.java
index 2ed1481..cf774b8 100644
--- a/accord-core/src/main/java/accord/local/CommandStore.java
+++ b/accord-core/src/main/java/accord/local/CommandStore.java
@@ -30,7 +30,7 @@
import accord.local.CommandStores.RangesForEpoch;
import accord.primitives.Keys;
import accord.primitives.Range;
-import accord.primitives.Txn.Kind.Kinds;
+import accord.primitives.Routables;
import accord.utils.async.AsyncChain;
import accord.api.ConfigurationService.EpochReady;
@@ -67,7 +67,7 @@
import accord.utils.async.AsyncResults;
import static accord.api.ConfigurationService.EpochReady.DONE;
-import static accord.local.KeyHistory.DEPS;
+import static accord.local.KeyHistory.COMMANDS;
import static accord.local.PreLoadContext.contextFor;
import static accord.local.PreLoadContext.empty;
import static accord.primitives.AbstractRanges.UnionMode.MERGE_ADJACENT;
@@ -149,9 +149,11 @@
// TODO (expected): schedule regular pruning of these collections
// bootstrapBeganAt and shardDurableAt are both canonical data sets mostly used for debugging / constructing
private NavigableMap<TxnId, Ranges> bootstrapBeganAt = ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY); // additive (i.e. once inserted, rolled-over until invalidated, and the floor entry contains additions)
+ // TODO (required): updates to these variables should be buffered like with Command updates in SafeCommandStore, to be rolled-back in event of a failure processing the command.
private RedundantBefore redundantBefore = RedundantBefore.EMPTY;
// TODO (expected): store this only once per node
private DurableBefore durableBefore = DurableBefore.EMPTY;
+ private MaxConflicts maxConflicts = MaxConflicts.EMPTY;
protected RangesForEpoch rangesForEpoch;
// TODO (desired): merge with redundantBefore?
@@ -223,21 +225,6 @@
public abstract <T> AsyncChain<T> submit(PreLoadContext context, Function<? super SafeCommandStore, T> apply);
public abstract void shutdown();
- private static Timestamp maxApplied(SafeCommandStore safeStore, Seekables<?, ?> keysOrRanges, Ranges slice)
- {
- return safeStore.mapReduce(keysOrRanges, slice, KeyHistory.NONE, Kinds.Ws,
- SafeCommandStore.TestTimestamp.STARTED_AFTER, Timestamp.NONE,
- SafeCommandStore.TestDep.ANY_DEPS, null,
- Status.PreApplied, Status.Truncated,
- (p1, key, txnId, executeAt, status, deps, max) -> Timestamp.max(max, executeAt),
- null, Timestamp.NONE, Timestamp.MAX::equals);
- }
-
- public AsyncChain<Timestamp> maxAppliedFor(Seekables<?, ?> keysOrRanges, Ranges slice)
- {
- return submit(PreLoadContext.contextFor(keysOrRanges), safeStore -> maxApplied(safeStore, keysOrRanges, slice));
- }
-
// implementations are expected to override this for persistence
protected void setRejectBefore(ReducingRangeMap<Timestamp> newRejectBefore)
{
@@ -269,7 +256,6 @@
this.durableBefore = durableBefore;
}
-
/**
* To be overridden by implementations, to ensure the new state is persisted.
*/
@@ -279,6 +265,20 @@
}
/**
+ * To be overridden by implementations, to ensure the new state is persisted.
+ */
+ protected void setMaxConflicts(MaxConflicts maxConflicts)
+ {
+ this.maxConflicts = maxConflicts;
+ }
+
+ protected void updateMaxConflicts(Command prev, Command updated, Seekables<?, ?> keysOrRanges)
+ {
+ if (updated.executeAt() != null && (prev == null || prev.executeAt() == null || prev.executeAt().compareTo(updated.executeAt()) < 0))
+ setMaxConflicts(maxConflicts.update(keysOrRanges != null ? keysOrRanges : updated.partialTxn().keys(), updated.executeAt()));
+ }
+
+ /**
* This method may be invoked on a non-CommandStore thread
*/
protected synchronized void setSafeToRead(NavigableMap<Timestamp, Ranges> newSafeToRead)
@@ -303,6 +303,9 @@
setRedundantBefore(newRedundantBefore);
}
+ /**
+ * We expect keys to be sliced to those owned by the replica in the coordination epoch
+ */
final Timestamp preaccept(TxnId txnId, Seekables<?, ?> keys, SafeCommandStore safeStore, boolean permitFastPath)
{
NodeTimeService time = safeStore.time();
@@ -321,11 +324,17 @@
// TODO (expected): reject if any transaction exists with a higher timestamp OR a higher epoch
// this permits us to agree fast path decisions across epoch changes
- Timestamp maxConflict = safeStore.maxConflict(keys, safeStore.ranges().coordinates(txnId));
- if (permitFastPath && txnId.compareTo(maxConflict) > 0 && txnId.epoch() >= time.epoch())
+ // TODO (expected): we should (perhaps) separate conflicts for reads and writes
+ Timestamp minNonConflicting = maxConflicts.get(keys);
+ if (permitFastPath && txnId.compareTo(minNonConflicting) >= 0 && txnId.epoch() >= time.epoch())
return txnId;
- return time.uniqueNow(maxConflict);
+ return time.uniqueNow(minNonConflicting);
+ }
+
+ public Timestamp maxConflict(Routables<?> keysOrRanges)
+ {
+ return maxConflicts.get(keysOrRanges);
}
protected void unsafeRunIn(Runnable fn)
@@ -443,6 +452,7 @@
TxnId id = TxnId.fromValues(epoch - 1, 0, node.id());
Timestamp before = Timestamp.minForEpoch(epoch);
FullRoute<?> route = node.computeRoute(id, ranges);
+ // TODO (required): we need to ensure anyone we receive a reply from proposes newer timestamps for anything we don't see
CollectDeps.withDeps(node, id, route, ranges, before, (deps, fail) -> {
if (fail != null)
{
@@ -452,7 +462,7 @@
{
// TODO (correcness) : PreLoadContext only works with Seekables, which doesn't allow mixing Keys and Ranges... But Deps has both Keys AND Ranges!
// ATM all known implementations store ranges in-memory, but this will not be true soon, so this will need to be addressed
- execute(contextFor(null, deps.txnIds(), deps.keyDeps.keys(), DEPS), safeStore -> {
+ execute(contextFor(null, deps.txnIds(), deps.keyDeps.keys(), COMMANDS), safeStore -> {
safeStore.registerHistoricalTransactions(deps);
}).begin((success, fail2) -> {
if (fail2 != null) fetchMajorityDeps(coordination, node, epoch, ranges);
diff --git a/accord-core/src/main/java/accord/local/Commands.java b/accord-core/src/main/java/accord/local/Commands.java
index 7f19b2a..6c8b77e 100644
--- a/accord-core/src/main/java/accord/local/Commands.java
+++ b/accord-core/src/main/java/accord/local/Commands.java
@@ -42,7 +42,6 @@
import accord.primitives.PartialTxn;
import accord.primitives.Participants;
import accord.primitives.Ranges;
-import accord.primitives.Routables;
import accord.primitives.Route;
import accord.primitives.Seekables;
import accord.primitives.SyncPoint;
@@ -61,10 +60,8 @@
import static accord.api.ProgressLog.ProgressShard.UnmanagedHome;
import static accord.api.ProgressLog.ProgressShard.Unsure;
import static accord.local.Cleanup.ERASE;
-import static accord.local.Cleanup.TRUNCATE;
import static accord.local.Cleanup.shouldCleanup;
import static accord.local.Command.Truncated.erased;
-import static accord.local.Command.Truncated.invalidated;
import static accord.local.Command.Truncated.truncatedApply;
import static accord.local.Command.Truncated.truncatedApplyWithOutcome;
import static accord.local.Commands.EnsureAction.Add;
@@ -72,6 +69,8 @@
import static accord.local.Commands.EnsureAction.Set;
import static accord.local.Commands.EnsureAction.TryAdd;
import static accord.local.Commands.EnsureAction.TrySet;
+import static accord.local.KeyHistory.TIMESTAMPS;
+import static accord.local.PreLoadContext.contextFor;
import static accord.local.RedundantStatus.PRE_BOOTSTRAP_OR_STALE;
import static accord.local.SaveStatus.Applying;
import static accord.local.SaveStatus.Erased;
@@ -84,6 +83,7 @@
import static accord.local.Status.Committed;
import static accord.local.Status.Durability;
import static accord.local.Status.Invalidated;
+import static accord.local.Status.KnownExecuteAt.ExecuteAtKnown;
import static accord.local.Status.NotDefined;
import static accord.local.Status.PreAccepted;
import static accord.local.Status.PreApplied;
@@ -174,7 +174,7 @@
Invariants.checkState(validate(command.status(), command, Ranges.EMPTY, preacceptRanges, shard, route, Set, partialTxn, Set, null, Ignore));
// FIXME: this should go into a consumer method
- CommonAttributes attrs = set(safeStore, command, command, Ranges.EMPTY, preacceptRanges, shard, route, partialTxn, Set, null, Ignore);
+ CommonAttributes attrs = set(command, Ranges.EMPTY, preacceptRanges, shard, route, partialTxn, Set, null, Ignore);
if (command.executeAt() == null)
{
// unlike in the Accord paper, we partition shards within a node, so that to ensure a total order we must either:
@@ -183,14 +183,14 @@
// if we are performing recovery (i.e. non-zero ballot), do not permit a fast path decision as we want to
// invalidate any transactions that were not completed by their initial coordinator
// TODO (desired): limit preaccept to keys we include, to avoid inflating unnecessary state
- Timestamp executeAt = safeStore.commandStore().preaccept(txnId, partialTxn.keys(), safeStore, ballot.equals(Ballot.ZERO));
- command = safeCommand.preaccept(attrs, executeAt, ballot);
+ Timestamp executeAt = safeStore.commandStore().preaccept(txnId, attrs.partialTxn().keys(), safeStore, ballot.equals(Ballot.ZERO));
+ command = safeCommand.preaccept(safeStore, attrs, executeAt, ballot);
safeStore.progressLog().preaccepted(command, shard);
}
else
{
// TODO (expected, ?): in the case that we are pre-committed but had not been preaccepted/accepted, should we inform progressLog?
- safeCommand.markDefined(attrs, ballot);
+ safeCommand.markDefined(safeStore, attrs, ballot);
}
safeStore.notifyListeners(safeCommand);
@@ -218,7 +218,7 @@
return true;
}
- public static AcceptOutcome accept(SafeCommandStore safeStore, TxnId txnId, Ballot ballot, PartialRoute<?> route, Seekables<?, ?> keys, @Nullable RoutingKey progressKey, Timestamp executeAt, PartialDeps partialDeps)
+ public static AcceptOutcome accept(SafeCommandStore safeStore, TxnId txnId, Ballot ballot, PartialRoute<?> route, Seekables<?, ?> keysOrRanges, @Nullable RoutingKey progressKey, Timestamp executeAt, PartialDeps partialDeps)
{
SafeCommand safeCommand = safeStore.get(txnId, executeAt, route);
Command command = safeCommand.current();
@@ -256,13 +256,9 @@
// TODO (desired, clarity/efficiency): we don't need to set the route here, and perhaps we don't even need to
// distributed partialDeps at all, since all we gain is not waiting for these transactions to commit during
// recovery. We probably don't want to directly persist a Route in any other circumstances, either, to ease persistence.
- CommonAttributes attrs = set(safeStore, command, command, coordinateRanges, acceptRanges, shard, route, null, Ignore, partialDeps, Set);
+ CommonAttributes attrs = set(command, coordinateRanges, acceptRanges, shard, route, null, Ignore, partialDeps, Set);
- // set only registers by transaction keys, which we mightn't already have received
- if (!command.known().isDefinitionKnown())
- safeStore.register(keys, acceptRanges, command);
-
- command = safeCommand.accept(attrs, executeAt, ballot);
+ command = safeCommand.accept(safeStore, keysOrRanges, attrs, executeAt, ballot);
safeStore.progressLog().accepted(command, shard);
safeStore.notifyListeners(safeCommand);
@@ -299,7 +295,7 @@
logger.trace("{}: accepted invalidated", command.txnId());
- safeCommand.acceptInvalidated(ballot);
+ safeCommand.acceptInvalidated(safeStore, ballot);
safeStore.notifyListeners(safeCommand);
return AcceptOutcome.Success;
}
@@ -348,13 +344,13 @@
return CommitOutcome.Insufficient;
// FIXME: split up set
- CommonAttributes attrs = set(safeStore, command, command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, Set);
+ CommonAttributes attrs = set(command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, Set);
logger.trace("{}: committed with executeAt: {}, deps: {}", txnId, executeAt, partialDeps);
if (newStatus == SaveStatus.Stable)
{
WaitingOn waitingOn = initialiseWaitingOn(safeStore, txnId, executeAt, attrs.partialDeps(), attrs.route());
- command = safeCommand.stable(attrs, Ballot.max(command.acceptedOrCommitted(), ballot), executeAt, waitingOn);
+ command = safeCommand.stable(safeStore, attrs, Ballot.max(command.acceptedOrCommitted(), ballot), executeAt, waitingOn);
safeStore.progressLog().stable(command, shard);
safeStore.agent().metricsEventsListener().onStable(command);
// TODO (expected, safety): introduce intermediate status to avoid reentry when notifying listeners (which might notify us)
@@ -363,7 +359,7 @@
else
{
Invariants.checkArgument(command.acceptedOrCommitted().compareTo(ballot) <= 0);
- command = safeCommand.commit(attrs, ballot, executeAt);
+ command = safeCommand.commit(safeStore, attrs, ballot, executeAt);
safeStore.progressLog().precommitted(command);
safeStore.notifyListeners(safeCommand);
safeStore.agent().metricsEventsListener().onCommitted(command);
@@ -397,7 +393,7 @@
if (command.route() == null || !command.route().kind().isFullRoute())
attrs = updateRoute(command, route);
- safeCommand.precommit(attrs, executeAt);
+ safeCommand.precommit(safeStore, attrs, executeAt);
safeStore.progressLog().precommitted(command);
safeStore.notifyListeners(safeCommand);
logger.trace("{}: precommitted with executeAt: {}", txnId, executeAt);
@@ -426,8 +422,8 @@
PartialDeps none = Deps.NONE.slice(coordinateRanges);
PartialTxn partialTxn = emptyTxn.slice(coordinateRanges, true);
Invariants.checkState(validate(command.status(), command, Ranges.EMPTY, coordinateRanges, progressShard, route, Set, partialTxn, Set, none, Set));
- CommonAttributes newAttributes = set(safeStore, command, command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, none, Set);
- safeCommand.stable(newAttributes, Ballot.ZERO, localSyncId, WaitingOn.EMPTY);
+ CommonAttributes newAttributes = set(command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, none, Set);
+ safeCommand.stable(safeStore, newAttributes, Ballot.ZERO, localSyncId, WaitingOn.EMPTY);
safeStore.notifyListeners(safeCommand);
}
@@ -446,8 +442,8 @@
// TODO (desired, consider): in the case of sync points, the coordinator is unlikely to be a home shard, do we mind this? should document at least
ProgressShard progressShard = No;
Invariants.checkState(validate(command.status(), command, Ranges.EMPTY, coordinateRanges, progressShard, route, Set, partialTxn, Set, partialDeps, Set));
- CommonAttributes attrs = set(safeStore, command, command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, partialDeps, Set);
- safeCommand.stable(attrs, Ballot.ZERO, txnId, initialiseWaitingOn(safeStore, txnId, txnId, attrs.partialDeps(), route));
+ CommonAttributes attrs = set(command, Ranges.EMPTY, coordinateRanges, progressShard, route, partialTxn, Set, partialDeps, Set);
+ safeCommand.stable(safeStore, attrs, Ballot.ZERO, txnId, initialiseWaitingOn(safeStore, txnId, txnId, attrs.partialDeps(), route));
maybeExecute(safeStore, safeCommand, false, true);
}
@@ -460,7 +456,7 @@
// NOTE: if this is ever made a non-empty txn this will introduce a potential bug where the txn is registered against CommandsForKeys
Txn emptyTxn = safeStore.agent().emptyTxn(localSyncId.kind(), keys);
- safeCommand.preapplied(command, command.executeAt(), command.waitingOn(), emptyTxn.execute(localSyncId, localSyncId, null), emptyTxn.result(localSyncId, localSyncId, null));
+ safeCommand.preapplied(safeStore, command, command.executeAt(), command.waitingOn(), emptyTxn.execute(localSyncId, localSyncId, null), emptyTxn.result(localSyncId, localSyncId, null));
maybeExecute(safeStore, safeCommand, true, false);
}
@@ -485,7 +481,7 @@
else if (command.saveStatus().isUninitialised() && !safeStore.ranges().allAt(command.txnId().epoch()).intersects(scope))
return; // don't bother propagating the invalidation to future epochs where the replica didn't already witness the command
- safeCommand.commitInvalidated();
+ safeCommand.commitInvalidated(safeStore);
safeStore.progressLog().clear(command.txnId());
logger.trace("{}: committed invalidated", safeCommand.txnId());
safeStore.notifyListeners(safeCommand, command, command.durableListeners(), safeCommand.transientListeners());
@@ -516,10 +512,10 @@
if (!validate(command.status(), command, coordinateRanges, acceptRanges, shard, route, TryAdd, partialTxn, Add, partialDeps, command.hasBeen(Committed) ? TryAdd : TrySet, safeStore))
return ApplyOutcome.Insufficient; // TODO (expected, consider): this should probably be an assertion failure if !TrySet
- CommonAttributes attrs = set(safeStore, command, command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, command.hasBeen(Committed) ? TryAdd : TrySet);
+ CommonAttributes attrs = set(command, coordinateRanges, acceptRanges, shard, route, partialTxn, Add, partialDeps, command.hasBeen(Committed) ? TryAdd : TrySet);
WaitingOn waitingOn = !command.hasBeen(Stable) ? initialiseWaitingOn(safeStore, txnId, executeAt, attrs.partialDeps(), attrs.route()) : command.asCommitted().waitingOn();
- safeCommand.preapplied(attrs, executeAt, waitingOn, writes, result);
+ safeCommand.preapplied(safeStore, attrs, executeAt, waitingOn, writes, result);
safeStore.notifyListeners(safeCommand);
logger.trace("{}: apply, status set to Executed with executeAt: {}, deps: {}", txnId, executeAt, partialDeps);
@@ -573,7 +569,7 @@
{
logger.trace("{} applied, setting status to Applied and notifying listeners", txnId);
SafeCommand safeCommand = safeStore.get(txnId);
- safeCommand.applied();
+ safeCommand.applied(safeStore);
safeStore.notifyListeners(safeCommand);
}
@@ -638,7 +634,8 @@
{
CommandStore unsafeStore = safeStore.commandStore();
TxnId txnId = command.txnId();
- PreLoadContext context = command.contextForSelf();
+ // TODO (expected): there is some coupling going on here - concept of TIMESTAMPS only needed if implementation tracks on apply
+ PreLoadContext context = contextFor(command.txnId(), command.partialTxn().keys(), TIMESTAMPS);
// this is sometimes called from a listener update, which will not have the keys in context
if (safeStore.canExecuteWith(context))
{
@@ -684,7 +681,7 @@
{
case Stable:
// TODO (desirable, efficiency): maintain distinct ReadyToRead and ReadyToWrite states
- command = safeCommand.readyToExecute();
+ command = safeCommand.readyToExecute(safeStore);
logger.trace("{}: set to ReadyToExecute", command.txnId());
safeStore.progressLog().readyToExecute(command);
safeStore.notifyListeners(safeCommand);
@@ -697,7 +694,7 @@
if (intersects)
{
- safeCommand.applying();
+ safeCommand.applying(safeStore);
logger.trace("{}: applying", command.txnId());
apply(safeStore, executed);
return true;
@@ -707,7 +704,7 @@
// TODO (desirable, performance): This could be performed immediately upon Committed
// but: if we later support transitive dependency elision this could be dangerous
logger.trace("{}: applying no-op", command.txnId());
- safeCommand.applied();
+ safeCommand.applied(safeStore);
if (command.txnId().kind() == ExclusiveSyncPoint)
{
Ranges ranges = safeStore.ranges().allAt(command.txnId().epoch());
@@ -775,7 +772,7 @@
Command dependency = dependencySafeCommand.current();
Invariants.checkState(dependency.hasBeen(PreCommitted));
TxnId dependencyId = dependency.txnId();
- if (waitingId.kind().awaitsOnlyDeps() && !dependency.is(Invalidated) && dependency.executeAt().compareTo(waitingId) > 0)
+ if (waitingId.kind().awaitsOnlyDeps() && dependency.known().executeAt == ExecuteAtKnown && dependency.executeAt().compareTo(waitingId) > 0)
waitingOn.updateExecuteAtLeast(dependency.executeAt());
if (dependency.hasBeen(Truncated))
@@ -859,25 +856,47 @@
// TODO (now): document and justify all calls
public static void setTruncatedApply(SafeCommandStore safeStore, SafeCommand safeCommand)
{
- setTruncatedApply(safeStore, safeCommand, null);
+ setTruncatedApply(safeStore, safeCommand, null, null);
}
- public static void setTruncatedApply(SafeCommandStore safeStore, SafeCommand safeCommand, Route<?> maybeFullRoute)
+ public static void setTruncatedApply(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable Timestamp executeAt, Route<?> maybeFullRoute)
{
- purge(safeStore, safeCommand, maybeFullRoute, TRUNCATE, true);
+ Command command = safeCommand.current();
+ if (command.saveStatus().compareTo(TruncatedApply) >= 0) return;
+ FullRoute<?> route = Route.tryCastToFullRoute(maybeFullRoute);
+ if (route == null) route = Route.tryCastToFullRoute(command.route());
+ if (executeAt == null) executeAt = command.executeAtIfKnown();
+ if (route == null || executeAt == null)
+ {
+ safeCommand.update(safeStore, null, erased(command));
+ }
+ else
+ {
+ CommonAttributes attributes = command.mutable().route(route);
+ if (!safeCommand.txnId().kind().awaitsOnlyDeps())
+ {
+ safeCommand.update(safeStore, null, truncatedApply(attributes, TruncatedApply, executeAt, null, null));
+ }
+ else if (safeCommand.current().saveStatus().hasBeen(Applied))
+ {
+ Timestamp executesAtLeast = safeCommand.current().executesAtLeast();
+ if (executesAtLeast == null) safeCommand.update(safeStore, null, erased(command));
+ else safeCommand.update(safeStore, null, truncatedApply(attributes, TruncatedApply, executeAt, null, null, executesAtLeast));
+ }
+ }
}
public static void setErased(SafeCommandStore safeStore, SafeCommand safeCommand)
{
Listeners.Immutable durableListeners = safeCommand.current().durableListeners();
- Command command = purge(safeStore, safeCommand, null, ERASE, true);
+ Command command = purge(safeStore, safeCommand, null, null, ERASE, true);
safeStore.notifyListeners(safeCommand, command, durableListeners, safeCommand.transientListeners());
}
/**
* Purge all or part of the metadata for a Commmand
*/
- public static Command purge(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable Unseekables<?> maybeFullRoute, Cleanup cleanup, boolean notifyListeners)
+ public static Command purge(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable Seekables<?, ?> keysOrRanges, @Nullable Unseekables<?> maybeFullRoute, Cleanup cleanup, boolean notifyListeners)
{
Command command = safeCommand.current();
@@ -893,7 +912,7 @@
Command result;
switch (cleanup)
{
- default: throw new AssertionError("Unexpected status: " + cleanup);
+ default: throw new AssertionError("Unexpected cleanup: " + cleanup);
case TRUNCATE_WITH_OUTCOME:
Invariants.checkArgument(!command.hasBeen(Truncated));
if (command.hasBeen(PreApplied))
@@ -904,8 +923,9 @@
// TODO (expected, consider): should we downgrade to no truncation in this case? Or are we stale?
case TRUNCATE:
+ // TODO (expected): consider passing through any information we have about the reason for loading, so we can infer APPLIED if !PreCommitted
Invariants.checkState(command.saveStatus().compareTo(TruncatedApply) < 0);
- if (!command.hasBeen(PreCommitted)) result = invalidated(command);
+ if (!command.hasBeen(PreCommitted)) result = erased(command);
else result = truncatedApply(command, Route.tryCastToFullRoute(maybeFullRoute));
break;
@@ -915,7 +935,7 @@
break;
}
- safeCommand.set(result);
+ safeCommand.update(safeStore, keysOrRanges, result);
safeStore.progressLog().clear(safeCommand.txnId());
if (notifyListeners)
safeStore.notifyListeners(safeCommand, result, command.durableListeners(), safeCommand.transientListeners());
@@ -928,7 +948,7 @@
if (command.saveStatus().compareTo(cleanup.appliesIfNot) >= 0)
return false;
- purge(safeStore, safeCommand, maybeFullRoute, cleanup, true);
+ purge(safeStore, safeCommand, null, maybeFullRoute, cleanup, true);
return true;
}
@@ -1182,7 +1202,7 @@
}
}
- static Command removeRedundantDependencies(SafeCommandStore safeStore, SafeCommand safeCommand, TxnId redundant)
+ static Command removeRedundantDependencies(SafeCommandStore safeStore, SafeCommand safeCommand, @Nullable TxnId redundant)
{
CommandStore commandStore = safeStore.commandStore();
Command.Committed current = safeCommand.current().asCommitted();
@@ -1192,7 +1212,8 @@
safeStore.commandStore().removeRedundantDependencies(current.route().participants(), update);
// if we are a range transaction, being redundant for this transaction does not imply we are redundant for all transactions
- update.removeWaitingOn(redundant);
+ if (redundant != null)
+ update.removeWaitingOn(redundant);
return safeCommand.updateWaitingOn(update);
}
@@ -1255,14 +1276,14 @@
}
@SuppressWarnings({"unchecked", "rawtypes"})
- private static CommonAttributes set(SafeCommandStore safeStore, Command command, CommonAttributes attrs,
- Ranges existingRanges, Ranges additionalRanges, ProgressShard shard, Route<?> route,
+ private static CommonAttributes set(CommonAttributes attrs,
+ Ranges existingRanges, Ranges additionalRanges,
+ ProgressShard shard, Route<?> route,
@Nullable PartialTxn partialTxn, EnsureAction ensurePartialTxn,
@Nullable PartialDeps partialDeps, EnsureAction ensurePartialDeps)
{
Invariants.checkState(shard != Unsure);
Ranges allRanges = existingRanges.with(additionalRanges);
-
attrs = attrs.mutable().route(Route.merge(attrs.route(), (Route)route));
// TODO (soon): stop round-robin hashing; partition only on ranges
@@ -1275,15 +1296,6 @@
if (attrs.partialTxn() != null)
{
partialTxn = partialTxn.slice(allRanges, shard.isHome());
- if (command.txnId().kind().isGloballyVisible())
- {
- Invariants.checkState(attrs.partialTxn().covers(existingRanges));
- Routables.foldl(partialTxn.keys(), additionalRanges, (keyOrRange, p, v, i) -> {
- // TODO (expected, efficiency): we may register the same ranges more than once
- safeStore.register(keyOrRange, allRanges, command);
- return v;
- }, 0, 0, 1);
- }
attrs = attrs.mutable().partialTxn(attrs.partialTxn().with(partialTxn));
break;
}
@@ -1291,11 +1303,7 @@
case Set:
case TrySet:
// TODO (desired): only includeQuery if shard.isHome(); this affects state eviction and is low priority given size in C*
- attrs = attrs.mutable().partialTxn(partialTxn = partialTxn.slice(allRanges, true));
- // TODO (expected, efficiency): we may register the same ranges more than once
- // TODO (desirable, efficiency): no need to register on PreAccept if already Accepted
- if (command.txnId().kind().isGloballyVisible())
- safeStore.register(partialTxn.keys(), allRanges, command);
+ attrs = attrs.mutable().partialTxn(partialTxn.slice(allRanges, true));
break;
}
diff --git a/accord-core/src/main/java/accord/local/DurableBefore.java b/accord-core/src/main/java/accord/local/DurableBefore.java
index ae872dc..9da0723 100644
--- a/accord-core/src/main/java/accord/local/DurableBefore.java
+++ b/accord-core/src/main/java/accord/local/DurableBefore.java
@@ -25,7 +25,6 @@
import accord.api.RoutingKey;
import accord.local.Status.Durability;
import accord.primitives.Participants;
-import accord.primitives.Range;
import accord.primitives.Ranges;
import accord.primitives.TxnId;
import accord.primitives.Unseekables;
@@ -154,20 +153,7 @@
return DurableBefore.EMPTY;
Entry entry = new Entry(majority, universal);
- Builder builder = new Builder(ranges.get(0).endInclusive(), ranges.size() * 2);
- Range prev = null;
- for (int i = 0 ; i < ranges.size() ; ++i)
- {
- Range cur = ranges.get(i);
- if (prev != null && !prev.end().equals(cur.start()))
- builder.append(prev.end(), null, (a, b) -> a); // if we are equal to prev end, take the prev value not zero
- builder.append(cur.start(), entry, (a, b) -> { throw new IllegalStateException(); });
- prev = cur;
- }
- if (prev != null)
- builder.append(prev.end(), null, (a, b) -> a); // if we are equal to prev end, take the prev value not zero
-
- return builder.build();
+ return create(ranges, entry, Builder::new);
}
public static DurableBefore merge(DurableBefore a, DurableBefore b)
diff --git a/accord-core/src/main/java/accord/local/KeyHistory.java b/accord-core/src/main/java/accord/local/KeyHistory.java
index d5c9c3d..7f061d6 100644
--- a/accord-core/src/main/java/accord/local/KeyHistory.java
+++ b/accord-core/src/main/java/accord/local/KeyHistory.java
@@ -24,8 +24,8 @@
*/
public enum KeyHistory
{
- DEPS,
- ALL,
+ TIMESTAMPS,
+ COMMANDS,
NONE;
public boolean isNone()
diff --git a/accord-core/src/main/java/accord/local/Listeners.java b/accord-core/src/main/java/accord/local/Listeners.java
index 4afd6b4..1392971 100644
--- a/accord-core/src/main/java/accord/local/Listeners.java
+++ b/accord-core/src/main/java/accord/local/Listeners.java
@@ -23,6 +23,7 @@
import accord.utils.DeterministicSet;
+// TODO (expected): these are immutable collections over sortable keys, so should simply use BTree (or arrays)
public class Listeners<L extends Command.Listener> extends DeterministicSet<L>
{
public static Listeners EMPTY = new Listeners<>();
diff --git a/accord-core/src/main/java/accord/local/MaxConflicts.java b/accord-core/src/main/java/accord/local/MaxConflicts.java
new file mode 100644
index 0000000..3ccf3c3
--- /dev/null
+++ b/accord-core/src/main/java/accord/local/MaxConflicts.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.local;
+
+import javax.annotation.Nonnull;
+
+import accord.api.RoutingKey;
+import accord.primitives.Ranges;
+import accord.primitives.Routables;
+import accord.primitives.Seekables;
+import accord.primitives.Timestamp;
+import accord.utils.ReducingIntervalMap;
+import accord.utils.ReducingRangeMap;
+
+// TODO (expected): track read/write conflicts separately
+public class MaxConflicts extends ReducingRangeMap<Timestamp>
+{
+ public static final MaxConflicts EMPTY = new MaxConflicts();
+
+ private MaxConflicts()
+ {
+ super();
+ }
+
+ private MaxConflicts(boolean inclusiveEnds, RoutingKey[] starts, Timestamp[] values)
+ {
+ super(inclusiveEnds, starts, values);
+ }
+
+ public Timestamp get(Seekables<?, ?> keysOrRanges)
+ {
+ return foldl(keysOrRanges, Timestamp::max, Timestamp.NONE);
+ }
+
+ public Timestamp get(Routables<?> keysOrRanges)
+ {
+ return foldl(keysOrRanges, Timestamp::max, Timestamp.NONE);
+ }
+
+ MaxConflicts update(Seekables<?, ?> keysOrRanges, Timestamp maxConflict)
+ {
+ return merge(this, create(keysOrRanges, maxConflict));
+ }
+
+ public static MaxConflicts create(Ranges ranges, @Nonnull Timestamp maxConflict)
+ {
+ if (ranges.isEmpty())
+ return MaxConflicts.EMPTY;
+
+ return create(ranges, maxConflict, MaxConflicts.Builder::new);
+ }
+
+ public static MaxConflicts create(Seekables<?, ?> keysOrRanges, @Nonnull Timestamp maxConflict)
+ {
+ if (keysOrRanges.isEmpty())
+ return MaxConflicts.EMPTY;
+
+ return create(keysOrRanges, maxConflict, Builder::new);
+ }
+
+ public static MaxConflicts merge(MaxConflicts a, MaxConflicts b)
+ {
+ return ReducingIntervalMap.merge(a, b, Timestamp::max, MaxConflicts.Builder::new);
+ }
+
+ static class Builder extends AbstractBoundariesBuilder<RoutingKey, Timestamp, MaxConflicts>
+ {
+ protected Builder(boolean inclusiveEnds, int capacity)
+ {
+ super(inclusiveEnds, capacity);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ protected MaxConflicts buildInternal()
+ {
+ return new MaxConflicts(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new Timestamp[0]));
+ }
+ }
+}
diff --git a/accord-core/src/main/java/accord/local/Node.java b/accord-core/src/main/java/accord/local/Node.java
index 413055b..3bf2186 100644
--- a/accord-core/src/main/java/accord/local/Node.java
+++ b/accord-core/src/main/java/accord/local/Node.java
@@ -38,6 +38,8 @@
import javax.annotation.Nullable;
import accord.coordinate.CoordinateEphemeralRead;
+import accord.coordinate.CoordinationAdapter;
+import accord.coordinate.CoordinationAdapter.Factory.Step;
import accord.utils.DeterministicSet;
import accord.utils.Invariants;
import com.google.common.annotations.VisibleForTesting;
@@ -59,12 +61,9 @@
import accord.coordinate.Barrier;
import accord.config.LocalConfig;
import accord.coordinate.CoordinateTransaction;
-import accord.coordinate.Execute;
import accord.coordinate.MaybeRecover;
import accord.coordinate.Outcome;
-import accord.coordinate.Persist;
import accord.coordinate.RecoverWithRoute;
-import accord.messages.Apply;
import accord.messages.Callback;
import accord.messages.LocalRequest;
import accord.messages.Reply;
@@ -155,9 +154,7 @@
private final ConfigurationService configService;
private final TopologyManager topology;
private final CommandStores commandStores;
- private final Execute.Factory executionFactory;
- private final Persist.Factory persistFactory;
- private final Apply.Factory applyFactory;
+ private final CoordinationAdapter.Factory coordinationAdapters;
private final LongSupplier nowSupplier;
private final ToLongFunction<TimeUnit> nowTimeUnit;
@@ -175,7 +172,7 @@
public Node(Id id, MessageSink messageSink, LocalRequest.Handler localRequestHandler,
ConfigurationService configService, LongSupplier nowSupplier, ToLongFunction<TimeUnit> nowTimeUnit,
Supplier<DataStore> dataSupplier, ShardDistributor shardDistributor, Agent agent, RandomSource random, Scheduler scheduler, TopologySorter.Supplier topologySorter,
- Function<Node, ProgressLog.Factory> progressLogFactory, CommandStores.Factory factory, Execute.Factory executionFactory, Persist.Factory persistFactory, Apply.Factory applyFactory,
+ Function<Node, ProgressLog.Factory> progressLogFactory, CommandStores.Factory factory, CoordinationAdapter.Factory coordinationAdapters,
LocalConfig localConfig)
{
this.id = id;
@@ -183,9 +180,7 @@
this.messageSink = messageSink;
this.localRequestHandler = localRequestHandler;
this.configService = configService;
- this.executionFactory = executionFactory;
- this.persistFactory = persistFactory;
- this.applyFactory = applyFactory;
+ this.coordinationAdapters = coordinationAdapters;
this.topology = new TopologyManager(topologySorter, id);
this.nowSupplier = nowSupplier;
this.nowTimeUnit = nowTimeUnit;
@@ -557,6 +552,9 @@
messageSink.reply(replyingToNode, replyContext, send);
}
+ /**
+ * TODO (required): Make sure we cannot re-issue the same txnid on startup
+ */
public TxnId nextTxnId(Txn.Kind rw, Domain domain)
{
return new TxnId(uniqueNow(), rw, domain);
@@ -615,16 +613,21 @@
public FullRoute<?> computeRoute(TxnId txnId, Seekables<?, ?> keysOrRanges)
{
- RoutingKey homeKey = trySelectHomeKey(txnId, keysOrRanges);
+ return computeRoute(txnId.epoch(), keysOrRanges);
+ }
+
+ public FullRoute<?> computeRoute(long epoch, Seekables<?, ?> keysOrRanges)
+ {
+ RoutingKey homeKey = trySelectHomeKey(epoch, keysOrRanges);
if (homeKey == null)
- homeKey = selectRandomHomeKey(txnId);
+ homeKey = selectRandomHomeKey(epoch);
return keysOrRanges.toRoute(homeKey);
}
- private @Nullable RoutingKey trySelectHomeKey(TxnId txnId, Seekables<?, ?> keysOrRanges)
+ private @Nullable RoutingKey trySelectHomeKey(long epoch, Seekables<?, ?> keysOrRanges)
{
- Ranges owned = topology().localForEpoch(txnId.epoch()).ranges();
+ Ranges owned = topology().localForEpoch(epoch).ranges();
int i = (int)keysOrRanges.findNextIntersection(0, owned, 0);
return i >= 0 ? keysOrRanges.get(i).someIntersectingRoutingKey(owned) : null;
}
@@ -670,12 +673,17 @@
public RoutingKey selectRandomHomeKey(TxnId txnId)
{
- Ranges ranges = topology().localForEpoch(txnId.epoch()).ranges();
+ return selectRandomHomeKey(txnId.epoch());
+ }
+
+ public RoutingKey selectRandomHomeKey(long epoch)
+ {
+ Ranges ranges = topology().localForEpoch(epoch).ranges();
// TODO (expected): should we try to pick keys in the same Keyspace in C*? Might want to adapt this to an Agent behaviour
if (ranges.isEmpty()) // should not really happen, but pick some other replica to serve as home key
- ranges = topology().globalForEpoch(txnId.epoch()).ranges();
+ ranges = topology().globalForEpoch(epoch).ranges();
if (ranges.isEmpty())
- throw illegalState("Unable to select a HomeKey as the topology does not have any ranges for epoch " + txnId.epoch());
+ throw illegalState("Unable to select a HomeKey as the topology does not have any ranges for epoch " + epoch);
Range range = ranges.get(random.nextInt(ranges.size()));
return range.someIntersectingRoutingKey(null);
}
@@ -743,19 +751,9 @@
scheduler.now(processMsg);
}
- public Execute.Factory executionFactory()
+ public <R> CoordinationAdapter<R> coordinationAdapter(TxnId txnId, Step step)
{
- return executionFactory;
- }
-
- public Persist.Factory persistFactory()
- {
- return persistFactory;
- }
-
- public Apply.Factory applyFactory()
- {
- return applyFactory;
+ return coordinationAdapters.get(txnId, step);
}
public Scheduler scheduler()
diff --git a/accord-core/src/main/java/accord/local/PreLoadContext.java b/accord-core/src/main/java/accord/local/PreLoadContext.java
index 0dea059..fedf6f2 100644
--- a/accord-core/src/main/java/accord/local/PreLoadContext.java
+++ b/accord-core/src/main/java/accord/local/PreLoadContext.java
@@ -94,12 +94,17 @@
* Both can be done without. For range transactions calculateDeps needs to be asynchronous anyway to support
* potentially large scans, and for register we do not need to load into memory, we can perform a blind write.
*/
+ // TODO (required): specify epochs for which we should load, so we can narrow to owned keys
default Seekables<?, ?> keys() { return Keys.EMPTY; }
default KeyHistory keyHistory() { return KeyHistory.NONE; }
default boolean isSubsetOf(PreLoadContext superset)
{
+ KeyHistory requiredHistory = keyHistory();
+ if (requiredHistory != KeyHistory.NONE && requiredHistory != superset.keyHistory())
+ return false;
+
if (superset.keys().domain() != keys().domain() || !superset.keys().containsAll(keys()))
return false;
diff --git a/accord-core/src/main/java/accord/local/RedundantBefore.java b/accord-core/src/main/java/accord/local/RedundantBefore.java
index 9697e6d..c3301a1 100644
--- a/accord-core/src/main/java/accord/local/RedundantBefore.java
+++ b/accord-core/src/main/java/accord/local/RedundantBefore.java
@@ -28,6 +28,7 @@
import accord.primitives.Participants;
import accord.primitives.Range;
import accord.primitives.Ranges;
+import accord.primitives.RoutableKey;
import accord.primitives.Seekables;
import accord.primitives.Timestamp;
import accord.primitives.TxnId;
@@ -43,6 +44,7 @@
import static accord.local.RedundantStatus.NOT_OWNED;
import static accord.local.RedundantStatus.PRE_BOOTSTRAP_OR_STALE;
import static accord.local.RedundantStatus.SHARD_REDUNDANT;
+import static accord.utils.Invariants.illegalState;
public class RedundantBefore extends ReducingRangeMap<RedundantBefore.Entry>
{
@@ -88,11 +90,6 @@
public final @Nonnull TxnId bootstrappedAt;
/**
- * The maximum of each of locallyAppliedOrInvalidatedBefore, shardAppliedOrInvalidatedBefore and bootstrappedAt.
- */
- public final @Nonnull TxnId minConflict;
-
- /**
* staleUntilAtLeast provides a minimum TxnId until which we know we will be unable to completely execute
* transactions locally for the impacted range.
*
@@ -108,7 +105,6 @@
this.locallyAppliedOrInvalidatedBefore = locallyAppliedOrInvalidatedBefore;
this.shardAppliedOrInvalidatedBefore = shardAppliedOrInvalidatedBefore;
this.bootstrappedAt = bootstrappedAt;
- this.minConflict = Timestamp.max(Timestamp.max(locallyAppliedOrInvalidatedBefore, shardAppliedOrInvalidatedBefore), bootstrappedAt);
this.staleUntilAtLeast = staleUntilAtLeast;
}
@@ -181,13 +177,6 @@
: prev == FULLY ? PARTIALLY : POST_BOOTSTRAP;
}
- static Timestamp getAndMergeMinConflict(Entry entry, @Nonnull Timestamp prev)
- {
- if (entry == null)
- return prev;
- return Timestamp.max(prev, entry.minConflict);
- }
-
static <T extends Deps> Deps.AbstractBuilder<T> collectDep(Entry entry, @Nonnull Deps.AbstractBuilder<T> prev, @Nonnull EpochSupplier minEpoch, @Nonnull EpochSupplier executeAt)
{
if (entry == null || entry.outOfBounds(minEpoch, executeAt))
@@ -221,6 +210,17 @@
return executeRanges;
}
+ static Ranges removeShardRedundant(Entry entry, @Nonnull Ranges notRedundant, TxnId txnId, @Nullable Timestamp executeAt)
+ {
+ if (entry == null || (executeAt == null ? entry.outOfBounds(txnId) : entry.outOfBounds(txnId, executeAt)))
+ return notRedundant;
+
+ if (txnId.compareTo(entry.shardAppliedOrInvalidatedBefore) < 0)
+ return notRedundant.subtract(Ranges.of(entry.range));
+
+ return notRedundant;
+ }
+
RedundantStatus get(TxnId txnId)
{
// we have to first check bootstrappedAt, since we are not locally redundant for the covered range
@@ -309,6 +309,7 @@
{
super(inclusiveEnds, starts, values);
staleRanges = extractStaleRanges(values);
+ checkParanoid(starts, values);
}
private static Ranges extractStaleRanges(Entry[] values)
@@ -375,8 +376,17 @@
return Entry.get(entry, txnId, executeAt);
}
- public RedundantStatus status(TxnId txnId, EpochSupplier executeAt, Participants<?> participants)
+ public TxnId redundantBefore(RoutableKey key)
{
+ Entry entry = get(key);
+ if (entry == null)
+ return TxnId.NONE;
+ return entry.shardAppliedOrInvalidatedBefore;
+ }
+
+ public RedundantStatus status(TxnId txnId, EpochSupplier executeAt, Participants<?> participants)
+ { // TODO (required): consider how the use of txnId for executeAt affects exclusive sync points for cleanup
+ // may want to issue synthetic sync points for local evaluation in later epochs
if (executeAt == null) executeAt = txnId;
return foldl(participants, Entry::getAndMerge, NOT_OWNED, txnId, executeAt, ignore -> false);
}
@@ -391,11 +401,6 @@
return foldl(participants, Entry::getAndMerge, PreBootstrapOrStale.NOT_OWNED, txnId, executeAt, r -> r == PARTIALLY);
}
- public Timestamp minConflict(Seekables<?, ?> participants)
- {
- return foldl(participants, Entry::getAndMergeMinConflict, Timestamp.NONE, ignore -> false);
- }
-
public <T extends Deps> Deps.AbstractBuilder<T> collectDeps(Seekables<?, ?> participants, Deps.AbstractBuilder<T> builder, EpochSupplier minEpoch, EpochSupplier executeAt)
{
return foldl(participants, Entry::collectDep, builder, minEpoch, executeAt, ignore -> false);
@@ -409,6 +414,15 @@
/**
* Subtract any ranges we consider stale or pre-bootstrap
*/
+ public Ranges removeShardRedundant(TxnId txnId, @Nonnull Timestamp executeAt, Ranges ranges)
+ {
+ Invariants.checkArgument(executeAt != null, "executeAt must not be null");
+ return foldl(ranges, Entry::removeShardRedundant, ranges, txnId, executeAt, r -> false);
+ }
+
+ /**
+ * Subtract any ranges we consider stale or pre-bootstrap
+ */
public Ranges expectToExecute(TxnId txnId, @Nonnull Timestamp executeAt, Ranges ranges)
{
Invariants.checkArgument(executeAt != null, "executeAt must not be null");
@@ -468,9 +482,32 @@
}
@Override
+ public void append(RoutingKey start, RoutingKey end, @Nonnull Entry value)
+ {
+ if (value.range.start().compareTo(start) != 0 || value.range.end().compareTo(end) != 0)
+ throw illegalState();
+ super.append(start, end, value);
+ }
+
+ @Override
protected RedundantBefore buildInternal()
{
return new RedundantBefore(inclusiveEnds, starts.toArray(new RoutingKey[0]), values.toArray(new Entry[0]));
}
}
+
+ private static void checkParanoid(RoutingKey[] starts, Entry[] values)
+ {
+ if (!Invariants.isParanoid())
+ return;
+
+ for (int i = 0 ; i < values.length ; ++i)
+ {
+ if (values[i] != null)
+ {
+ Invariants.checkArgument(starts[i].equals(values[i].range.start()));
+ Invariants.checkArgument(starts[i + 1].equals(values[i].range.end()));
+ }
+ }
+ }
}
diff --git a/accord-core/src/main/java/accord/local/SafeCommand.java b/accord-core/src/main/java/accord/local/SafeCommand.java
index 6a3a2a0..ea02ce2 100644
--- a/accord-core/src/main/java/accord/local/SafeCommand.java
+++ b/accord-core/src/main/java/accord/local/SafeCommand.java
@@ -18,10 +18,13 @@
package accord.local;
+import javax.annotation.Nullable;
+
import accord.api.Result;
import accord.local.Command.TransientListener;
import accord.local.Command.Truncated;
import accord.primitives.Ballot;
+import accord.primitives.Seekables;
import accord.primitives.Timestamp;
import accord.primitives.TxnId;
import accord.primitives.Writes;
@@ -55,7 +58,19 @@
return txnId;
}
- private <C extends Command> C update(C update)
+ // TODO (expected): it isn't ideal to pass keysOrRanges for the special case of Accept. We can either:
+ // 1 - remove the special case that permits accept without the definition
+ // 2 - store some pseudo transaction with only the keys
+ // 3 - just come up with something a bit neater
+ <C extends Command> C update(SafeCommandStore safeStore, @Nullable Seekables<?, ?> keysOrRanges, C update)
+ {
+ Command prev = current();
+ set(update);
+ safeStore.update(prev, update, keysOrRanges);
+ return update;
+ }
+
+ private <C extends Command> C incidentalUpdate(C update)
{
set(update);
return update;
@@ -63,7 +78,7 @@
public Command addListener(Command.DurableAndIdempotentListener listener)
{
- return update(Command.addListener(current(), listener));
+ return incidentalUpdate(Command.addListener(current(), listener));
}
public void addAndInvokeListener(SafeCommandStore safeStore, TransientListener listener)
@@ -77,92 +92,92 @@
Command current = current();
if (!current.durableListeners().contains(listener))
return current;
- return update(Command.removeListener(current(), listener));
+ return incidentalUpdate(Command.removeListener(current(), listener));
}
public Command.Committed updateWaitingOn(Command.WaitingOn.Update waitingOn)
{
- return update(Command.updateWaitingOn(current().asCommitted(), waitingOn));
+ return incidentalUpdate(Command.updateWaitingOn(current().asCommitted(), waitingOn));
}
public Command updateAttributes(CommonAttributes attrs)
{
- return update(current().updateAttributes(attrs));
+ return incidentalUpdate(current().updateAttributes(attrs));
}
- public Command.PreAccepted preaccept(CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
+ public Command.PreAccepted preaccept(SafeCommandStore safeStore, CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
{
- return update(Command.preaccept(current(), attrs, executeAt, ballot));
+ return update(safeStore, null, Command.preaccept(current(), attrs, executeAt, ballot));
}
- public Command.Accepted markDefined(CommonAttributes attributes, Ballot promised)
+ public Command.Accepted markDefined(SafeCommandStore safeStore, CommonAttributes attributes, Ballot promised)
{
- return update(Command.markDefined(current(), attributes, promised));
+ return update(safeStore, null, Command.markDefined(current(), attributes, promised));
}
public Command updatePromised(Ballot promised)
{
- return update(current().updatePromised(promised));
+ return incidentalUpdate(current().updatePromised(promised));
}
- public Command.Accepted accept(CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
+ public Command.Accepted accept(SafeCommandStore safeStore, Seekables<?, ?> keysOrRanges, CommonAttributes attrs, Timestamp executeAt, Ballot ballot)
{
- return update(Command.accept(current(), attrs, executeAt, ballot));
+ return update(safeStore, keysOrRanges, Command.accept(current(), attrs, executeAt, ballot));
}
- public Command.Accepted acceptInvalidated(Ballot ballot)
+ public Command.Accepted acceptInvalidated(SafeCommandStore safeStore, Ballot ballot)
{
- return update(Command.acceptInvalidated(current(), ballot));
+ return update(safeStore, null, Command.acceptInvalidated(current(), ballot));
}
- public Command.Committed commit(CommonAttributes attrs, Ballot ballot, Timestamp executeAt)
+ public Command.Committed commit(SafeCommandStore safeStore, CommonAttributes attrs, Ballot ballot, Timestamp executeAt)
{
- return update(Command.commit(current(), attrs, ballot, executeAt));
+ return update(safeStore, null, Command.commit(current(), attrs, ballot, executeAt));
}
- public Command.Committed stable(CommonAttributes attrs, Ballot ballot, Timestamp executeAt, Command.WaitingOn waitingOn)
+ public Command.Committed stable(SafeCommandStore safeStore, CommonAttributes attrs, Ballot ballot, Timestamp executeAt, Command.WaitingOn waitingOn)
{
- return update(Command.stable(current(), attrs, ballot, executeAt, waitingOn));
+ return update(safeStore, null, Command.stable(current(), attrs, ballot, executeAt, waitingOn));
}
- public Truncated commitInvalidated()
+ public Truncated commitInvalidated(SafeCommandStore safeStore)
{
Command current = current();
if (current.hasBeen(Status.Truncated))
return (Truncated) current;
- return update(Truncated.invalidated(current));
+ return update(safeStore, null, Truncated.invalidated(current));
}
- public Command precommit(CommonAttributes attrs, Timestamp executeAt)
+ public Command precommit(SafeCommandStore safeStore, CommonAttributes attrs, Timestamp executeAt)
{
- return update(Command.precommit(attrs, current(), executeAt));
+ return update(safeStore, null, Command.precommit(attrs, current(), executeAt));
}
- public Command.Committed readyToExecute()
+ public Command.Committed readyToExecute(SafeCommandStore safeStore)
{
- return update(Command.readyToExecute(current().asCommitted()));
+ return update(safeStore, null, Command.readyToExecute(current().asCommitted()));
}
- public Command.Executed preapplied(CommonAttributes attrs, Timestamp executeAt, Command.WaitingOn waitingOn, Writes writes, Result result)
+ public Command.Executed preapplied(SafeCommandStore safeStore, CommonAttributes attrs, Timestamp executeAt, Command.WaitingOn waitingOn, Writes writes, Result result)
{
- return update(Command.preapplied(current(), attrs, executeAt, waitingOn, writes, result));
+ return update(safeStore, null, Command.preapplied(current(), attrs, executeAt, waitingOn, writes, result));
}
- public Command.Executed applying()
+ public Command.Executed applying(SafeCommandStore safeStore)
{
- return update(Command.applying(current().asExecuted()));
+ return update(safeStore, null, Command.applying(current().asExecuted()));
}
- public Command.Executed applied()
+ public Command.Executed applied(SafeCommandStore safeStore)
{
- return update(Command.applied(current().asExecuted()));
+ return update(safeStore, null, Command.applied(current().asExecuted()));
}
public Command.NotDefined uninitialised()
{
Invariants.checkArgument(current() == null);
- return update(Command.NotDefined.uninitialised(txnId));
+ return incidentalUpdate(Command.NotDefined.uninitialised(txnId));
}
public Command initialise()
@@ -170,6 +185,6 @@
Command current = current();
if (!current.saveStatus().isUninitialised())
return current;
- return update(Command.NotDefined.notDefined(current, current.promised()));
+ return incidentalUpdate(Command.NotDefined.notDefined(current, current.promised()));
}
}
diff --git a/accord-core/src/main/java/accord/local/SafeCommandStore.java b/accord-core/src/main/java/accord/local/SafeCommandStore.java
index de68697..9c846ed 100644
--- a/accord-core/src/main/java/accord/local/SafeCommandStore.java
+++ b/accord-core/src/main/java/accord/local/SafeCommandStore.java
@@ -19,13 +19,8 @@
package accord.local;
import java.util.Iterator;
-import java.util.List;
-import java.util.function.Predicate;
-import java.util.function.Supplier;
import javax.annotation.Nullable;
-import com.google.common.base.Predicates;
-
import accord.api.Agent;
import accord.api.DataStore;
import accord.api.ProgressLog;
@@ -58,17 +53,17 @@
{
public interface CommandFunction<P1, I, O>
{
- O apply(P1 p1, Seekable keyOrRange, TxnId txnId, Timestamp executeAt, Status status, Supplier<List<TxnId>> depsSupplier, I in);
+ O apply(P1 p1, Seekable keyOrRange, TxnId txnId, Timestamp executeAt, I in);
}
- public enum TestTimestamp
+ public enum TestStartedAt
{
STARTED_BEFORE,
STARTED_AFTER,
- MAY_EXECUTE_BEFORE, // started before and uncommitted, or committed and executes before
- EXECUTES_AFTER
+ ANY
}
public enum TestDep { WITH, WITHOUT, ANY_DEPS }
+ public enum TestStatus { ANY_STATUS, IS_PROPOSED, IS_STABLE }
/**
* If the transaction exists (with some associated data) in the CommandStore, return it. Otherwise return null.
@@ -98,8 +93,6 @@
return maybeTruncate(safeCommand, command, txnId, null);
}
- public abstract void removeCommandFromSeekableDeps(Seekable seekable, TxnId txnId, Timestamp executeAt, Status status);
-
// decidedExecuteAt == null if not yet PreCommitted
/**
@@ -160,27 +153,45 @@
protected abstract SafeCommand getInternal(TxnId txnId);
protected abstract SafeCommand getInternalIfLoadedAndInitialised(TxnId txnId);
-
public abstract boolean canExecuteWith(PreLoadContext context);
+ protected void update(Command prev, Command updated, @Nullable Seekables<?, ?> keysOrRanges)
+ {
+ SaveStatus oldSaveStatus = prev == null ? SaveStatus.Uninitialised : prev.saveStatus();
+ SaveStatus newSaveStatus = updated.saveStatus();
+ if (keysOrRanges == null && !newSaveStatus.known.definition.isKnown())
+ return;
+
+ if (newSaveStatus.status.equals(oldSaveStatus.status) && oldSaveStatus.known.definition.isKnown())
+ return;
+
+ TxnId txnId = updated.txnId();
+ if (!txnId.kind().isGloballyVisible())
+ return;
+
+// if (keysOrRanges != null)
+// keysOrRanges = keysOrRanges.slice(ranges().allBetween(updated.txnId(), updated.executeAt()));
+ commandStore().updateMaxConflicts(prev, updated, keysOrRanges);
+ }
+
+ /**
+ * Visits keys first and then ranges, both in ascending order.
+ * Within each key or range visits all visible txnids needed for the given scope in ascending order of queried timestamp.
+ * TODO (expected): no need for slice in most (all?) cases
+ */
+ public abstract <P1, T> T mapReduceActive(Seekables<?, ?> keys, Ranges slice, @Nullable Timestamp withLowerTxnId, Kinds kinds, CommandFunction<P1, T, T> map, P1 p1, T initialValue);
+
/**
* Visits keys first and then ranges, both in ascending order.
* Within each key or range visits all unevicted txnids needed for the given scope in ascending order of queried timestamp.
*/
- public <P1, T> T mapReduce(Seekables<?, ?> keys, Ranges slice, KeyHistory keyHistory,
- Kinds testKind, TestTimestamp testTimestamp, Timestamp timestamp,
- TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus,
- CommandFunction<P1, T, T> map, P1 p1, T initialValue)
- {
- return mapReduce(keys, slice, keyHistory, testKind, testTimestamp, timestamp, testDep, depId, minStatus, maxStatus, map, p1, initialValue, Predicates.alwaysFalse());
- }
-
- public abstract <P1, T> T mapReduce(Seekables<?, ?> keys, Ranges slice, KeyHistory keyHistory,
- Kinds testKind, TestTimestamp testTimestamp, Timestamp timestamp,
- TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus,
- CommandFunction<P1, T, T> map, P1 p1, T initialValue, Predicate<? super T> terminate);
- protected abstract void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command);
- protected abstract void register(Seekable keyOrRange, Ranges slice, Command command);
+ public abstract <P1, T> T mapReduceFull(Seekables<?, ?> keys, Ranges slice,
+ TxnId testTxnId,
+ Kinds testKind,
+ TestStartedAt testStartedAt,
+ TestDep testDep,
+ TestStatus testStatus,
+ CommandFunction<P1, T, T> map, P1 p1, T initialValue);
public abstract CommandStore commandStore();
public abstract DataStore dataStore();
@@ -188,14 +199,8 @@
public abstract ProgressLog progressLog();
public abstract NodeTimeService time();
public abstract CommandStores.RangesForEpoch ranges();
- public abstract Timestamp maxConflict(Seekables<?, ?> keys, Ranges slice);
- public abstract void registerHistoricalTransactions(Deps deps);
- public abstract void erase(SafeCommand safeCommand);
- public long latestEpoch()
- {
- return time().epoch();
- }
+ public abstract void registerHistoricalTransactions(Deps deps);
public boolean isTruncated(Command command)
{
diff --git a/accord-core/src/main/java/accord/local/Status.java b/accord-core/src/main/java/accord/local/Status.java
index bb44255..6d7431e 100644
--- a/accord-core/src/main/java/accord/local/Status.java
+++ b/accord-core/src/main/java/accord/local/Status.java
@@ -22,6 +22,7 @@
import accord.primitives.Ballot;
import accord.primitives.Timestamp;
import accord.primitives.TxnId;
+import accord.utils.Invariants;
import java.util.List;
import java.util.Objects;
@@ -47,7 +48,7 @@
{
NotDefined (None, Nothing),
PreAccepted (PreAccept, DefinitionAndRoute),
- AcceptedInvalidate(Accept, Maybe, DefinitionUnknown, ExecuteAtNotWitnessed, DepsUnknown, Unknown), // may or may not have witnessed
+ AcceptedInvalidate(Accept, Maybe, DefinitionUnknown, ExecuteAtUnknown, DepsUnknown, Unknown), // may or may not have witnessed
Accepted (Accept, Covering, DefinitionUnknown, ExecuteAtProposed, DepsProposed, Unknown), // may or may not have witnessed
/**
@@ -124,7 +125,7 @@
*/
public static class Known
{
- public static final Known Nothing = new Known(Maybe, DefinitionUnknown, ExecuteAtNotWitnessed, DepsUnknown, Unknown);
+ public static final Known Nothing = new Known(Maybe, DefinitionUnknown, ExecuteAtUnknown, DepsUnknown, Unknown);
// TODO (expected): deprecate DefinitionOnly
public static final Known DefinitionOnly = new Known(Maybe, DefinitionKnown, ExecuteAtUnknown, DepsUnknown, Unknown);
public static final Known DefinitionAndRoute = new Known(Full, DefinitionKnown, ExecuteAtUnknown, DepsUnknown, Unknown);
@@ -155,6 +156,7 @@
this.executeAt = executeAt;
this.deps = deps;
this.outcome = outcome;
+ checkInvariants();
}
public Known atLeast(Known that)
@@ -415,6 +417,13 @@
{
return outcome.isInvalidated();
}
+
+ public void checkInvariants()
+ {
+ if (outcome.isInvalidated()) Invariants.checkState( deps != DepsKnown && executeAt != ExecuteAtKnown);
+ else if (outcome.isOrWasApply()) Invariants.checkState(deps != NoDeps && executeAt != NoExecuteAt);
+ Invariants.checkState(!isDefinitionKnown() || hasFullRoute());
+ }
}
public enum KnownRoute
@@ -466,11 +475,6 @@
public enum KnownExecuteAt
{
/**
- * No decision is known to have been reached. The transaction has not been witnessed so executeAt is null.
- */
- ExecuteAtNotWitnessed,
-
- /**
* No decision is known to have been reached. If executeAt is not null, it represents either when
* the transaction was witnessed, or some earlier ExecuteAtProposed that was invalidated by AcceptedInvalidate
*/
diff --git a/accord-core/src/main/java/accord/messages/Accept.java b/accord-core/src/main/java/accord/messages/Accept.java
index 3e6aa9b..1bbc878 100644
--- a/accord-core/src/main/java/accord/messages/Accept.java
+++ b/accord-core/src/main/java/accord/messages/Accept.java
@@ -155,7 +155,7 @@
@Override
public KeyHistory keyHistory()
{
- return KeyHistory.DEPS;
+ return KeyHistory.COMMANDS;
}
@Override
@@ -181,6 +181,7 @@
public final AcceptOutcome outcome;
public final Ballot supersededBy;
+ // TODO (expected): only send back deps that weren't in those we received
public final @Nullable PartialDeps deps;
private AcceptReply(AcceptOutcome outcome)
diff --git a/accord-core/src/main/java/accord/messages/Apply.java b/accord-core/src/main/java/accord/messages/Apply.java
index cfc9f91..96d57f1 100644
--- a/accord-core/src/main/java/accord/messages/Apply.java
+++ b/accord-core/src/main/java/accord/messages/Apply.java
@@ -23,17 +23,17 @@
import accord.api.Result;
import accord.api.RoutingKey;
import accord.local.Commands;
+import accord.local.KeyHistory;
import accord.local.Node;
import accord.local.Node.Id;
import accord.local.SafeCommand;
import accord.local.SafeCommandStore;
import accord.messages.Apply.ApplyReply;
import accord.primitives.Deps;
-import accord.primitives.Keys;
+import accord.primitives.FullRoute;
import accord.primitives.PartialDeps;
import accord.primitives.PartialRoute;
import accord.primitives.PartialTxn;
-import accord.primitives.Ranges;
import accord.primitives.Route;
import accord.primitives.Seekables;
import accord.primitives.Timestamp;
@@ -49,15 +49,15 @@
public static final Factory FACTORY = Apply::new;
public static class SerializationSupport
{
- public static Apply create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Kind kind, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, PartialTxn txn, Writes writes, Result result)
+ public static Apply create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Kind kind, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, PartialTxn txn, @Nullable FullRoute<?> fullRoute, Writes writes, Result result)
{
- return new Apply(kind, txnId, scope, waitForEpoch, keys, executeAt, deps, txn, writes, result);
+ return new Apply(kind, txnId, scope, waitForEpoch, keys, executeAt, deps, txn, fullRoute, writes, result);
}
}
public interface Factory
{
- Apply create(Kind kind, Id to, Topologies participates, Topologies executes, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result);
+ Apply create(Kind kind, Id to, Topologies participates, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result);
}
public final Kind kind;
@@ -65,39 +65,40 @@
public final Seekables<?, ?> keys;
public final PartialDeps deps; // TODO (expected): this should be nullable, and only included if we did not send Commit (or if sending Maximal apply)
public final @Nullable PartialTxn txn;
+ public final @Nullable FullRoute<?> fullRoute;
public final Writes writes;
public final Result result;
public enum Kind { Minimal, Maximal }
- protected Apply(Kind kind, Id to, Topologies participates, Topologies executes, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+ protected Apply(Kind kind, Id to, Topologies participates, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
{
super(to, participates, route, txnId);
Invariants.checkState(txnId.kind() != Txn.Kind.Write || writes != null);
- Ranges slice = kind == Kind.Maximal || executes == participates ? scope.covering() : executes.computeRangesForNode(to);
// TODO (desired): it's wasteful to encode the full set of ranges owned by the recipient node;
// often it will be cheaper to include the FullRoute for Deps scope (or come up with some other safety-preserving encoding scheme)
this.kind = kind;
- this.deps = deps.slice(slice);
- this.keys = txn.keys().slice(slice);
- this.txn = kind == Kind.Maximal ? txn.slice(slice, true) : null;
+ this.deps = deps.slice(scope.covering());
+ this.keys = txn.keys().slice(scope.covering());
+ this.txn = kind == Kind.Maximal ? txn.slice(scope.covering(), true) : null;
+ this.fullRoute = kind == Kind.Maximal ? route : null;
this.executeAt = executeAt;
this.writes = writes;
this.result = result;
}
- public static void sendMaximal(Node node, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+ public static void sendMaximal(Node node, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
{
Topologies executes = executes(node, route, executeAt);
Topologies participates = participates(node, route, txnId, executeAt, executes);
- node.send(participates.nodes(), to -> applyMaximal(FACTORY, to, participates, executes, txnId, route, txn, executeAt, deps, writes, result));
+ node.send(participates.nodes(), to -> applyMaximal(FACTORY, to, participates, txnId, route, txn, executeAt, deps, writes, result));
}
- public static void sendMaximal(Node node, Id to, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
+ public static void sendMaximal(Node node, Id to, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps deps, Writes writes, Result result)
{
Topologies executes = executes(node, route, executeAt);
Topologies participates = participates(node, route, txnId, executeAt, executes);
- node.send(to, applyMaximal(FACTORY, to, participates, executes, txnId, route, txn, executeAt, deps, writes, result));
+ node.send(to, applyMaximal(FACTORY, to, participates, txnId, route, txn, executeAt, deps, writes, result));
}
public static Topologies executes(Node node, Unseekables<?> route, Timestamp executeAt)
@@ -110,17 +111,17 @@
return txnId.epoch() == executeAt.epoch() ? executes : node.topology().preciseEpochs(route, txnId.epoch(), executeAt.epoch());
}
- public static Apply applyMinimal(Factory factory, Id to, Topologies sendTo, Topologies applyTo, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
+ public static Apply applyMinimal(Factory factory, Id to, Topologies all, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
{
- return factory.create(Kind.Minimal, to, sendTo, applyTo, txnId, route, txn, executeAt, stableDeps, writes, result);
+ return factory.create(Kind.Minimal, to, all, txnId, route, txn, executeAt, stableDeps, writes, result);
}
- public static Apply applyMaximal(Factory factory, Id to, Topologies participates, Topologies executes, TxnId txnId, Route<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
+ public static Apply applyMaximal(Factory factory, Id to, Topologies participates, TxnId txnId, FullRoute<?> route, Txn txn, Timestamp executeAt, Deps stableDeps, Writes writes, Result result)
{
- return factory.create(Kind.Maximal, to, participates, executes, txnId, route, txn, executeAt, stableDeps, writes, result);
+ return factory.create(Kind.Maximal, to, participates, txnId, route, txn, executeAt, stableDeps, writes, result);
}
- protected Apply(Kind kind, TxnId txnId, PartialRoute<?> route, long waitForEpoch, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, @Nullable PartialTxn txn, Writes writes, Result result)
+ protected Apply(Kind kind, TxnId txnId, PartialRoute<?> route, long waitForEpoch, Seekables<?, ?> keys, Timestamp executeAt, PartialDeps deps, @Nullable PartialTxn txn, @Nullable FullRoute<?> fullRoute, Writes writes, Result result)
{
super(txnId, route, waitForEpoch);
this.kind = kind;
@@ -128,6 +129,7 @@
this.deps = deps;
this.keys = keys;
this.txn = txn;
+ this.fullRoute = fullRoute;
this.writes = writes;
this.result = result;
}
@@ -143,18 +145,18 @@
@Override
public ApplyReply apply(SafeCommandStore safeStore)
{
- return apply(safeStore, txn, txnId, executeAt, deps, scope, writes, result, progressKey);
+ return apply(safeStore, txn, txnId, executeAt, deps, fullRoute != null ? fullRoute : scope, writes, result, progressKey);
}
- public static ApplyReply apply(SafeCommandStore safeStore, PartialTxn txn, TxnId txnId, Timestamp executeAt, PartialDeps deps, PartialRoute<?> scope, Writes writes, Result result, RoutingKey progressKey)
+ public static ApplyReply apply(SafeCommandStore safeStore, PartialTxn txn, TxnId txnId, Timestamp executeAt, PartialDeps deps, Route<?> route, Writes writes, Result result, RoutingKey progressKey)
{
- SafeCommand safeCommand = safeStore.get(txnId, executeAt, scope);
- return apply(safeStore, safeCommand, txn, txnId, executeAt, deps, scope, writes, result, progressKey);
+ SafeCommand safeCommand = safeStore.get(txnId, executeAt, route);
+ return apply(safeStore, safeCommand, txn, txnId, executeAt, deps, route, writes, result, progressKey);
}
- public static ApplyReply apply(SafeCommandStore safeStore, SafeCommand safeCommand, PartialTxn txn, TxnId txnId, Timestamp executeAt, PartialDeps deps, PartialRoute<?> scope, Writes writes, Result result, RoutingKey progressKey)
+ public static ApplyReply apply(SafeCommandStore safeStore, SafeCommand safeCommand, PartialTxn txn, TxnId txnId, Timestamp executeAt, PartialDeps deps, Route<?> route, Writes writes, Result result, RoutingKey progressKey)
{
- switch (Commands.apply(safeStore, safeCommand, txnId, scope, progressKey, executeAt, deps, txn, writes, result))
+ switch (Commands.apply(safeStore, safeCommand, txnId, route, progressKey, executeAt, deps, txn, writes, result))
{
default:
case Insufficient:
@@ -187,8 +189,13 @@
@Override
public Seekables<?, ?> keys()
{
- if (txn == null) return Keys.EMPTY;
- return txn.keys();
+ return keys;
+ }
+
+ @Override
+ public KeyHistory keyHistory()
+ {
+ return KeyHistory.COMMANDS;
}
@Override
diff --git a/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java b/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java
index d5c4041..ec06085 100644
--- a/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java
+++ b/accord-core/src/main/java/accord/messages/ApplyThenWaitUntilApplied.java
@@ -27,16 +27,23 @@
import accord.api.Result;
import accord.api.RoutingKey;
import accord.local.CommandStore;
+import accord.local.Node;
import accord.local.PreLoadContext;
import accord.local.SafeCommandStore;
import accord.messages.Apply.ApplyReply;
+import accord.primitives.Deps;
+import accord.primitives.FullRoute;
import accord.primitives.PartialDeps;
-import accord.primitives.PartialRoute;
+import accord.primitives.PartialTxn;
+import accord.primitives.Participants;
import accord.primitives.Ranges;
import accord.primitives.Seekables;
+import accord.primitives.Txn;
import accord.primitives.TxnId;
import accord.primitives.Writes;
+import accord.topology.Topologies;
+import static accord.messages.TxnRequest.computeScope;
import static accord.utils.Invariants.illegalState;
/*
@@ -52,28 +59,40 @@
@SuppressWarnings("unused")
public static class SerializerSupport
{
- public static ApplyThenWaitUntilApplied create(TxnId txnId, PartialRoute<?> route, PartialDeps deps, Seekables<?, ?> partialTxnKeys, Writes writes, Result result, boolean notifyAgent)
+ public static ApplyThenWaitUntilApplied create(TxnId txnId, Participants<?> readScope, long executeAtEpoch, FullRoute<?> route, PartialTxn txn, PartialDeps deps, Writes writes, Result result, Seekables<?, ?> notify)
{
- return new ApplyThenWaitUntilApplied(txnId, route, deps, partialTxnKeys, writes, result, notifyAgent);
+ return new ApplyThenWaitUntilApplied(txnId, readScope, executeAtEpoch, route, txn, deps, writes, result, notify);
}
}
- public final PartialRoute<?> route;
+ public final FullRoute<?> route;
+ public final PartialTxn txn;
public final PartialDeps deps;
public final Writes writes;
- public final Result txnResult;
- public final boolean notifyAgent;
- public final Seekables<?, ?> partialTxnKeys;
+ public final Result result;
+ public final Seekables<?, ?> notify;
- ApplyThenWaitUntilApplied(TxnId txnId, PartialRoute<?> route, PartialDeps deps, Seekables<?, ?> partialTxnKeys, Writes writes, Result txnResult, boolean notifyAgent)
+ public ApplyThenWaitUntilApplied(Node.Id to, Topologies topologies, FullRoute<?> route, TxnId txnId, Txn txn, Deps deps, Participants<?> readScope, long executeAtEpoch, Writes writes, Result result, Seekables<?, ?> notify)
{
- super(txnId, partialTxnKeys.toParticipants(), txnId.epoch());
+ super(to, topologies, txnId, readScope, executeAtEpoch);
+ Ranges slice = computeScope(to, topologies, null, 0, (i,r)->r, Ranges::with);
this.route = route;
+ this.txn = txn.slice(slice, true);
+ this.deps = deps.slice(slice);
+ this.writes = writes;
+ this.result = result;
+ this.notify = notify == null ? null : notify.slice(slice);
+ }
+
+ protected ApplyThenWaitUntilApplied(TxnId txnId, Participants<?> readScope, long executeAtEpoch, FullRoute<?> route, PartialTxn txn, PartialDeps deps, Writes writes, Result result, Seekables<?, ?> notify)
+ {
+ super(txnId, readScope, executeAtEpoch);
+ this.route = route;
+ this.txn = txn;
this.deps = deps;
this.writes = writes;
- this.txnResult = txnResult;
- this.notifyAgent = notifyAgent;
- this.partialTxnKeys = partialTxnKeys;
+ this.result = result;
+ this.notify = notify;
}
@Override
@@ -86,7 +105,7 @@
public CommitOrReadNack apply(SafeCommandStore safeStore)
{
RoutingKey progressKey = TxnRequest.progressKey(node, txnId.epoch(), txnId, route);
- ApplyReply applyReply = Apply.apply(safeStore, null, txnId, txnId, deps, route, writes, txnResult, progressKey);
+ ApplyReply applyReply = Apply.apply(safeStore, txn, txnId, txnId, deps, route, writes, result, progressKey);
switch (applyReply)
{
default:
@@ -94,6 +113,7 @@
case Insufficient:
throw illegalState("ApplyThenWaitUntilApplied is always sent with a maximal `Commit` so how can `Apply` have an `Insufficient` result");
case Redundant:
+ // TODO (required): redundant is not necessarily safe for awaitsOnlyDeps commands as might need a future epoch
case Applied:
// In both cases it's fine to continue to process and return a response saying
// things were applied
@@ -115,8 +135,9 @@
@Override
protected void onAllSuccess(@Nullable Ranges unavailable, @Nullable Data data, @Nullable Throwable fail)
{
- if (notifyAgent)
- node.agent().onLocalBarrier(partialTxnKeys, txnId);
+ // TODO (expected): don't like the coupling going on here
+ if (notify != null)
+ node.agent().onLocalBarrier(notify, txnId);
super.onAllSuccess(unavailable, data, fail);
}
@@ -129,7 +150,7 @@
@Override
public String toString()
{
- return "WaitForDependenciesThenApply{" +
+ return "ApplyThenWaitUntilApplied{" +
"txnId:" + txnId +
'}';
}
diff --git a/accord-core/src/main/java/accord/messages/BeginRecovery.java b/accord-core/src/main/java/accord/messages/BeginRecovery.java
index 5979bd7..c20c6f3 100644
--- a/accord-core/src/main/java/accord/messages/BeginRecovery.java
+++ b/accord-core/src/main/java/accord/messages/BeginRecovery.java
@@ -41,11 +41,11 @@
import static accord.local.SafeCommandStore.TestDep.WITH;
import static accord.local.SafeCommandStore.TestDep.WITHOUT;
-import static accord.local.SafeCommandStore.TestTimestamp.EXECUTES_AFTER;
-import static accord.local.SafeCommandStore.TestTimestamp.STARTED_AFTER;
-import static accord.local.SafeCommandStore.TestTimestamp.STARTED_BEFORE;
-import static accord.local.Status.Accepted;
-import static accord.local.Status.Committed;
+import static accord.local.SafeCommandStore.TestStartedAt.ANY;
+import static accord.local.SafeCommandStore.TestStatus.IS_STABLE;
+import static accord.local.SafeCommandStore.TestStatus.IS_PROPOSED;
+import static accord.local.SafeCommandStore.TestStartedAt.STARTED_AFTER;
+import static accord.local.SafeCommandStore.TestStartedAt.STARTED_BEFORE;
import static accord.local.Status.Phase;
import static accord.local.Status.PreAccepted;
import static accord.local.Status.PreCommitted;
@@ -137,16 +137,16 @@
// them as a dependency (but we have to make sure we consider dependency rules, so if there's no write and only reads)
// we might still have new transactions block our execution.
Ranges ranges = safeStore.ranges().allAt(txnId);
- rejectsFastPath = hasAcceptedStartedAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
+ rejectsFastPath = hasAcceptedOrCommittedStartedAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
if (!rejectsFastPath)
- rejectsFastPath = hasCommittedExecutesAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
+ rejectsFastPath = hasStableExecutesAfterWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
// TODO (expected, testing): introduce some good unit tests for verifying these two functions in a real repair scenario
// committed txns with an earlier txnid and have our txnid as a dependency
- earlierCommittedWitness = committedStartedBeforeAndWitnessed(safeStore, txnId, ranges, partialTxn.keys());
+ earlierCommittedWitness = stableStartedBeforeAndWitnessed(safeStore, txnId, ranges, partialTxn.keys());
// accepted txns with an earlier txnid that don't have our txnid as a dependency
- earlierAcceptedNoWitness = acceptedStartedBeforeWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
+ earlierAcceptedNoWitness = acceptedOrCommittedStartedBeforeWithoutWitnessing(safeStore, txnId, ranges, partialTxn.keys());
}
Status status = command.status();
@@ -212,7 +212,7 @@
@Override
public KeyHistory keyHistory()
{
- return KeyHistory.ALL;
+ return KeyHistory.COMMANDS;
}
@Override
@@ -326,14 +326,14 @@
}
}
- private static Deps acceptedStartedBeforeWithoutWitnessing(SafeCommandStore safeStore, TxnId startedBefore, Ranges ranges, Seekables<?, ?> keys)
+ private static Deps acceptedOrCommittedStartedBeforeWithoutWitnessing(SafeCommandStore safeStore, TxnId startedBefore, Ranges ranges, Seekables<?, ?> keys)
{
try (Deps.Builder builder = Deps.builder())
{
// any transaction that started
- safeStore.mapReduce(keys, ranges, KeyHistory.ALL, startedBefore.kind().witnessedBy(), STARTED_BEFORE, startedBefore, WITHOUT, startedBefore, Accepted, PreCommitted,
- (startedBefore0, keyOrRange, txnId, executeAt, status, deps, prev) -> {
- if (executeAt.compareTo(startedBefore) > 0)
+ safeStore.mapReduceFull(keys, ranges, startedBefore, startedBefore.kind().witnessedBy(), STARTED_BEFORE, WITHOUT, IS_PROPOSED,
+ (startedBefore0, keyOrRange, txnId, executeAt, prev) -> {
+ if (executeAt.compareTo(startedBefore0) > 0)
builder.add(keyOrRange, txnId);
return builder;
}, startedBefore, builder);
@@ -341,17 +341,17 @@
}
}
- private static Deps committedStartedBeforeAndWitnessed(SafeCommandStore safeStore, TxnId startedBefore, Ranges ranges, Seekables<?, ?> keys)
+ private static Deps stableStartedBeforeAndWitnessed(SafeCommandStore safeStore, TxnId startedBefore, Ranges ranges, Seekables<?, ?> keys)
{
try (Deps.Builder builder = Deps.builder())
{
- safeStore.mapReduce(keys, ranges, KeyHistory.ALL, startedBefore.kind().witnessedBy(), STARTED_BEFORE, startedBefore, WITH, startedBefore, Committed, null,
- (p1, keyOrRange, txnId, executeAt, status, deps, prev) -> builder.add(keyOrRange, txnId), null, (Deps.AbstractBuilder<Deps>)builder);
+ safeStore.mapReduceFull(keys, ranges, startedBefore, startedBefore.kind().witnessedBy(), STARTED_BEFORE, WITH, IS_STABLE,
+ (p1, keyOrRange, txnId, executeAt, prev) -> builder.add(keyOrRange, txnId), null, (Deps.AbstractBuilder<Deps>)builder);
return builder.build();
}
}
- private static boolean hasAcceptedStartedAfterWithoutWitnessing(SafeCommandStore safeStore, TxnId startedAfter, Ranges ranges, Seekables<?, ?> keys)
+ private static boolean hasAcceptedOrCommittedStartedAfterWithoutWitnessing(SafeCommandStore safeStore, TxnId startedAfter, Ranges ranges, Seekables<?, ?> keys)
{
/*
* The idea here is to discover those transactions that were started after us and have been Accepted
@@ -362,11 +362,11 @@
*
* TODO (required): consider carefully how _adding_ ranges to a CommandStore affects this
*/
- return safeStore.mapReduce(keys, ranges, KeyHistory.ALL, startedAfter.kind().witnessedBy(), STARTED_AFTER, startedAfter, WITHOUT, startedAfter, Accepted, PreCommitted,
- (p1, keyOrRange, txnId, executeAt, status, deps, prev) -> true, null, false, i -> i);
+ return safeStore.mapReduceFull(keys, ranges, startedAfter, startedAfter.kind().witnessedBy(), STARTED_AFTER, WITHOUT, IS_PROPOSED,
+ (p1, keyOrRange, txnId, executeAt, prev) -> true, null, false);
}
- private static boolean hasCommittedExecutesAfterWithoutWitnessing(SafeCommandStore safeStore, TxnId startedAfter, Ranges ranges, Seekables<?, ?> keys)
+ private static boolean hasStableExecutesAfterWithoutWitnessing(SafeCommandStore safeStore, TxnId executesAfter, Ranges ranges, Seekables<?, ?> keys)
{
/*
* The idea here is to discover those transactions that have been decided to execute after us
@@ -375,7 +375,7 @@
* witnessed us we are safe to propose the pre-accept timestamp regardless, whereas if any transaction
* has not witnessed us we can safely invalidate it.
*/
- return safeStore.mapReduce(keys, ranges, KeyHistory.ALL, startedAfter.kind().witnessedBy(), EXECUTES_AFTER, startedAfter, WITHOUT, startedAfter, Committed, null,
- (p1, keyOrRange, txnId, executeAt, status, deps, prev) -> true, null, false, i -> i);
+ return safeStore.mapReduceFull(keys, ranges, executesAfter, executesAfter.kind().witnessedBy(), ANY, WITHOUT, IS_STABLE,
+ (p1, keyOrRange, txnId, executeAt, prev) -> true, null, false);
}
}
diff --git a/accord-core/src/main/java/accord/messages/CheckStatus.java b/accord-core/src/main/java/accord/messages/CheckStatus.java
index f888f48..3cc235d 100644
--- a/accord-core/src/main/java/accord/messages/CheckStatus.java
+++ b/accord-core/src/main/java/accord/messages/CheckStatus.java
@@ -37,14 +37,12 @@
import accord.local.SafeCommandStore;
import accord.local.SaveStatus;
import accord.local.Status;
-import accord.primitives.AbstractKeys;
import accord.primitives.Ballot;
import accord.primitives.EpochSupplier;
import accord.primitives.PartialDeps;
import accord.primitives.PartialRoute;
import accord.primitives.PartialTxn;
import accord.primitives.ProgressToken;
-import accord.primitives.Range;
import accord.primitives.Ranges;
import accord.primitives.Routables;
import accord.primitives.Route;
@@ -159,13 +157,13 @@
if (!command.has(Known.DefinitionOnly) && Route.isRoute(query) && safeStore.ranges().allAt(txnId.epoch()).contains(Route.castToRoute(query).homeKey()))
Commands.informHome(safeStore, safeCommand, Route.castToRoute(query));
- Ranges ranges = safeStore.ranges().allBetween(command.txnId().epoch(), command.executeAtIfKnownOrTxnId().epoch());
- InvalidIfNot invalidIfNotAtLeast = invalidIfNotAtLeast(safeStore);
+ InvalidIfNot invalidIfNotAtLeast = invalidIfNot(safeStore, command);
boolean isCoordinating = isCoordinating(node, command);
Durability durability = command.durability();
Route<?> route = command.route();
if (Route.isFullRoute(route))
durability = Durability.mergeAtLeast(durability, safeStore.commandStore().durableBefore().min(txnId, route));
+ Ranges ranges = safeStore.ranges().allBetween(command.txnId().epoch(), command.executeAtIfKnownOrTxnId().epoch());
switch (includeInfo)
{
@@ -204,9 +202,11 @@
else node.reply(replyTo, replyContext, ok, null);
}
- private InvalidIfNot invalidIfNotAtLeast(SafeCommandStore safeStore)
+ private InvalidIfNot invalidIfNot(SafeCommandStore safeStore, Command command)
{
- return Infer.invalidIfNotAtLeast(safeStore, txnId, query);
+ if (command.known().isDecidedToExecute())
+ return NotKnownToBeInvalid;
+ return Infer.invalidIfNot(safeStore, txnId, query);
}
public interface CheckStatusReply extends Reply
@@ -329,33 +329,7 @@
if (keysOrRanges.isEmpty())
return new FoundKnownMap();
- switch (keysOrRanges.domain())
- {
- default: throw new AssertionError("Unhandled domain type: " + keysOrRanges.domain());
- case Range:
- {
- Ranges ranges = (Ranges)keysOrRanges;
- Builder builder = new Builder(ranges.get(0).endInclusive(), 2 * ranges.size());
- for (Range range : ranges)
- {
- builder.append(range.start(), known, FoundKnown::atLeast);
- builder.append(range.end(), null, FoundKnown::atLeast);
- }
- return builder.build();
- }
- case Key:
- {
- AbstractKeys<RoutingKey> keys = (AbstractKeys<RoutingKey>) keysOrRanges;
- Builder builder = new Builder(keys.get(0).asRange().endInclusive(), 2 * keys.size());
- for (RoutingKey key : keys)
- {
- Range range = key.asRange();
- builder.append(range.start(), known, FoundKnown::atLeast);
- builder.append(range.end(), null, FoundKnown::atLeast);
- }
- return builder.build();
- }
- }
+ return create(keysOrRanges, known, Builder::new);
}
public static FoundKnownMap merge(FoundKnownMap a, FoundKnownMap b)
@@ -430,6 +404,11 @@
return foldl((known, prev) -> known.isTruncated(), false, i -> i);
}
+ public boolean hasInvalidated()
+ {
+ return foldl((known, prev) -> known.isInvalidated(), false, i -> i);
+ }
+
public Known knownFor(Routables<?> routables)
{
return validForAll.atLeast(foldlWithDefault(routables, FoundKnownMap::reduceKnownFor, FoundKnown.Nothing, null, i -> false));
diff --git a/accord-core/src/main/java/accord/messages/Commit.java b/accord-core/src/main/java/accord/messages/Commit.java
index 05fd67f..c365c9a 100644
--- a/accord-core/src/main/java/accord/messages/Commit.java
+++ b/accord-core/src/main/java/accord/messages/Commit.java
@@ -21,6 +21,7 @@
import javax.annotation.Nullable;
import accord.local.Commands;
+import accord.local.KeyHistory;
import accord.local.Node;
import accord.local.Node.Id;
import accord.local.PreLoadContext;
@@ -32,7 +33,6 @@
import accord.primitives.Ballot;
import accord.primitives.Deps;
import accord.primitives.FullRoute;
-import accord.primitives.Keys;
import accord.primitives.PartialDeps;
import accord.primitives.PartialRoute;
import accord.primitives.PartialTxn;
@@ -58,21 +58,22 @@
import static accord.messages.Commit.WithTxn.HasNewlyOwnedTxnRanges;
import static accord.messages.Commit.WithTxn.HasTxn;
import static accord.messages.Commit.WithTxn.NoTxn;
-import static accord.utils.Invariants.checkArgument;
public class Commit extends TxnRequest<CommitOrReadNack>
{
public static class SerializerSupport
{
- public static Commit create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Kind kind, Ballot ballot, Timestamp executeAt, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData readData)
+ public static Commit create(TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Kind kind, Ballot ballot, Timestamp executeAt, Seekables<?, ?> keys, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData readData)
{
- return new Commit(kind, txnId, scope, waitForEpoch, ballot, executeAt, partialTxn, partialDeps, fullRoute, readData);
+ return new Commit(kind, txnId, scope, waitForEpoch, ballot, executeAt, keys, partialTxn, partialDeps, fullRoute, readData);
}
}
public final Kind kind;
public final Ballot ballot;
public final Timestamp executeAt;
+ public final Seekables<?, ?> keys;
+ // TODO (expected): share keys with partialTxn and partialDeps - in memory and on wire
public final @Nullable PartialTxn partialTxn;
public final @Nullable PartialDeps partialDeps;
public final @Nullable FullRoute<?> route;
@@ -139,18 +140,20 @@
this.kind = kind;
this.executeAt = executeAt;
+ this.keys = txn.keys().slice(scope.covering());
this.partialTxn = partialTxn;
this.partialDeps = deps.slice(scope.covering());
this.route = sendRoute;
this.readData = toExecuteFactory == null ? null : toExecuteFactory.apply(partialTxn != null ? partialTxn : txn, scope, partialDeps);
}
- protected Commit(Kind kind, TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Ballot ballot, Timestamp executeAt, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData readData)
+ protected Commit(Kind kind, TxnId txnId, PartialRoute<?> scope, long waitForEpoch, Ballot ballot, Timestamp executeAt, Seekables<?, ?> keys, @Nullable PartialTxn partialTxn, PartialDeps partialDeps, @Nullable FullRoute<?> fullRoute, @Nullable ReadData readData)
{
super(txnId, scope, waitForEpoch);
this.kind = kind;
this.ballot = ballot;
this.executeAt = executeAt;
+ this.keys = keys;
this.partialTxn = partialTxn;
this.partialDeps = partialDeps;
this.route = fullRoute;
@@ -212,25 +215,6 @@
}
}
- public static void stableMaximalAndBlockOnDeps(Node node, Topologies topologies, TxnId txnId, Txn txn, FullRoute<?> route, Timestamp executeAt, Deps deps, Callback<ReadReply> callback)
- {
- checkArgument(topologies.size() == 1);
- Topology topology = topologies.get(0);
- for (Node.Id to : topology.nodes())
- {
- // To simplify making sure the agent is notified once and is notified before the barrier coordination
- // returns a result; we never notify the agent on the coordinator as part of WaitForDependenciesThenApply execution
- // Also always send a maximal commit since we don't block on deps often and that saves having to have an Insufficient code path
- // going back for the Apply in `ApplyThenWaitUntilApplied
- boolean notifyAgent = !to.equals(node.id());
- Commit commit = new Commit(
- CommitWithTxn, to, topology, topologies, txnId,
- txn, route, Ballot.ZERO, executeAt, deps,
- (maybePartialTransaction, partialRoute, partialDeps) -> new ApplyThenWaitUntilApplied(txnId, partialRoute, partialDeps, maybePartialTransaction.keys().slice(partialDeps.covering), txn.execute(txnId, txnId, null), txn.result(txnId, txnId, null), notifyAgent));
- node.send(to, commit, callback);
- }
- }
-
public static void stableMaximal(Node node, Node.Id to, Txn txn, TxnId txnId, Timestamp executeAt, FullRoute<?> route, Deps deps)
{
// the replica may be missing the original commit, or the additional commit, so send everything
@@ -248,7 +232,13 @@
@Override
public Seekables<?, ?> keys()
{
- return partialTxn != null ? partialTxn.keys() : Keys.EMPTY;
+ return keys;
+ }
+
+ @Override
+ public KeyHistory keyHistory()
+ {
+ return KeyHistory.COMMANDS;
}
@Override
diff --git a/accord-core/src/main/java/accord/messages/GetDeps.java b/accord-core/src/main/java/accord/messages/GetDeps.java
index 16feb38..1e77c24 100644
--- a/accord-core/src/main/java/accord/messages/GetDeps.java
+++ b/accord-core/src/main/java/accord/messages/GetDeps.java
@@ -119,7 +119,7 @@
@Override
public KeyHistory keyHistory()
{
- return KeyHistory.DEPS;
+ return KeyHistory.COMMANDS;
}
public static class GetDepsOk implements Reply
diff --git a/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java b/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java
index 030d34b..3943d5d 100644
--- a/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java
+++ b/accord-core/src/main/java/accord/messages/GetEphemeralReadDeps.java
@@ -99,7 +99,7 @@
@Override
public String toString()
{
- return "GetDeps{" +
+ return "GetEphemeralReadDeps{" +
"txnId:" + txnId +
", keys:" + keys +
'}';
@@ -120,7 +120,7 @@
@Override
public KeyHistory keyHistory()
{
- return KeyHistory.DEPS;
+ return KeyHistory.COMMANDS;
}
public static class GetEphemeralReadDepsOk implements Reply
diff --git a/accord-core/src/main/java/accord/messages/GetMaxConflict.java b/accord-core/src/main/java/accord/messages/GetMaxConflict.java
new file mode 100644
index 0000000..ec59b4d
--- /dev/null
+++ b/accord-core/src/main/java/accord/messages/GetMaxConflict.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package accord.messages;
+
+import javax.annotation.Nonnull;
+
+import accord.local.KeyHistory;
+import accord.local.Node;
+import accord.local.SafeCommandStore;
+import accord.primitives.FullRoute;
+import accord.primitives.PartialRoute;
+import accord.primitives.Ranges;
+import accord.primitives.Seekables;
+import accord.primitives.Timestamp;
+import accord.primitives.TxnId;
+import accord.topology.Topologies;
+import accord.utils.Invariants;
+
+public class GetMaxConflict extends TxnRequest.WithUnsynced<GetMaxConflict.GetMaxConflictOk>
+{
+ public static final class SerializationSupport
+ {
+ public static GetMaxConflict create(PartialRoute<?> scope, long waitForEpoch, long minEpoch, Seekables<?, ?> keys, long executionEpoch)
+ {
+ return new GetMaxConflict(scope, waitForEpoch, minEpoch, keys, executionEpoch);
+ }
+ }
+
+ public final Seekables<?, ?> keys;
+ public final long executionEpoch;
+
+ public GetMaxConflict(Node.Id to, Topologies topologies, FullRoute<?> route, Seekables<?, ?> keys, long executionEpoch)
+ {
+ super(to, topologies, executionEpoch, route);
+ this.keys = keys.slice(scope.covering());
+ this.executionEpoch = executionEpoch;
+ }
+
+ protected GetMaxConflict(PartialRoute<?> scope, long waitForEpoch, long minEpoch, Seekables<?, ?> keys, long executionEpoch)
+ {
+ super(TxnId.NONE, scope, waitForEpoch, minEpoch, true);
+ this.keys = keys;
+ this.executionEpoch = executionEpoch;
+ }
+
+ @Override
+ public void process()
+ {
+ node.mapReduceConsumeLocal(this, minUnsyncedEpoch, executionEpoch, this);
+ }
+
+ @Override
+ public GetMaxConflictOk apply(SafeCommandStore safeStore)
+ {
+ Ranges ranges = safeStore.ranges().allBetween(minUnsyncedEpoch, executionEpoch);
+ Timestamp maxConflict = safeStore.commandStore().maxConflict(keys.slice(ranges));
+ return new GetMaxConflictOk(maxConflict, Math.max(safeStore.time().epoch(), node.epoch()));
+ }
+
+ @Override
+ public GetMaxConflictOk reduce(GetMaxConflictOk reply1, GetMaxConflictOk reply2)
+ {
+ return new GetMaxConflictOk(Timestamp.max(reply1.maxConflict, reply2.maxConflict), Math.max(reply1.latestEpoch, reply2.latestEpoch));
+ }
+
+ @Override
+ public void accept(GetMaxConflictOk result, Throwable failure)
+ {
+ node.reply(replyTo, replyContext, result, failure);
+ }
+
+ @Override
+ public MessageType type()
+ {
+ return MessageType.GET_EPHEMERAL_READ_DEPS_REQ;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "GetMaxConflict{" +
+ ", keys:" + keys +
+ '}';
+ }
+
+ @Override
+ public TxnId primaryTxnId()
+ {
+ return null;
+ }
+
+ @Override
+ public Seekables<?, ?> keys()
+ {
+ return keys;
+ }
+
+ @Override
+ public KeyHistory keyHistory()
+ {
+ return KeyHistory.NONE;
+ }
+
+ public static class GetMaxConflictOk implements Reply
+ {
+ public final Timestamp maxConflict;
+ public final long latestEpoch;
+
+ public GetMaxConflictOk(@Nonnull Timestamp maxConflict, long latestEpoch)
+ {
+ this.maxConflict = Invariants.nonNull(maxConflict);
+ this.latestEpoch = latestEpoch;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "GetMaxConflictOk(" + maxConflict + ',' + latestEpoch + '}';
+ }
+
+ @Override
+ public MessageType type()
+ {
+ return MessageType.GET_EPHEMERAL_READ_DEPS_RSP;
+ }
+ }
+}
diff --git a/accord-core/src/main/java/accord/messages/PreAccept.java b/accord-core/src/main/java/accord/messages/PreAccept.java
index 0736b4b..ad964ff 100644
--- a/accord-core/src/main/java/accord/messages/PreAccept.java
+++ b/accord-core/src/main/java/accord/messages/PreAccept.java
@@ -27,14 +27,11 @@
import org.slf4j.LoggerFactory;
import accord.local.Node.Id;
-import accord.impl.DepsBuilder;
import accord.local.*;
import accord.messages.TxnRequest.WithUnsynced;
import accord.topology.Shard;
import accord.topology.Topologies;
-import static accord.local.SafeCommandStore.TestDep.ANY_DEPS;
-import static accord.local.SafeCommandStore.TestTimestamp.STARTED_BEFORE;
import static accord.primitives.Txn.Kind.ExclusiveSyncPoint;
public class PreAccept extends WithUnsynced<PreAccept.PreAcceptReply> implements EpochSupplier
@@ -86,7 +83,7 @@
@Override
public KeyHistory keyHistory()
{
- return KeyHistory.DEPS;
+ return KeyHistory.COMMANDS;
}
@Override
@@ -132,7 +129,7 @@
// all transactions with lower txnId as expired.
Ranges ranges = safeStore.ranges().allBetween(minUnsyncedEpoch, txnId);
return new PreAcceptOk(txnId, command.executeAt(),
- calculatePartialDeps(safeStore, txnId, partialTxn.keys(), EpochSupplier.constant(minUnsyncedEpoch), txnId, ranges));
+ calculatePartialDeps(safeStore, txnId, command.partialTxn().keys(), EpochSupplier.constant(minUnsyncedEpoch), txnId, ranges));
case Truncated:
case RejectedBallot:
@@ -248,21 +245,22 @@
static PartialDeps calculatePartialDeps(SafeCommandStore safeStore, TxnId txnId, Seekables<?, ?> keys, EpochSupplier minEpoch, Timestamp executeAt, Ranges ranges)
{
// TODO (expected): do not build covering ranges; no longer especially valuable given use of FullRoute
- DepsBuilder builder = new DepsBuilder(txnId);
- // could use MAY_EXECUTE_BEFORE to prune those we know execute later.
// NOTE: ExclusiveSyncPoint *relies* on STARTED_BEFORE to ensure it reports a dependency on *every* earlier TxnId that may execute after it.
// This is necessary for reporting to a bootstrapping replica which TxnId it must not prune from dependencies
// i.e. the source replica reports to the target replica those TxnId that STARTED_BEFORE and EXECUTES_AFTER.
- safeStore.mapReduce(keys, ranges, KeyHistory.DEPS, txnId.kind().witnesses(), STARTED_BEFORE, executeAt, ANY_DEPS, null, null, null,
- (p1, keyOrRange, testTxnId, testExecuteAt, status, deps, in) -> {
- builder.add(testTxnId, keyOrRange, status, testExecuteAt, deps.get());
- return in;
- }, null, builder);
-
- try (Deps.AbstractBuilder<PartialDeps> redundantBuilder = new PartialDeps.Builder(ranges))
+ try (Deps.AbstractBuilder<PartialDeps> builder = new PartialDeps.Builder(ranges);
+ Deps.AbstractBuilder<PartialDeps> redundantBuilder = new PartialDeps.Builder(ranges))
{
+ safeStore.mapReduceActive(keys, ranges, executeAt, txnId.kind().witnesses(),
+ (p1, keyOrRange, testTxnId, testExecuteAt, in) -> {
+ if (p1 == null || !testTxnId.equals(p1))
+ in.add(keyOrRange, testTxnId);
+ return in;
+ }, executeAt.equals(txnId) ? null : txnId, builder);
+
+ // TODO (required): make sure any sync point is in the past
PartialDeps redundant = safeStore.commandStore().redundantBefore().collectDeps(keys, redundantBuilder, minEpoch, executeAt).build();
- return builder.buildPartialDeps(safeStore, ranges).with(redundant);
+ return builder.build().with(redundant);
}
}
diff --git a/accord-core/src/main/java/accord/messages/Propagate.java b/accord-core/src/main/java/accord/messages/Propagate.java
index 99d4333..4a20fa1 100644
--- a/accord-core/src/main/java/accord/messages/Propagate.java
+++ b/accord-core/src/main/java/accord/messages/Propagate.java
@@ -22,6 +22,7 @@
import accord.coordinate.Infer;
import accord.local.Command;
import accord.local.Commands;
+import accord.local.KeyHistory;
import accord.local.Node;
import accord.local.SafeCommand;
import accord.local.SafeCommandStore;
@@ -220,6 +221,7 @@
@Override
public Seekables<?, ?> keys()
{
+ // TODO (required): these may be insufficient; must make update of CommandsForKey async
if (partialTxn != null)
return partialTxn.keys();
@@ -230,6 +232,12 @@
}
@Override
+ public KeyHistory keyHistory()
+ {
+ return KeyHistory.COMMANDS;
+ }
+
+ @Override
public void preProcess(Node on, Node.Id from, ReplyContext replyContext)
{
throw new UnsupportedOperationException();
@@ -267,6 +275,7 @@
if (isShardTruncated)
{
// TODO (required): do not markShardStale for reads; in general optimise handling of case where we cannot recover a known no-op transaction
+ // TODO (required): permit staleness to be gated by some configuration state
achieved = applyOrUpgradeTruncated(safeStore, safeCommand, command, executeAtIfKnown);
if (achieved == null)
return null;
@@ -408,7 +417,7 @@
if (ranges.isEmpty())
{
// TODO (expected): we might prefer to adopt Redundant status, and permit ourselves to later accept the result of the execution and/or definition
- Commands.setTruncatedApply(safeStore, safeCommand, route);
+ Commands.setTruncatedApply(safeStore, safeCommand, executeAtIfKnown, route);
return null;
}
@@ -417,7 +426,7 @@
{
// we only coordinate this transaction, so being unable to retrieve its state does not imply any staleness
// TODO (now): double check this doesn't stop us coordinating the transaction (it shouldn't, as doesn't imply durability)
- Commands.setTruncatedApply(safeStore, safeCommand, route);
+ Commands.setTruncatedApply(safeStore, safeCommand, executeAtIfKnown, route);
return null;
}
@@ -443,7 +452,7 @@
return required;
// TODO (expected): we might prefer to adopt Redundant status, and permit ourselves to later accept the result of the execution and/or definition
- Commands.setTruncatedApply(safeStore, safeCommand, route);
+ Commands.setTruncatedApply(safeStore, safeCommand, executeAtIfKnown, route);
return null;
}
diff --git a/accord-core/src/main/java/accord/messages/ReadData.java b/accord-core/src/main/java/accord/messages/ReadData.java
index d1e2529..18600a6 100644
--- a/accord-core/src/main/java/accord/messages/ReadData.java
+++ b/accord-core/src/main/java/accord/messages/ReadData.java
@@ -267,9 +267,7 @@
static Ranges unavailable(SafeCommandStore safeStore, Command command)
{
- Timestamp executeAt = command.executeAt();
- if (command.txnId().kind().awaitsOnlyDeps())
- executeAt = Timestamp.nonNullOrMax(executeAt, command.asCommitted().waitingOn.executeAtLeast());
+ Timestamp executeAt = command.executesAtLeast();
// TODO (required): for awaitsOnlyDeps commands, if we cannot infer an actual executeAtLeast we should confirm no situation where txnId is not an adequately conservative value for unavailable/unsafeToRead
return safeStore.ranges().unsafeToReadAt(executeAt);
}
diff --git a/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java b/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java
index fb11002..75c2126 100644
--- a/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java
+++ b/accord-core/src/main/java/accord/messages/ReadEphemeralTxnData.java
@@ -26,7 +26,6 @@
import accord.local.SafeCommand;
import accord.local.SafeCommandStore;
import accord.local.SaveStatus;
-import accord.local.Status;
import accord.primitives.Deps;
import accord.primitives.FullRoute;
import accord.primitives.PartialDeps;
@@ -105,23 +104,38 @@
@Override
void read(SafeCommandStore safeStore, Command command)
{
- Command.WaitingOn waitingOn = command.asCommitted().waitingOn;
- Timestamp executeAtLeast = waitingOn.executeAtLeast();
- if (executeAtLeast != null && executeAtLeast.epoch() > executeAtEpoch)
+ long retryInLaterEpoch = retryInLaterEpoch(executeAtEpoch, safeStore, command);
+ if (retryInLaterEpoch > 0)
{
- long executeAtLeastEpoch = executeAtLeast.epoch();
- Ranges removed = safeStore.ranges().removed(executeAtEpoch, executeAtLeastEpoch);
- if (removed.intersects(command.partialTxn().keys()))
- {
- Ranges unavailable = unavailable(safeStore, command).with(removed).intersecting(partialTxn.keys());
- cancel(safeStore);
- node.reply(replyTo, replyContext, new ReadOkWithFutureEpoch(unavailable, null, executeAtLeastEpoch), null);
- return;
- }
+ // TODO (expected): wait for all stores' results and report only the ranges that execute later to be retried
+ cancel(safeStore);
+ node.reply(replyTo, replyContext, new ReadOkWithFutureEpoch(null, null, retryInLaterEpoch), null);
}
super.read(safeStore, command);
}
+ static long retryInLaterEpoch(long executeAtEpoch, SafeCommandStore safeStore, Command command)
+ {
+ TxnId txnId = command.txnId();
+ if (!txnId.kind().awaitsOnlyDeps())
+ return 0;
+
+ // TODO (required): should we disambiguate between cases where a truncated command has been executed locally
+ // versus made redundant by e.g. bootstrap, staleness etc? that *should* be handled by checking
+ // unavailable ranges
+ Timestamp executesAtLeast = command.executesAtLeast();
+ if (executesAtLeast != null && executesAtLeast.epoch() > executeAtEpoch)
+ {
+ long executeAtLeastEpoch = executesAtLeast.epoch();
+ Ranges removed = safeStore.ranges().removed(executeAtEpoch, executeAtLeastEpoch);
+ if (removed.intersects(command.route()))
+ return executeAtLeastEpoch;
+ }
+
+ return 0;
+ }
+
+
@Override
protected ReadOk constructReadOk(Ranges unavailable, Data data)
{
diff --git a/accord-core/src/main/java/accord/messages/SetGloballyDurable.java b/accord-core/src/main/java/accord/messages/SetGloballyDurable.java
index 4f76bcd..3fa3932 100644
--- a/accord-core/src/main/java/accord/messages/SetGloballyDurable.java
+++ b/accord-core/src/main/java/accord/messages/SetGloballyDurable.java
@@ -21,17 +21,22 @@
import accord.local.PreLoadContext;
import accord.local.DurableBefore;
import accord.local.SafeCommandStore;
+import accord.primitives.TxnId;
import static accord.messages.SimpleReply.Ok;
+// TODO (required): if we have epochs 1 and 2, and a syncpoint that executes on epoch 2, where the transactions have not
+// finished executing on epoch 1, it may not be safe to mark durable on replicas on epoch 1. This is a very unlikely
+// race condition, but must consider our behaviour - it may be simpler to wait for a sync point to execute on all epochs
+// that haven't been closed off.
public class SetGloballyDurable extends AbstractEpochRequest<SimpleReply>
implements Request, PreLoadContext
{
public final DurableBefore durableBefore;
- public SetGloballyDurable(DurableBefore durableBefore)
+ public SetGloballyDurable(TxnId txnId, DurableBefore durableBefore)
{
- super(null);
+ super(txnId);
this.durableBefore = durableBefore;
}
diff --git a/accord-core/src/main/java/accord/messages/TxnRequest.java b/accord-core/src/main/java/accord/messages/TxnRequest.java
index 4cd3942..f6ea2c4 100644
--- a/accord-core/src/main/java/accord/messages/TxnRequest.java
+++ b/accord-core/src/main/java/accord/messages/TxnRequest.java
@@ -51,20 +51,35 @@
this(to, topologies, txnId, route, latestRelevantEpochIndex(to, topologies, route));
}
- private WithUnsynced(Id to, Topologies topologies, TxnId txnId, FullRoute<?> route, int startIndex)
+ protected WithUnsynced(Id to, Topologies topologies, TxnId txnId, FullRoute<?> route, int startIndex)
+ {
+ this(to, topologies, txnId, txnId.epoch(), route, startIndex);
+ }
+
+ public WithUnsynced(Id to, Topologies topologies, long epoch, FullRoute<?> route)
+ {
+ this(to, topologies, TxnId.NONE, epoch, route, latestRelevantEpochIndex(to, topologies, route), true);
+ }
+
+ protected WithUnsynced(Id to, Topologies topologies, TxnId txnId, long epoch, FullRoute<?> route, int startIndex)
+ {
+ this(to, topologies, txnId, epoch, route, startIndex, false);
+ }
+
+ protected WithUnsynced(Id to, Topologies topologies, TxnId txnId, long epoch, FullRoute<?> route, int startIndex, boolean doNotComputeProgressKey)
{
super(to, topologies, route, txnId, startIndex);
this.minUnsyncedEpoch = topologies.oldestEpoch();
- this.doNotComputeProgressKey = doNotComputeProgressKey(topologies, startIndex, txnId, waitForEpoch());
+ this.doNotComputeProgressKey = doNotComputeProgressKey || doNotComputeProgressKey(topologies, startIndex, epoch, waitForEpoch());
- Ranges ranges = topologies.forEpoch(txnId.epoch()).rangesForNode(to);
+ Ranges ranges = topologies.forEpoch(epoch).rangesForNode(to);
if (doNotComputeProgressKey)
{
- Invariants.checkState(!route.intersects(ranges)); // confirm dest is not a replica on txnId.epoch
+ Invariants.checkState(txnId.equals(TxnId.NONE) || !route.intersects(ranges)); // confirm dest is not a replica on txnId.epoch
}
else if (Invariants.isParanoid())
{
- long progressEpoch = Math.min(waitForEpoch(), txnId.epoch());
+ long progressEpoch = Math.min(waitForEpoch(), epoch);
Ranges computesRangesOn = topologies.forEpoch(progressEpoch).rangesForNode(to);
if (computesRangesOn == null)
Invariants.checkState(!route.intersects(ranges));
@@ -156,8 +171,11 @@
return progressKey(node, waitForEpoch, txnId, scope);
}
- public static RoutingKey progressKey(Node node, long waitForEpoch, TxnId txnId, PartialRoute<?> scope)
+ public static RoutingKey progressKey(Node node, long waitForEpoch, TxnId txnId, Route<?> scope)
{
+ if (txnId == null)
+ return null;
+
// if waitForEpoch < txnId.epoch, then this replica's ownership is unchanged
long progressEpoch = min(waitForEpoch, txnId.epoch());
return node.trySelectProgressKey(progressEpoch, scope, scope.homeKey());
@@ -265,7 +283,7 @@
return scope;
}
- private static boolean doNotComputeProgressKey(Topologies topologies, int startIndex, TxnId txnId, long waitForEpoch)
+ private static boolean doNotComputeProgressKey(Topologies topologies, int startIndex, long epoch, long waitForEpoch)
{
// to understand this calculation we must bear in mind the following:
// - startIndex is the "latest relevant" which means we skip over recent epochs where we are not owners at all,
@@ -280,7 +298,7 @@
// (as it might be done so with stale ring information)
// TODO (low priority, clarity): this would be better defined as "hasProgressKey"
- return waitForEpoch < txnId.epoch() && startIndex > 0
- && topologies.get(startIndex).epoch() < txnId.epoch();
+ return waitForEpoch < epoch && startIndex > 0
+ && topologies.get(startIndex).epoch() < epoch;
}
}
diff --git a/accord-core/src/main/java/accord/messages/WaitUntilApplied.java b/accord-core/src/main/java/accord/messages/WaitUntilApplied.java
index 2cb398f..74cd578 100644
--- a/accord-core/src/main/java/accord/messages/WaitUntilApplied.java
+++ b/accord-core/src/main/java/accord/messages/WaitUntilApplied.java
@@ -27,11 +27,11 @@
import accord.local.SafeCommandStore;
import accord.primitives.Participants;
import accord.primitives.Ranges;
-import accord.primitives.Timestamp;
import accord.primitives.TxnId;
import accord.topology.Topologies;
import static accord.local.SaveStatus.Applied;
+import static accord.local.SaveStatus.TruncatedApply;
/**
* Wait until the transaction has been applied locally
@@ -48,8 +48,8 @@
}
}
- private static final ExecuteOn EXECUTE_ON = new ExecuteOn(Applied, Applied);
- private long futureEpoch;
+ private static final ExecuteOn EXECUTE_ON = new ExecuteOn(Applied, TruncatedApply);
+ private long retryInLaterEpoch;
public WaitUntilApplied(Node.Id to, Topologies topologies, TxnId txnId, Participants<?> readScope, long executeAtEpoch)
{
@@ -76,17 +76,16 @@
@Override
void read(SafeCommandStore safeStore, Command command)
{
- Command.WaitingOn waitingOn = command.asCommitted().waitingOn;
- Timestamp executeAtLeast = waitingOn.executeAtLeast();
- if (executeAtLeast != null)
- this.futureEpoch = Math.max(futureEpoch, executeAtLeast.epoch());
+ long retryInLaterEpoch = ReadEphemeralTxnData.retryInLaterEpoch(executeAtEpoch, safeStore, command);
+ if (retryInLaterEpoch > this.retryInLaterEpoch)
+ this.retryInLaterEpoch = retryInLaterEpoch;
onOneSuccess(safeStore.commandStore(), unavailable(safeStore, command));
}
@Override
protected ReadOk constructReadOk(Ranges unavailable, Data data)
{
- return new ReadOkWithFutureEpoch(unavailable, data, futureEpoch);
+ return new ReadOkWithFutureEpoch(unavailable, data, retryInLaterEpoch);
}
@Override
diff --git a/accord-core/src/main/java/accord/primitives/AbstractKeys.java b/accord-core/src/main/java/accord/primitives/AbstractKeys.java
index 2c699f1..c120195 100644
--- a/accord-core/src/main/java/accord/primitives/AbstractKeys.java
+++ b/accord-core/src/main/java/accord/primitives/AbstractKeys.java
@@ -31,6 +31,7 @@
import accord.utils.ArrayBuffers.ObjectBuffers;
import accord.utils.IndexedFold;
import accord.utils.IndexedFoldToLong;
+import accord.utils.IndexedTriFold;
import accord.utils.Invariants;
import accord.utils.SortedArrays;
import net.nicoulaj.compilecommand.annotations.Inline;
@@ -234,6 +235,18 @@
}
@Inline
+ public final long foldl(AbstractKeys<K> intersect, IndexedFoldToLong<K> fold, long param, long accumulator, long terminalValue)
+ {
+ return Routables.foldl(this, intersect, fold, param, accumulator, terminalValue);
+ }
+
+ @Inline
+ public final <P1, P2, V> V foldl(AbstractKeys<K> intersect, IndexedTriFold<P1, P2, K, V> fold, P1 p1, P2 p2, V accumulator)
+ {
+ return Routables.foldl(this, intersect, fold, p1, p2, accumulator, i -> false);
+ }
+
+ @Inline
public final void forEach(Ranges rs, Consumer<? super K> forEach)
{
Routables.foldl(this, rs, (k, consumer, i) -> { consumer.accept(k); return consumer; }, forEach);
diff --git a/accord-core/src/main/java/accord/primitives/KeyDeps.java b/accord-core/src/main/java/accord/primitives/KeyDeps.java
index a53e80d..47cd345 100644
--- a/accord-core/src/main/java/accord/primitives/KeyDeps.java
+++ b/accord-core/src/main/java/accord/primitives/KeyDeps.java
@@ -515,11 +515,11 @@
return new SortedArrayList<>(txnIds);
}
- public List<TxnId> txnIds(Key key)
+ public SortedRelationList<TxnId> txnIds(Key key)
{
int keyIndex = keys.indexOf(key);
if (keyIndex < 0)
- return Collections.emptyList();
+ return SortedRelationList.EMPTY;
return txnIdsForKeyIndex(keyIndex);
}
diff --git a/accord-core/src/main/java/accord/primitives/PartialDeps.java b/accord-core/src/main/java/accord/primitives/PartialDeps.java
index e13ce62..9367328 100644
--- a/accord-core/src/main/java/accord/primitives/PartialDeps.java
+++ b/accord-core/src/main/java/accord/primitives/PartialDeps.java
@@ -45,6 +45,8 @@
// TODO (expected): we no longer need this if everyone has a FullRoute
// could also retain a simple bitset over the original FullRoute
+ // TODO (required) remove this and related concepts, as can cause problems with topology changes for a single store
+ // where the store has some ranges that we participate in, and some we do not; we will not correctly construct covering in some cases
public final Ranges covering;
public PartialDeps(Ranges covering, KeyDeps keyDeps, RangeDeps rangeDeps)
diff --git a/accord-core/src/main/java/accord/primitives/RangeDeps.java b/accord-core/src/main/java/accord/primitives/RangeDeps.java
index 56ac2dd..00e047c 100644
--- a/accord-core/src/main/java/accord/primitives/RangeDeps.java
+++ b/accord-core/src/main/java/accord/primitives/RangeDeps.java
@@ -45,7 +45,10 @@
* <p>The relationship between Range and TxnId is maintained via {@code int[]} utilising {@link RelationMultiMap}
* functionality.
*
- * TODO (expected): de-overlap ranges per txnId if possible cheaply, or else reduce use of partial ranges where possible
+ * TODO (required): de-overlap ranges per txnId if possible cheaply, or else reduce use of partial ranges where possible
+ * TODO (required): permit building out-of-order
+ * TODO (required): currently permitting duplicates
+ * TODO (required): randomised testing of all iteration methods (just found a double increment bug)
*/
public class RangeDeps implements Iterable<Map.Entry<Range, TxnId>>
{
@@ -517,7 +520,7 @@
int start = txnIdx == 0 ? txnIds.length : txnIdsToRanges[txnIdx - 1];
int end = txnIdsToRanges[txnIdx];
for (int i = start; i < end ; ++i)
- accumulate = fold.apply(p1, ranges[txnIdsToRanges[i++]], accumulate);
+ accumulate = fold.apply(p1, ranges[txnIdsToRanges[i]], accumulate);
return accumulate;
}
diff --git a/accord-core/src/main/java/accord/primitives/Ranges.java b/accord-core/src/main/java/accord/primitives/Ranges.java
index 5cc07c8..ac293dc 100644
--- a/accord-core/src/main/java/accord/primitives/Ranges.java
+++ b/accord-core/src/main/java/accord/primitives/Ranges.java
@@ -19,6 +19,9 @@
package accord.primitives;
import accord.api.RoutingKey;
+import accord.utils.IndexedFoldToLong;
+import accord.utils.IndexedTriFold;
+import net.nicoulaj.compilecommand.annotations.Inline;
import javax.annotation.Nonnull;
import java.util.ArrayList;
@@ -196,4 +199,16 @@
return ImmutableMap.of(Boolean.TRUE, Ranges.ofSortedAndDeoverlapped(trues.toArray(new Range[0])),
Boolean.FALSE, Ranges.ofSortedAndDeoverlapped(falses.toArray(new Range[0])));
}
+
+ @Inline
+ public final long foldl(Ranges intersect, IndexedFoldToLong<Range> fold, long param, long accumulator, long terminalValue)
+ {
+ return Routables.foldl(this, intersect, fold, param, accumulator, terminalValue);
+ }
+
+ @Inline
+ public final <P1, P2, V> V foldl(Ranges intersect, IndexedTriFold<P1, P2, Range, V> fold, P1 p1, P2 p2, V accumulator)
+ {
+ return Routables.foldl(this, intersect, fold, p1, p2, accumulator, i -> false);
+ }
}
diff --git a/accord-core/src/main/java/accord/primitives/SyncPoint.java b/accord-core/src/main/java/accord/primitives/SyncPoint.java
index fedc64c..d22f375 100644
--- a/accord-core/src/main/java/accord/primitives/SyncPoint.java
+++ b/accord-core/src/main/java/accord/primitives/SyncPoint.java
@@ -33,9 +33,9 @@
{
public static class SerializationSupport
{
- public static SyncPoint construct(TxnId syncId, Deps waitFor, Seekables<?,?> keysOrRanges, RoutingKey homeKey, boolean finishedAsync)
+ public static SyncPoint construct(TxnId syncId, Deps waitFor, Seekables<?,?> keysOrRanges, RoutingKey homeKey)
{
- return new SyncPoint(syncId, waitFor, keysOrRanges, homeKey, finishedAsync);
+ return new SyncPoint(syncId, waitFor, keysOrRanges, homeKey);
}
}
@@ -43,25 +43,22 @@
public final Deps waitFor;
public final S keysOrRanges;
public final RoutingKey homeKey;
- public final boolean finishedAsync;
- public SyncPoint(TxnId syncId, Deps waitFor, S keysOrRanges, FullRoute route, boolean finishedAsync)
+ public SyncPoint(TxnId syncId, Deps waitFor, S keysOrRanges, FullRoute<?> route)
{
Invariants.checkArgument(keysOrRanges.toRoute(route.homeKey()).equals(route), "Expected homeKey %s from route %s to be in ranges %s", route.homeKey(), route, keysOrRanges);
this.syncId = syncId;
this.waitFor = waitFor;
this.keysOrRanges = keysOrRanges;
this.homeKey = route.homeKey();
- this.finishedAsync = finishedAsync;
}
- private SyncPoint(TxnId syncId, Deps waitFor, S keysOrRanges, RoutingKey homeKey, boolean finishedAsync)
+ private SyncPoint(TxnId syncId, Deps waitFor, S keysOrRanges, RoutingKey homeKey)
{
this.syncId = syncId;
this.waitFor = waitFor;
this.keysOrRanges = keysOrRanges;
this.homeKey = homeKey;
- this.finishedAsync = finishedAsync;
}
public FullRoute route()
diff --git a/accord-core/src/main/java/accord/primitives/Timestamp.java b/accord-core/src/main/java/accord/primitives/Timestamp.java
index 05dfe86..41c8326 100644
--- a/accord-core/src/main/java/accord/primitives/Timestamp.java
+++ b/accord-core/src/main/java/accord/primitives/Timestamp.java
@@ -37,11 +37,12 @@
* which we may also want to retain when merging in other contexts (such as in Deps).
*/
private static final int MERGE_FLAGS = 0x8000;
+ // TODO (required): is this the correct set of identity bits?
private static final long IDENTITY_LSB = 0xFFFFFFFFFFFF001EL;
- private static final int IDENTITY_FLAGS = 0x001E;
+ public static final int IDENTITY_FLAGS = 0x001E;
public static final long MAX_EPOCH = (1L << 48) - 1;
private static final long HLC_INCR = 1L << 16;
- private static final long MAX_FLAGS = HLC_INCR - 1;
+ static final long MAX_FLAGS = HLC_INCR - 1;
public static Timestamp fromBits(long msb, long lsb, Id node)
{
diff --git a/accord-core/src/main/java/accord/primitives/Txn.java b/accord-core/src/main/java/accord/primitives/Txn.java
index 3675f9f..c88f471 100644
--- a/accord-core/src/main/java/accord/primitives/Txn.java
+++ b/accord-core/src/main/java/accord/primitives/Txn.java
@@ -47,9 +47,13 @@
public interface Txn
{
+ /**
+ * NOTE: we keep Read/Write adjacent to make it easier to check for non-standard flags in serialization
+ */
enum Kind
{
Read,
+ Write,
/**
* A non-durable read that cannot be recovered and provides only per-key linearizability guarantees.
@@ -58,16 +62,6 @@
*/
EphemeralRead,
- Write,
-
- /**
- * A pseudo-transaction used only to find an executeAt that is higher than any transaction that could have
- * reached consensus before it started.
- *
- * Invisible to other transactions.
- */
- NoOp,
-
/**
* A pseudo-transaction whose deps represent the complete set of transactions that may execute before it,
* without interfering with their execution.
@@ -189,7 +183,18 @@
public boolean isGloballyVisible()
{
- return this != EphemeralRead && this != LocalOnly;
+ switch (this)
+ {
+ default: throw new AssertionError("Unhandled Kind: " + this);
+ case EphemeralRead:
+ case LocalOnly:
+ return false;
+ case Write:
+ case Read:
+ case ExclusiveSyncPoint:
+ case SyncPoint:
+ return true;
+ }
}
public boolean isSyncPoint()
@@ -217,7 +222,6 @@
default: throw new AssertionError();
case EphemeralRead:
case Read:
- case NoOp:
return Ws;
case Write:
return RorWs;
@@ -240,14 +244,13 @@
return AnyGloballyVisible;
case SyncPoint:
case ExclusiveSyncPoint:
- case NoOp:
return SyncPoints;
}
}
public char shortName()
{
- return toString().charAt(0);
+ return shortName;
}
}
diff --git a/accord-core/src/main/java/accord/primitives/TxnId.java b/accord-core/src/main/java/accord/primitives/TxnId.java
index 145ece4..5647ae9 100644
--- a/accord-core/src/main/java/accord/primitives/TxnId.java
+++ b/accord-core/src/main/java/accord/primitives/TxnId.java
@@ -166,7 +166,7 @@
return new TxnId(epochMsb(epoch), 0, Id.NONE);
}
- private static final Pattern PARSE = Pattern.compile("\\[(?<epoch>[0-9]+),(?<hlc>[0-9]+),(?<flags>[0-9]+),(?<node>[0-9]+)]");
+ private static final Pattern PARSE = Pattern.compile("\\[(?<epoch>[0-9]+),(?<hlc>[0-9]+),(?<flags>[0-9]+)\\([KREWNSXL]\\),(?<node>[0-9]+)]");
public static TxnId parse(String txnIdString)
{
Matcher m = PARSE.matcher(txnIdString);
diff --git a/accord-core/src/main/java/accord/utils/ArrayBuffers.java b/accord-core/src/main/java/accord/utils/ArrayBuffers.java
index a554559..ba633cc 100644
--- a/accord-core/src/main/java/accord/utils/ArrayBuffers.java
+++ b/accord-core/src/main/java/accord/utils/ArrayBuffers.java
@@ -372,7 +372,7 @@
boolean discardInternal(B buffer, int bufferSize, int usedSize, boolean force)
{
- if (bufferSize > maxSize)
+ if (bufferSize == 0 || bufferSize > maxSize)
return true;
if (bufferSize == usedSize && !force)
diff --git a/accord-core/src/main/java/accord/utils/Invariants.java b/accord-core/src/main/java/accord/utils/Invariants.java
index 43603fe..85464bf 100644
--- a/accord-core/src/main/java/accord/utils/Invariants.java
+++ b/accord-core/src/main/java/accord/utils/Invariants.java
@@ -27,6 +27,7 @@
public class Invariants
{
+ // TODO (now): configure by system parameter and turn off by default
private static final boolean PARANOID = true;
private static final boolean DEBUG = true;
@@ -41,7 +42,7 @@
public static IllegalStateException illegalState(String msg)
{
- throw new IllegalStateException(msg);
+ throw new IllegalStateException(msg);
}
public static IllegalStateException illegalState()
@@ -54,7 +55,6 @@
throw new IllegalArgumentException(msg);
}
-
private static void illegalArgument()
{
illegalArgument(null);
@@ -315,7 +315,7 @@
}
catch (ClassCastException e)
{
- throw new IllegalArgumentException(format("Unable to cast %s to %s", o, klass.getName()));
+ throw illegalArgument(format("Unable to cast %s to %s", o, klass.getName()));
}
}
diff --git a/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java b/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java
index 7a935fe..1f2c316 100644
--- a/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java
+++ b/accord-core/src/main/java/accord/utils/ReducingIntervalMap.java
@@ -225,7 +225,8 @@
}
start = second.start();
- if (first.hasCurrent() && first.start().compareTo(start) < 0) builder.append(first.start(), start, first.value());
+ if (first.hasCurrent() && first.start().compareTo(start) < 0 && first.value() != null)
+ builder.append(first.start(), start, builder.slice(first.start(), start, first.value()));
Invariants.checkState(start.compareTo(second.start()) <= 0);
}
@@ -304,6 +305,7 @@
}
// loop over any range covered by both
+ // TODO (expected): optimise merging of very different sized maps (i.e. for inserts)
while (left.hasCurrent() && right.hasCurrent())
{
int cmp = left.end().compareTo(right.end());
@@ -460,9 +462,23 @@
if (sameAsTailKey || sameAsTailValue)
{
if (sameAsTailValue)
+ {
values.set(tailIdx, value == null ? null : tailValue);
+ }
+ else if (tailValue != null)
+ {
+ values.set(tailIdx, reduce.apply(tailValue, value));
+ }
+ else if (tailIdx >= 1 && value.equals(values.get(tailIdx - 1)))
+ {
+ // just remove the null value and start
+ values.remove(tailIdx);
+ starts.remove(tailIdx);
+ }
else
- values.set(tailIdx, tailValue == null ? value : reduce.apply(tailValue, value));
+ {
+ values.set(tailIdx, value);
+ }
}
else
{
diff --git a/accord-core/src/main/java/accord/utils/ReducingRangeMap.java b/accord-core/src/main/java/accord/utils/ReducingRangeMap.java
index 904586b..13beb8c 100644
--- a/accord-core/src/main/java/accord/utils/ReducingRangeMap.java
+++ b/accord-core/src/main/java/accord/utils/ReducingRangeMap.java
@@ -20,6 +20,7 @@
import accord.api.RoutingKey;
import accord.primitives.*;
+import java.util.Arrays;
import java.util.function.BiFunction;
import java.util.function.Predicate;
@@ -297,6 +298,22 @@
return accumulator;
}
+ private int find(RoutableKey key)
+ {
+ int idx = Arrays.binarySearch(starts, key);
+ if (idx < 0) idx = -2 - idx;
+ else if (inclusiveEnds) --idx;
+ return idx;
+ }
+
+ public V get(RoutableKey key)
+ {
+ int idx = find(key);
+ if (idx < 0 || idx >= values.length)
+ return null;
+ return values[idx];
+ }
+
public static <V> ReducingRangeMap<V> create(Ranges ranges, V value)
{
if (value == null)
@@ -305,12 +322,86 @@
if (ranges.isEmpty())
return new ReducingRangeMap<>();
- ReducingRangeMap.Builder<V> builder = new ReducingRangeMap.Builder<>(ranges.get(0).endInclusive(), ranges.size() * 2);
- for (Range range : ranges)
+ return create(ranges, value, ReducingRangeMap.Builder::new);
+ }
+
+ public static <V, M extends ReducingRangeMap<V>> M create(Unseekables<?> keysOrRanges, V value, BuilderFactory<RoutingKey, V, M> builder)
+ {
+ switch (keysOrRanges.domain())
{
- builder.append(range.start(), value, (a, b) -> { throw new IllegalStateException(); });
- builder.append(range.end(), null, (a, b) -> a); // if we are equal to prev end, take the prev value not zero
+ default: throw new AssertionError("Unhandled domain: " + keysOrRanges.domain());
+ case Range: return create((Ranges) keysOrRanges, value, builder);
+ case Key: return create((AbstractUnseekableKeys) keysOrRanges, value, builder);
}
+ }
+
+ public static <V, M extends ReducingRangeMap<V>> M create(Seekables<?, ?> keysOrRanges, V value, BuilderFactory<RoutingKey, V, M> builder)
+ {
+ switch (keysOrRanges.domain())
+ {
+ default: throw new AssertionError("Unhandled domain: " + keysOrRanges.domain());
+ case Range: return create((Ranges) keysOrRanges, value, builder);
+ case Key: return create((Keys) keysOrRanges, value, builder);
+ }
+ }
+
+ public static <V, M extends ReducingRangeMap<V>> M create(Ranges ranges, V value, BuilderFactory<RoutingKey, V, M> factory)
+ {
+ if (value == null)
+ throw new IllegalArgumentException("value is null");
+
+ AbstractBoundariesBuilder<RoutingKey, V, M> builder = factory.create(ranges.get(0).endInclusive(), ranges.size() * 2);
+ for (Range cur : ranges)
+ {
+ builder.append(cur.start(), value, (a, b) -> { throw new IllegalStateException(); });
+ builder.append(cur.end(), null, (a, b) -> { throw new IllegalStateException(); });
+ }
+
+ return builder.build();
+ }
+
+ public static <V, M extends ReducingRangeMap<V>> M create(AbstractUnseekableKeys keys, V value, BuilderFactory<RoutingKey, V, M> factory)
+ {
+ if (value == null)
+ throw new IllegalArgumentException("value is null");
+
+ AbstractBoundariesBuilder<RoutingKey, V, M> builder = factory.create(keys.get(0).asRange().endInclusive(), keys.size() * 2);
+ for (int i = 0 ; i < keys.size() ; ++i)
+ {
+ Range range = keys.get(i).asRange();
+ builder.append(range.start(), value, (a, b) -> { throw new IllegalStateException(); });
+ builder.append(range.end(), null, (a, b) -> { throw new IllegalStateException(); });
+ }
+
+ return builder.build();
+ }
+
+ public static <V, M extends ReducingRangeMap<V>> M create(Keys keys, V value, BuilderFactory<RoutingKey, V, M> factory)
+ {
+ if (value == null)
+ throw new IllegalArgumentException("value is null");
+
+ RoutingKey prev = keys.get(0).toUnseekable();
+ AbstractBoundariesBuilder<RoutingKey, V, M> builder;
+ {
+ Range range = prev.asRange();
+ builder = factory.create(prev.asRange().endInclusive(), keys.size() * 2);
+ builder.append(range.start(), value, (a, b) -> { throw new IllegalStateException(); });
+ builder.append(range.end(), null, (a, b) -> { throw new IllegalStateException(); });
+ }
+
+ for (int i = 1 ; i < keys.size() ; ++i)
+ {
+ RoutingKey unseekable = keys.get(i).toUnseekable();
+ if (unseekable.equals(prev))
+ continue;
+
+ Range range = unseekable.asRange();
+ builder.append(range.start(), value, (a, b) -> { throw new IllegalStateException(); });
+ builder.append(range.end(), null, (a, b) -> { throw new IllegalStateException(); });
+ prev = unseekable;
+ }
+
return builder.build();
}
diff --git a/accord-core/src/test/java/accord/Utils.java b/accord-core/src/test/java/accord/Utils.java
index 18f1526..78b1298 100644
--- a/accord-core/src/test/java/accord/Utils.java
+++ b/accord-core/src/test/java/accord/Utils.java
@@ -32,8 +32,7 @@
import accord.api.Scheduler;
import accord.config.LocalConfig;
import accord.config.MutableLocalConfig;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
import accord.impl.InMemoryCommandStores;
import accord.impl.IntKey;
import accord.impl.SimpleProgressLog;
@@ -48,7 +47,6 @@
import accord.local.Node;
import accord.local.NodeTimeService;
import accord.local.ShardDistributor;
-import accord.messages.Apply;
import accord.messages.LocalRequest;
import accord.primitives.Keys;
import accord.primitives.Range;
@@ -176,7 +174,7 @@
scheduler,
SizeOfIntersectionSorter.SUPPLIER,
SimpleProgressLog::new,
- InMemoryCommandStores.Synchronized::new, ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+ InMemoryCommandStores.Synchronized::new, new CoordinationAdapter.DefaultFactory(),
localConfig);
awaitUninterruptibly(node.unsafeStart());
return node;
diff --git a/accord-core/src/test/java/accord/burn/BurnTest.java b/accord-core/src/test/java/accord/burn/BurnTest.java
index af3f50b..bc8c226 100644
--- a/accord-core/src/test/java/accord/burn/BurnTest.java
+++ b/accord-core/src/test/java/accord/burn/BurnTest.java
@@ -108,12 +108,12 @@
* Min hash value for the test domain, this value must be respected by the hash function
* @see {@link BurnTest#hash(int)}
*/
- private static final int HASH_RANGE_START = 0;
+ public static final int HASH_RANGE_START = 0;
/**
* Max hash value for the test domain, this value must be respected by the hash function
* @see {@link BurnTest#hash(int)}
*/
- private static final int HASH_RANGE_END = 1 << 16;
+ public static final int HASH_RANGE_END = 1 << 16;
private static final Range[] EMPTY_RANGES = new Range[0];
static List<Packet> generate(RandomSource random, MessageListener listener, Function<? super CommandStore, AsyncExecutor> executor, List<Id> clients, List<Id> nodes, int[] keys, int operations)
diff --git a/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java b/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java
index c8ea753..250f84e 100644
--- a/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java
+++ b/accord-core/src/test/java/accord/coordinate/CoordinateTransactionTest.java
@@ -73,6 +73,7 @@
import static accord.impl.IntKey.range;
import static accord.local.Status.Applied;
import static accord.primitives.Routable.Domain.Key;
+import static accord.primitives.Routable.Domain.Range;
import static accord.primitives.Txn.Kind.Read;
import static accord.primitives.Txn.Kind.Write;
import static accord.utils.Invariants.checkState;
@@ -121,7 +122,7 @@
Node node = cluster.get(1);
assertNotNull(node);
- TxnId txnId = node.nextTxnId(Write, Key);
+ TxnId txnId = node.nextTxnId(Write, Range);
Ranges keys = ranges(range(1, 2));
Txn txn = writeTxn(keys);
FullRangeRoute route = keys.toRoute(keys.get(0).someIntersectingRoutingKey(null));
@@ -219,7 +220,7 @@
assertEquals(previousBarrierTimestamp, globalBarrierTimestamp);
// Sync over nothing should work
- SyncPoint<Ranges> syncPoint = getUninterruptibly(CoordinateSyncPoint.inclusive(node, ranges(range(99, 100)), false));
+ SyncPoint<Ranges> syncPoint = getUninterruptibly(CoordinateSyncPoint.inclusiveAndAwaitQuorum(node, ranges(range(99, 100))));
assertEquals(node.epoch(), syncPoint.syncId.epoch());
// Keys and so on for the upcoming transaction pair
@@ -247,13 +248,13 @@
Commands.preaccept(store, store.get(txnId, homeKey), txnId, txnId.epoch(), txn.slice(store.ranges().allAt(txnId.epoch()), true), route, null))));
- AsyncResult<SyncPoint<Ranges>> syncInclusiveSyncFuture = CoordinateSyncPoint.inclusive(node, ranges, false);
+ AsyncResult<SyncPoint<Ranges>> syncInclusiveSyncFuture = CoordinateSyncPoint.inclusiveAndAwaitQuorum(node, ranges);
// Shouldn't complete because it is blocked waiting for the dependency just created to apply
sleep(500);
assertFalse(syncInclusiveSyncFuture.isDone());
// Async sync should return a result immediately since we are going to wait on the sync point transaction that was created by the sync point
- AsyncResult<SyncPoint<Ranges>> asyncInclusiveSyncFuture = CoordinateSyncPoint.inclusive(node, ranges, true);
+ AsyncResult<SyncPoint<Ranges>> asyncInclusiveSyncFuture = CoordinateSyncPoint.inclusive(node, ranges);
SyncPoint<Ranges> localSyncPoint = getUninterruptibly(asyncInclusiveSyncFuture);
Semaphore localSyncOccurred = new Semaphore(0);
node.commandStores().ifLocal(PreLoadContext.contextFor(localSyncPoint.syncId), homeKey, epoch, epoch, safeStore ->
diff --git a/accord-core/src/test/java/accord/impl/CommandsForKeyGroupUpdaterTest.java b/accord-core/src/test/java/accord/impl/CommandsForKeyGroupUpdaterTest.java
deleted file mode 100644
index 164c282..0000000
--- a/accord-core/src/test/java/accord/impl/CommandsForKeyGroupUpdaterTest.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.impl;
-
-import accord.impl.CommandsForKeyGroupUpdater.Immutable;
-import accord.local.Command;
-import accord.local.SaveStatus;
-import accord.primitives.Timestamp;
-import accord.primitives.TxnId;
-import com.google.common.collect.ImmutableSortedSet;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-import java.util.List;
-
-public class CommandsForKeyGroupUpdaterTest
-{
- private static class Sentinel { private static final Sentinel INSTANCE = new Sentinel(); }
-
- private static final CommandTimeseries.CommandLoader<Sentinel> LOADER = new CommandTimeseries.CommandLoader<Sentinel>()
- {
- @Override public Sentinel saveForCFK(Command command) { return Sentinel.INSTANCE; }
- @Override public TxnId txnId(Sentinel data) { throw new UnsupportedOperationException(); }
- @Override public Timestamp executeAt(Sentinel data) { throw new UnsupportedOperationException(); }
- @Override public SaveStatus saveStatus(Sentinel data) { throw new UnsupportedOperationException(); }
- @Override public List<TxnId> depsIds(Sentinel data) { throw new UnsupportedOperationException(); }
- };
-
- private static final IntKey.Raw KEY = IntKey.key(1);
-
- private static TxnId txnId(int i)
- {
- return TxnId.fromValues(0, i, 0, 1);
- }
-
- private static Timestamp ts(int i)
- {
- return Timestamp.fromValues(0, i, 0, 1);
- }
-
- private static CommandTimeseries<Sentinel> timeseries()
- {
- CommandTimeseries.Builder<Sentinel> builder = new CommandTimeseries.Builder<>(KEY, LOADER);
- return builder.build();
- }
-
- private static CommandTimeseries<Sentinel> timeseries(Timestamp... timestamps)
- {
- CommandTimeseries.Builder<Sentinel> builder = new CommandTimeseries.Builder<>(KEY, LOADER);
- for (Timestamp timestamp : timestamps)
- builder.add(timestamp, Sentinel.INSTANCE);
- return builder.build();
- }
-
- private static CommandTimeseries<Sentinel> timeseries(int... timestamps)
- {
- return timeseries(timestamps(timestamps));
- }
-
- private static CommandsForKey cfk(CommandTimeseries<Sentinel> commands)
- {
- return new CommandsForKey(KEY, commands);
- }
-
- private static CommandsForKey cfk(Timestamp[] byId)
- {
- return cfk(timeseries(byId));
- }
-
- private static CommandsForKey cfk()
- {
- return new CommandsForKey(KEY, timeseries());
- }
-
- private static Timestamp[] timestamps(int... values)
- {
- Timestamp[] timestamps = new Timestamp[values.length];
- for (int i=0; i<values.length; i++)
- timestamps[i] = ts(values[i]);
- return timestamps;
- }
-
- private static void assertTimeseries(CommandTimeseries<?> timeseries, Timestamp... expected)
- {
- Assertions.assertEquals(ImmutableSortedSet.copyOf(expected), timeseries.commands.keySet());
- }
-
- private static void assertTimeseries(CommandTimeseries<?> timeseries, int... expected)
- {
- assertTimeseries(timeseries, timestamps(expected));
- }
-
- @Test
- void commonTest()
- {
- CommandsForKeyGroupUpdater.Mutable<Sentinel> updater = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-
- updater.common().commands().add(txnId(1), null);
-
- Assertions.assertEquals(cfk(timeseries(1)), updater.applyToDeps(cfk()));
- Assertions.assertEquals(cfk(timeseries(1)), updater.applyToAll(cfk()));
- }
-
- @Test
- void specificPrecedence()
- {
- CommandsForKeyGroupUpdater.Mutable<Sentinel> updater = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-
- updater.common().commands().add(txnId(2), null);
- updater.common().commands().add(txnId(1), null);
- updater.all().commands().remove(txnId(1));
-
- Assertions.assertEquals(cfk(timeseries(1, 2)), updater.applyToDeps(cfk()));
- Assertions.assertEquals(cfk(timeseries(2)), updater.applyToAll(cfk()));
- }
-
- /**
- * A later specifc update should override a previous common update (making it specific to the opposite cfk)
- */
- @Test
- void mergeOverridingSpecific()
- {
- CommandsForKeyGroupUpdater.Mutable<Sentinel> original = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-
- original.common().commands().add(txnId(1), null);
- original.common().commands().add(txnId(2), null);
-
- CommandsForKeyGroupUpdater.Mutable<Sentinel> update = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
- update.deps().commands().remove(txnId(1));
- update.deps().commands().add(txnId(3), null);
-
- CommandsForKeyGroupUpdater.Mutable<Sentinel> expected = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
-
- expected.common().commands().add(txnId(2), null);
- expected.deps().commands().remove(txnId(1));
- expected.deps().commands().add(txnId(3), null);
- expected.all().commands().add(txnId(1), null);
-
-// Immutable<Sentinel> merged = original.toImmutable().mergeNewer(update.toImmutable());
- Immutable<Sentinel> merged = Immutable.merge(original.toImmutable(), update.toImmutable(), Immutable.getFactory());
- Assertions.assertEquals(expected.toImmutable(), merged);
- }
-
- @Test
- void mergeOverridingCommon()
- {
- CommandsForKeyGroupUpdater.Mutable<Sentinel> original = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
- original.deps().commands().add(txnId(1), null);
- original.deps().commands().add(txnId(2), null);
- original.all().commands().add(txnId(2), null);
-
- CommandsForKeyGroupUpdater.Mutable<Sentinel> update = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
- update.common().commands().remove(txnId(1));
-
- CommandsForKeyGroupUpdater.Mutable<Sentinel> expected = new CommandsForKeyGroupUpdater.Mutable<>(LOADER);
- expected.deps().commands().add(txnId(2), null);
- expected.all().commands().add(txnId(2), null);
- expected.common().commands().remove(txnId(1));
-
- Immutable<Sentinel> merged = Immutable.merge(original.toImmutable(), update.toImmutable(), Immutable.getFactory());
- Assertions.assertEquals(expected.toImmutable(), merged);
- }
-}
-
diff --git a/accord-core/src/test/java/accord/impl/PrefixedIntHashKey.java b/accord-core/src/test/java/accord/impl/PrefixedIntHashKey.java
index 2ee13d4..7e75acc 100644
--- a/accord-core/src/test/java/accord/impl/PrefixedIntHashKey.java
+++ b/accord-core/src/test/java/accord/impl/PrefixedIntHashKey.java
@@ -33,6 +33,7 @@
import static accord.utils.Utils.toArray;
+// TODO (now): burn test is producing prefix keys over full integer range rather than 16bits
public class PrefixedIntHashKey implements RoutableKey
{
public static final int MIN_KEY = Integer.MIN_VALUE + 1;
@@ -111,11 +112,6 @@
public static final class Key extends PrefixedIntHashKey implements accord.api.Key
{
- private Key(int prefix, int key)
- {
- super(prefix, key, false);
- }
-
private Key(int prefix, int key, int hash)
{
super(prefix, key, hash);
@@ -124,9 +120,9 @@
public static abstract class PrefixedIntRoutingKey extends PrefixedIntHashKey implements RoutingKey
{
- private PrefixedIntRoutingKey(int prefix, int key, boolean isHash)
+ private PrefixedIntRoutingKey(int prefix, int hash)
{
- super(prefix, key, isHash);
+ super(prefix, Integer.MIN_VALUE, hash);
}
@Override
@@ -142,7 +138,7 @@
private Sentinel(int prefix, boolean isMin)
{
- super(prefix, isMin ? Integer.MIN_VALUE : Integer.MAX_VALUE, true);
+ super(prefix, isMin ? Integer.MIN_VALUE : Integer.MAX_VALUE);
this.isMin = isMin;
}
@@ -157,7 +153,7 @@
{
private Hash(int prefix, int hash)
{
- super(prefix, hash, true);
+ super(prefix, hash);
}
@Override
@@ -212,35 +208,14 @@
public final int hash;
private final boolean isHash;
- private PrefixedIntHashKey(int prefix, int key, boolean isHash)
- {
- this.prefix = prefix;
- this.isHash = isHash;
- if (isHash)
- {
- this.key = Integer.MIN_VALUE;
- this.hash = key;
- }
- else
- {
- this.key = key;
- this.hash = hash(key);
- }
- }
-
private PrefixedIntHashKey(int prefix, int key, int hash)
{
this.prefix = prefix;
- this.isHash = false; // this constructor is only used by Key
+ this.isHash = key == Integer.MIN_VALUE; // this constructor is only used by Key
this.key = key;
this.hash = hash;
}
- public static Key key(int prefix, int k)
- {
- return new Key(prefix, k);
- }
-
public static Key key(int prefix, int k, int hash)
{
return new Key(prefix, k, hash);
@@ -305,7 +280,7 @@
return Objects.hash(prefix, key, hash);
}
- static int hash(int key)
+ public static int hash(int key)
{
return CRCUtils.crc32LittleEnding(key);
}
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 1c61226..b949c5a 100644
--- a/accord-core/src/test/java/accord/impl/basic/Cluster.java
+++ b/accord-core/src/test/java/accord/impl/basic/Cluster.java
@@ -48,8 +48,7 @@
import accord.burn.random.FrequentLargeRange;
import accord.config.LocalConfig;
import accord.config.MutableLocalConfig;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
import accord.impl.CoordinateDurabilityScheduling;
import accord.impl.MessageListener;
import accord.impl.PrefixedIntHashKey;
@@ -62,7 +61,6 @@
import accord.local.Node;
import accord.local.NodeTimeService;
import accord.local.ShardDistributor;
-import accord.messages.Apply;
import accord.messages.LocalRequest;
import accord.messages.Message;
import accord.messages.MessageType;
@@ -299,7 +297,7 @@
() -> new ListStore(id), new ShardDistributor.EvenSplit<>(8, ignore -> new PrefixedIntHashKey.Splitter()),
nodeExecutor.agent(),
randomSupplier.get(), sinks, SizeOfIntersectionSorter.SUPPLIER,
- SimpleProgressLog::new, DelayedCommandStores.factory(sinks.pending, isLoadedCheck), ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+ SimpleProgressLog::new, DelayedCommandStores.factory(sinks.pending, isLoadedCheck), new CoordinationAdapter.DefaultFactory(),
localConfig);
CoordinateDurabilityScheduling durability = new CoordinateDurabilityScheduling(node);
// TODO (desired): randomise
diff --git a/accord-core/src/test/java/accord/impl/list/ListAgent.java b/accord-core/src/test/java/accord/impl/list/ListAgent.java
index 7c97a2e..092fb74 100644
--- a/accord-core/src/test/java/accord/impl/list/ListAgent.java
+++ b/accord-core/src/test/java/accord/impl/list/ListAgent.java
@@ -108,4 +108,10 @@
{
return new Txn.InMemory(kind, keysOrRanges, new ListRead(identity(), false, Keys.EMPTY, Keys.EMPTY), new ListQuery(NONE, Integer.MIN_VALUE), null);
}
+
+ public boolean collectMaxApplied()
+ {
+ // TODO (expected): randomise this to exercise both code paths
+ return false;
+ }
}
diff --git a/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java b/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java
index 7917524..4e8b3e9 100644
--- a/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java
+++ b/accord-core/src/test/java/accord/impl/list/ListFetchCoordinator.java
@@ -25,13 +25,17 @@
import accord.api.Data;
import accord.api.DataStore;
import accord.impl.AbstractFetchCoordinator;
+import accord.impl.InMemoryCommandStore;
import accord.local.CommandStore;
import accord.local.Node;
import accord.local.PreLoadContext;
+import accord.primitives.PartialDeps;
import accord.primitives.PartialTxn;
import accord.primitives.Ranges;
import accord.primitives.SyncPoint;
+import accord.primitives.Timestamp;
import accord.primitives.Txn;
+import accord.primitives.TxnId;
import accord.utils.Timestamped;
import accord.utils.async.AsyncResult;
@@ -66,4 +70,51 @@
else fail(from, received, fail);
}).beginAsResult());
}
+
+ @Override
+ protected FetchRequest newFetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn)
+ {
+ if (((ListAgent)node.agent()).collectMaxApplied())
+ return new CollectMaxAppliedFetchRequest(sourceEpoch, syncId, ranges, partialDeps, partialTxn);
+
+ return super.newFetchRequest(sourceEpoch, syncId, ranges, partialDeps, partialTxn);
+ }
+
+ class CollectMaxAppliedFetchRequest extends FetchRequest
+ {
+ private transient Timestamp maxApplied;
+
+ public CollectMaxAppliedFetchRequest(long sourceEpoch, TxnId syncId, Ranges ranges, PartialDeps partialDeps, PartialTxn partialTxn)
+ {
+ super(sourceEpoch, syncId, ranges, partialDeps, partialTxn);
+ }
+
+ @Override
+ protected void readComplete(CommandStore commandStore, Data result, Ranges unavailable)
+ {
+ Ranges slice = commandStore.unsafeRangesForEpoch().allAt(txnId).subtract(unavailable);
+ ((InMemoryCommandStore)commandStore).maxAppliedFor((Ranges)readScope, slice).begin((newMaxApplied, failure) -> {
+ if (failure != null)
+ {
+ commandStore.agent().onUncaughtException(failure);
+ }
+ else
+ {
+ synchronized (this)
+ {
+ if (maxApplied == null) maxApplied = newMaxApplied;
+ else maxApplied = Timestamp.max(maxApplied, newMaxApplied);
+ super.readComplete(commandStore, result, unavailable);
+ }
+ }
+ });
+ }
+
+ @Override
+ protected Timestamp maxApplied()
+ {
+ return maxApplied;
+ }
+ }
+
}
diff --git a/accord-core/src/test/java/accord/impl/list/ListRequest.java b/accord-core/src/test/java/accord/impl/list/ListRequest.java
index 7d796da..560cf39 100644
--- a/accord-core/src/test/java/accord/impl/list/ListRequest.java
+++ b/accord-core/src/test/java/accord/impl/list/ListRequest.java
@@ -53,6 +53,7 @@
import static accord.local.Status.Applied;
import static accord.local.Status.Phase.Cleanup;
+import static accord.local.Status.PreAccepted;
import static accord.local.Status.PreApplied;
import static accord.local.Status.PreCommitted;
import static accord.utils.Invariants.illegalState;
@@ -110,9 +111,9 @@
if (failure != null) callback.accept(null, failure);
else if (merged.maxKnowledgeSaveStatus.is(Status.Invalidated)) callback.accept(Outcome.Invalidated, null);
else if (merged.maxKnowledgeSaveStatus.is(Status.Truncated)) callback.accept(Outcome.Truncated, null);
- else if (!merged.maxKnowledgeSaveStatus.hasBeen(PreCommitted) && merged.maxSaveStatus.phase == Cleanup) callback.accept(Outcome.Truncated, null);
- else if (count == nodes().size()) callback.accept(Outcome.Lost, null);
else if (merged.maxKnowledgeSaveStatus.hasBeen(Applied)) callback.accept(new Outcome(Outcome.Kind.Applied, (ListResult) ((CheckStatus.CheckStatusOkFull) merged).result), null);
+ else if (!merged.maxKnowledgeSaveStatus.hasBeen(PreCommitted) && merged.maxSaveStatus.phase == Cleanup) callback.accept(Outcome.Truncated, null);
+ else if (!merged.maxSaveStatus.hasBeen(PreAccepted) && count >= (1 + nodes().size())/2) callback.accept(Outcome.Lost, null);
else callback.accept(Outcome.Other, null);
}
@@ -155,15 +156,14 @@
else if (fail instanceof CoordinationFailed)
{
RoutingKey homeKey = ((CoordinationFailed) fail).homeKey();
- TxnId txnId = ((CoordinationFailed) fail).txnId();
if (fail instanceof Invalidated)
{
- node.reply(client, replyContext, ListResult.invalidated(client, ((Packet)replyContext).requestId, txnId), null);
+ node.reply(client, replyContext, ListResult.invalidated(client, ((Packet)replyContext).requestId, id), null);
return;
}
- node.reply(client, replyContext, ListResult.heartBeat(client, ((Packet)replyContext).requestId, txnId), null);
- ((Cluster) node.scheduler()).onDone(() -> checkOnResult(homeKey, txnId, 0, null));
+ node.reply(client, replyContext, ListResult.heartBeat(client, ((Packet)replyContext).requestId, id), null);
+ ((Cluster) node.scheduler()).onDone(() -> checkOnResult(homeKey, id, 0, null));
}
else if (fail instanceof SimulatedFault)
{
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 e9afc8c..d669fd8 100644
--- a/accord-core/src/test/java/accord/impl/list/ListStore.java
+++ b/accord-core/src/test/java/accord/impl/list/ListStore.java
@@ -49,14 +49,18 @@
import accord.primitives.RoutableKey;
import accord.primitives.SyncPoint;
import accord.primitives.Timestamp;
+import accord.primitives.TxnId;
import accord.topology.Topologies;
import accord.topology.Topology;
import accord.utils.Timestamped;
import accord.utils.async.AsyncChain;
+import accord.utils.async.AsyncChains;
import accord.utils.async.AsyncResults;
import org.agrona.collections.Int2ObjectHashMap;
import org.agrona.collections.LongArrayList;
+import static accord.utils.Invariants.illegalState;
+
public class ListStore implements DataStore
{
private static class ChangeAt
@@ -188,12 +192,24 @@
private void checkAccess(Timestamp executeAt, Range range)
{
+ if (executeAt instanceof TxnId)
+ {
+ switch (((TxnId) executeAt).kind())
+ {
+ case EphemeralRead:
+ case ExclusiveSyncPoint:
+ // TODO (required): introduce some mechanism for EphemeralReads/ExclusiveSyncPoints to abort if local store has been expunged (and check it here)
+ return; // safe to access later
+ }
+ }
Ranges singleRanges = Ranges.of(range);
if (!allowed.containsAll(singleRanges))
- throw new IllegalStateException(String.format("Attempted to access range %s on node %s, which is not in the range %s;\nexecuteAt = %s\n%s",
- range, node, allowed,
- executeAt,
- history(singleRanges)));
+ {
+ // TODO (required): it is actually safe for a node on an old epoch to still be executing a transaction that has been executed in a later epoch,
+ // making this check over-enthusiastic.
+ illegalState(String.format("Attempted to access range %s on node %s, which is not in the range %s;\nexecuteAt = %s\n%s",
+ range, node, allowed, executeAt, history(singleRanges)));
+ }
}
private String history(String type, Object key, Predicate<Ranges> test)
@@ -465,6 +481,7 @@
}
// TODO (duplication): this is 95% of accord.coordinate.CoordinateShardDurable
+ // we already report all this information to EpochState; would be better to use that
private static class Await extends AsyncResults.SettableResult<SyncPoint<Ranges>> implements Callback<ReadReply>
{
private final Node node;
@@ -484,12 +501,14 @@
Await coordinate = new Await(node, minEpoch, sp);
coordinate.start();
return coordinate.recover(t -> {
- // TODO (effecicency): backoff
if (t instanceof Timeout ||
// TODO (expected): why are we not simply handling Insufficient properly?
t instanceof RuntimeException && "Insufficient".equals(t.getMessage()) ||
t instanceof SimulatedFault)
return coordinate(node, minEpoch, sp);
+ // cannot loop indefinitely
+ if (t instanceof RuntimeException && "Redundant".equals(t.getMessage()))
+ return AsyncChains.success(null);
return null;
});
}
diff --git a/accord-core/src/test/java/accord/impl/list/ListWrite.java b/accord-core/src/test/java/accord/impl/list/ListWrite.java
index ca80714..8966ec6 100644
--- a/accord-core/src/test/java/accord/impl/list/ListWrite.java
+++ b/accord-core/src/test/java/accord/impl/list/ListWrite.java
@@ -54,7 +54,7 @@
ListStore s = (ListStore) store;
if (!containsKey(key))
return Writes.SUCCESS;
- CommandsForKeys.updateLastExecutionTimestamps((AbstractSafeCommandStore<?, ?, ?, ?>) safeStore, (RoutableKey) key, executeAt, true);
+ CommandsForKeys.updateLastExecutionTimestamps((AbstractSafeCommandStore<?, ?, ?>) safeStore, (RoutableKey) key, executeAt, true);
return executor.apply(safeStore.commandStore()).submit(() -> {
int[] data = get(key);
diff --git a/accord-core/src/test/java/accord/impl/mock/MockCluster.java b/accord-core/src/test/java/accord/impl/mock/MockCluster.java
index 9e1fddb..f3b80c5 100644
--- a/accord-core/src/test/java/accord/impl/mock/MockCluster.java
+++ b/accord-core/src/test/java/accord/impl/mock/MockCluster.java
@@ -34,8 +34,7 @@
import accord.NetworkFilter;
import accord.api.MessageSink;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
import accord.config.LocalConfig;
import accord.impl.InMemoryCommandStores;
import accord.impl.IntKey;
@@ -47,7 +46,6 @@
import accord.local.Node.Id;
import accord.local.NodeTimeService;
import accord.local.ShardDistributor;
-import accord.messages.Apply;
import accord.config.MutableLocalConfig;
import accord.messages.Callback;
import accord.messages.LocalRequest;
@@ -139,9 +137,7 @@
SizeOfIntersectionSorter.SUPPLIER,
SimpleProgressLog::new,
InMemoryCommandStores.SingleThread::new,
- ExecuteTxn.FACTORY,
- PersistTxn.FACTORY,
- Apply.FACTORY,
+ new CoordinationAdapter.DefaultFactory(),
localConfig);
awaitUninterruptibly(node.unsafeStart());
node.onTopologyUpdate(topology, true);
diff --git a/accord-core/src/test/java/accord/local/ImmutableCommandTest.java b/accord-core/src/test/java/accord/local/ImmutableCommandTest.java
index 35afe00..a36ba8a 100644
--- a/accord-core/src/test/java/accord/local/ImmutableCommandTest.java
+++ b/accord-core/src/test/java/accord/local/ImmutableCommandTest.java
@@ -32,8 +32,7 @@
import accord.api.RoutingKey;
import accord.api.Scheduler;
import accord.api.TestableConfigurationService;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
import accord.config.LocalConfig;
import accord.impl.InMemoryCommandStore;
import accord.impl.InMemoryCommandStores;
@@ -46,7 +45,6 @@
import accord.impl.mock.MockStore;
import accord.local.Node.Id;
import accord.local.SaveStatus.LocalExecution;
-import accord.messages.Apply;
import accord.config.MutableLocalConfig;
import accord.primitives.FullKeyRoute;
import accord.primitives.Keys;
@@ -121,7 +119,7 @@
clock, NodeTimeService.unixWrapper(TimeUnit.MICROSECONDS, clock),
() -> storeSupport.data, new ShardDistributor.EvenSplit(8, ignore -> new IntKey.Splitter()), new TestAgent(), new DefaultRandom(), Scheduler.NEVER_RUN_SCHEDULED,
SizeOfIntersectionSorter.SUPPLIER, ignore -> ignore2 -> new NoOpProgressLog(), InMemoryCommandStores.Synchronized::new,
- ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+ new CoordinationAdapter.DefaultFactory(),
localConfig);
awaitUninterruptibly(node.unsafeStart());
node.onTopologyUpdate(storeSupport.local.get(), true);
diff --git a/accord-core/src/test/java/accord/messages/PreAcceptTest.java b/accord-core/src/test/java/accord/messages/PreAcceptTest.java
index f2593d7..4344f77 100644
--- a/accord-core/src/test/java/accord/messages/PreAcceptTest.java
+++ b/accord-core/src/test/java/accord/messages/PreAcceptTest.java
@@ -23,13 +23,9 @@
import java.util.List;
import java.util.concurrent.ExecutionException;
-import java.util.function.BiFunction;
-import java.util.stream.Stream;
import accord.api.RoutingKey;
import accord.impl.AbstractSafeCommandStore;
-import accord.impl.CommandTimeseries.CommandLoader;
-import accord.impl.CommandTimeseries;
import accord.impl.CommandsForKey;
import accord.impl.IntKey.Raw;
import accord.impl.IntKey;
@@ -86,11 +82,6 @@
return PreAccept.SerializerSupport.create(txnId, route.slice(FULL_RANGE), txnId.epoch(), txnId.epoch(), false, txnId.epoch(), txn.slice(FULL_RANGE, true), route);
}
- static <T, D> Stream<T> convert(CommandTimeseries<D> timeseries, BiFunction<CommandLoader<D>, D, T> get)
- {
- return timeseries.all().map(d -> get.apply(timeseries.loader(), d));
- }
-
@Test
void initialCommandTest() throws ExecutionException
{
@@ -103,7 +94,7 @@
{
Raw key = IntKey.key(10);
CommandStore commandStore = node.unsafeForKey(key);
- Assertions.assertFalse(inMemory(commandStore).hasDepsCommandsForKey(key));
+ Assertions.assertFalse(inMemory(commandStore).hasCommandsForKey(key));
TxnId txnId = clock.idForNode(1, ID2);
Txn txn = writeTxn(Keys.of(key));
@@ -112,8 +103,8 @@
preAccept.process(node, ID2, REPLY_CONTEXT);
commandStore.execute(PreLoadContext.contextFor(txnId, txn.keys()), safeStore -> {
- CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).depsCommandsForKey(key).current();
- TxnId commandId = convert(cfk.commands(), CommandLoader::txnId).findFirst().get();
+ CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).commandsForKey(key).current();
+ TxnId commandId = cfk.findFirst();
Command command = safeStore.ifInitialised(commandId).current();
Assertions.assertEquals(Status.PreAccepted, command.status());
});
@@ -140,7 +131,7 @@
{
Raw key = IntKey.key(10);
CommandStore commandStore = node.unsafeForKey(key);
- Assertions.assertFalse(inMemory(commandStore).hasDepsCommandsForKey(key));
+ Assertions.assertFalse(inMemory(commandStore).hasCommandsForKey(key));
TxnId txnId = clock.idForNode(1, ID2);
Txn txn = writeTxn(Keys.of(key));
@@ -163,7 +154,7 @@
{
Raw key = IntKey.key(10);
CommandStore commandStore = node.unsafeForKey(key);
- Assertions.assertFalse(inMemory(commandStore).hasDepsCommandsForKey(key));
+ Assertions.assertFalse(inMemory(commandStore).hasCommandsForKey(key));
TxnId txnId = clock.idForNode(1, ID2);
Txn txn = writeTxn(Keys.of(key));
@@ -281,8 +272,8 @@
preAccept.process(node, ID2, REPLY_CONTEXT);
commandStore.execute(PreLoadContext.contextFor(txnId, txn.keys()), safeStore -> {
- CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).depsCommandsForKey(key).current();
- TxnId commandId = convert(cfk.commands(), CommandLoader::txnId).findFirst().get();
+ CommandsForKey cfk = ((AbstractSafeCommandStore) safeStore).commandsForKey(key).current();
+ TxnId commandId = cfk.findFirst();
Command command = safeStore.ifInitialised(commandId).current();
Assertions.assertEquals(Status.PreAccepted, command.status());
});
diff --git a/accord-core/src/test/java/accord/messages/ReadDataTest.java b/accord-core/src/test/java/accord/messages/ReadDataTest.java
index 6aa4dd2..34ef106 100644
--- a/accord-core/src/test/java/accord/messages/ReadDataTest.java
+++ b/accord-core/src/test/java/accord/messages/ReadDataTest.java
@@ -44,6 +44,7 @@
import accord.impl.TopologyFactory;
import accord.impl.mock.MockCluster;
import accord.local.CheckedCommands;
+import accord.local.Command;
import accord.local.CommandStore;
import accord.local.Node;
import accord.local.PreLoadContext;
@@ -150,7 +151,7 @@
CheckedCommands.accept(safe, state.txnId, Ballot.ZERO, state.partialRoute, state.partialTxn.keys(), state.progressKey, state.executeAt, state.deps);
SafeCommand safeCommand = safe.ifInitialised(state.txnId);
- safeCommand.commit(safeCommand.current(), Ballot.ZERO, state.executeAt);
+ safeCommand.stable(safe, safeCommand.current(), Ballot.ZERO, state.executeAt, Command.WaitingOn.EMPTY);
})));
ReplyContext replyContext = state.process();
@@ -184,7 +185,7 @@
store = stores.get(1);
check(store.execute(PreLoadContext.contextFor(state.txnId, state.keys), safe -> {
SafeCommand command = safe.get(state.txnId, state.txnId, state.route);
- command.commitInvalidated();
+ command.commitInvalidated(safe);
}));
ReplyContext replyContext = state.process();
@@ -200,7 +201,7 @@
List<CommandStore> stores = stores(state);
stores.forEach(store -> check(store.execute(PreLoadContext.contextFor(state.txnId, state.keys), safe -> {
SafeCommand command = safe.get(state.txnId, state.txnId, state.route);
- command.commitInvalidated();
+ command.commitInvalidated(safe);
})));
ReplyContext replyContext = state.process();
diff --git a/accord-core/src/test/java/accord/primitives/DepsBuilderTest.java b/accord-core/src/test/java/accord/primitives/DepsBuilderTest.java
deleted file mode 100644
index cd56d22..0000000
--- a/accord-core/src/test/java/accord/primitives/DepsBuilderTest.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package accord.primitives;
-
-import accord.api.Agent;
-import accord.api.DataStore;
-import accord.api.Key;
-import accord.api.ProgressLog;
-import accord.impl.DepsBuilder;
-import accord.impl.DepsBuilder.Builder;
-import accord.impl.IntKey;
-import accord.local.*;
-import com.google.common.collect.ImmutableList;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-import javax.annotation.Nullable;
-import java.util.*;
-import java.util.function.Predicate;
-
-public class DepsBuilderTest
-{
- private static final Key KEY = IntKey.key(1);
- private static class InstrumentedSafeStore extends SafeCommandStore
- {
- public static class Removed
- {
- public final Set<TxnId> commands = new HashSet<>();
-
- void add(TxnId txnId, Timestamp executeAt)
- {
- commands.add(txnId);
- }
- }
-
- Map<Seekable, Removed> removed = new HashMap<>();
-
- @Override
- public void removeCommandFromSeekableDeps(Seekable seekable, TxnId txnId, Timestamp executeAt, Status status)
- {
- removed.computeIfAbsent(seekable, s -> new Removed()).add(txnId, executeAt);
- }
-
- @Override protected SafeCommand getInternal(TxnId txnId) { throw new UnsupportedOperationException(); }
- @Override protected SafeCommand getInternalIfLoadedAndInitialised(TxnId txnId) { throw new UnsupportedOperationException(); }
- @Override public boolean canExecuteWith(PreLoadContext context) { throw new UnsupportedOperationException(); }
- @Override public <P1, T> T mapReduce(Seekables<?, ?> keys, Ranges slice, KeyHistory keyHistory, Txn.Kind.Kinds testKind, TestTimestamp testTimestamp, Timestamp timestamp, TestDep testDep, @Nullable TxnId depId, @Nullable Status minStatus, @Nullable Status maxStatus, CommandFunction<P1, T, T> map, P1 p1, T initialValue, Predicate<? super T> terminate) { throw new UnsupportedOperationException(); }
- @Override protected void register(Seekables<?, ?> keysOrRanges, Ranges slice, Command command) { throw new UnsupportedOperationException(); }
- @Override protected void register(Seekable keyOrRange, Ranges slice, Command command) { throw new UnsupportedOperationException(); }
- @Override public CommandStore commandStore() { throw new UnsupportedOperationException(); }
- @Override public DataStore dataStore() { throw new UnsupportedOperationException(); }
- @Override public Agent agent() { throw new UnsupportedOperationException(); }
- @Override public ProgressLog progressLog() { throw new UnsupportedOperationException(); }
- @Override public NodeTimeService time() { throw new UnsupportedOperationException(); }
- @Override public CommandStores.RangesForEpoch ranges() { throw new UnsupportedOperationException(); }
- @Override public Timestamp maxConflict(Seekables<?, ?> keys, Ranges slice) { throw new UnsupportedOperationException(); }
- @Override public void registerHistoricalTransactions(Deps deps) { throw new UnsupportedOperationException(); }
- @Override public void erase(SafeCommand safeCommand) { throw new UnsupportedOperationException(); }
- }
- private static final Node.Id NODE = new Node.Id(0);
-
- private static TxnId txnId(int v)
- {
- return TxnId.fromValues(1, v, 0, 0);
- }
-
- private static List<TxnId> txnIds(int... vs)
- {
- ImmutableList.Builder<TxnId> builder = ImmutableList.builder();
- Arrays.sort(vs);
- for (int v : vs)
- builder.add(txnId(v));
- return builder.build();
- }
-
- private static Timestamp timestamp(int v)
- {
- return Timestamp.fromValues(1, v, NODE);
- }
-
- private static List<TxnId> build(SafeCommandStore safeStore, Seekable seekable, Builder builder)
- {
- List<TxnId> result = new ArrayList<>();
- builder.build(safeStore, seekable, result::add);
- result.sort(Comparator.naturalOrder());
- return result;
- }
-
- private static List<TxnId> build(SafeCommandStore safeStore, Builder builder)
- {
- return build(safeStore, KEY, builder);
- }
-
- @BeforeAll
- static void setUpClass()
- {
- DepsBuilder.setCanPruneUnsafe(true);
- }
-
- @AfterAll
- static void tearDownClass()
- {
- DepsBuilder.setCanPruneUnsafe(false);
- }
-
- @Test
- void basicTest()
- {
- InstrumentedSafeStore safeStore = new InstrumentedSafeStore();
- Builder builder = new Builder();
- builder.add(txnId(0), Status.Applied, timestamp(0), txnIds());
- builder.add(txnId(1), Status.Applied, timestamp(1), txnIds(0));
- Assertions.assertEquals(txnIds(1), build(safeStore, builder));
- }
-
- /**
- * Committed deps should be included if they point to an uncommitted command
- */
- @Test
- void uncommittedTest()
- {
- InstrumentedSafeStore safeStore = new InstrumentedSafeStore();
- Builder builder = new Builder();
- builder.add(txnId(0), Status.Accepted, timestamp(0), txnIds());
- builder.add(txnId(1), Status.Applied, timestamp(1), txnIds(0));
- builder.add(txnId(2), Status.Applied, timestamp(2), txnIds(1));
- builder.add(txnId(3), Status.Applied, timestamp(3), txnIds(2));
- Assertions.assertEquals(txnIds(0, 1, 2, 3), build(safeStore, builder));
- }
-
- /**
- * Committed txns should be included if they point to a command with a later execution time
- */
- @Test
- void executionOrderTest()
- {
- InstrumentedSafeStore safeStore = new InstrumentedSafeStore();
- Builder builder = new Builder();
- builder.add(txnId(0), Status.Applied, timestamp(2), txnIds());
- builder.add(txnId(1), Status.Applied, timestamp(1), txnIds(0));
- Assertions.assertEquals(txnIds(0, 1), build(safeStore, builder));
- }
-}
diff --git a/accord-core/src/test/java/accord/topology/TopologyRandomizer.java b/accord-core/src/test/java/accord/topology/TopologyRandomizer.java
index 555969d..7b3a919 100644
--- a/accord-core/src/test/java/accord/topology/TopologyRandomizer.java
+++ b/accord-core/src/test/java/accord/topology/TopologyRandomizer.java
@@ -50,6 +50,9 @@
import accord.utils.RandomSource;
import org.agrona.collections.IntHashSet;
+import static accord.burn.BurnTest.HASH_RANGE_END;
+import static accord.burn.BurnTest.HASH_RANGE_START;
+
// TODO (required, testing): add change replication factor
public class TopologyRandomizer
@@ -348,7 +351,7 @@
}
List<Shard> result = new ArrayList<>(shards.length + nodes.length);
result.addAll(Arrays.asList(shards));
- Range[] ranges = PrefixedIntHashKey.ranges(prefix, nodes.length);
+ Range[] ranges = PrefixedIntHashKey.ranges(prefix, HASH_RANGE_START, HASH_RANGE_END, nodes.length);
for (int i = 0; i < ranges.length; i++)
{
Range range = ranges[i];
diff --git a/accord-core/src/test/java/accord/utils/AccordGens.java b/accord-core/src/test/java/accord/utils/AccordGens.java
index 0186ff3..d87c2d5 100644
--- a/accord-core/src/test/java/accord/utils/AccordGens.java
+++ b/accord-core/src/test/java/accord/utils/AccordGens.java
@@ -187,9 +187,14 @@
return prefixedIntHashKey(prefixGen, rs -> rs.nextInt(PrefixedIntHashKey.MIN_KEY, Integer.MAX_VALUE));
}
- public static Gen<Key> prefixedIntHashKey(Gen.IntGen prefixGen, Gen.IntGen valueGen)
+ public static Gen<Key> prefixedIntHashKey(Gen.IntGen prefixGen, Gen.IntGen keyGen)
{
- return rs -> PrefixedIntHashKey.key(prefixGen.nextInt(rs), valueGen.nextInt(rs));
+ return rs -> {
+ int prefix = prefixGen.nextInt(rs);
+ int key = keyGen.nextInt(rs);
+ int hash = PrefixedIntHashKey.hash(key);
+ return PrefixedIntHashKey.key(prefix, key, hash);
+ };
}
public static Gen<Key> prefixedIntHashKeyInsideRanges(Ranges ranges)
@@ -201,7 +206,7 @@
// end inclusive, so +1 the result to include end and exclude start
int hash = rs.nextInt(start.hash, end.hash) + 1;
int key = CRCUtils.reverseCRC32LittleEnding(hash);
- PrefixedIntHashKey.Key ret = PrefixedIntHashKey.key(start.prefix, key);
+ PrefixedIntHashKey.Key ret = PrefixedIntHashKey.key(start.prefix, key, hash);
// we have tests to make sure this doesn't fail... just a safety check
assert ret.hash == hash;
return ret;
diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java b/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java
index 67ed4f3..1d9abff 100644
--- a/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java
+++ b/accord-maelstrom/src/main/java/accord/maelstrom/Cluster.java
@@ -43,8 +43,7 @@
import accord.api.Scheduler;
import accord.config.LocalConfig;
import accord.config.MutableLocalConfig;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
+import accord.coordinate.CoordinationAdapter;
import accord.impl.InMemoryCommandStores;
import accord.impl.SimpleProgressLog;
import accord.impl.SizeOfIntersectionSorter;
@@ -53,7 +52,6 @@
import accord.local.Node.Id;
import accord.local.NodeTimeService;
import accord.local.ShardDistributor;
-import accord.messages.Apply;
import accord.messages.Callback;
import accord.messages.LocalRequest;
import accord.messages.Reply;
@@ -328,7 +326,7 @@
MaelstromStore::new, new ShardDistributor.EvenSplit(8, ignore -> new MaelstromKey.Splitter()),
MaelstromAgent.INSTANCE,
randomSupplier.get(), sinks, SizeOfIntersectionSorter.SUPPLIER,
- SimpleProgressLog::new, InMemoryCommandStores.SingleThread::new, ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+ SimpleProgressLog::new, InMemoryCommandStores.SingleThread::new, new CoordinationAdapter.DefaultFactory(),
localConfig));
}
diff --git a/accord-maelstrom/src/main/java/accord/maelstrom/Main.java b/accord-maelstrom/src/main/java/accord/maelstrom/Main.java
index 50d1aee..a7ac2ce 100644
--- a/accord-maelstrom/src/main/java/accord/maelstrom/Main.java
+++ b/accord-maelstrom/src/main/java/accord/maelstrom/Main.java
@@ -34,9 +34,8 @@
import accord.api.Scheduler;
import accord.config.LocalConfig;
import accord.config.MutableLocalConfig;
+import accord.coordinate.CoordinationAdapter;
import accord.coordinate.Timeout;
-import accord.coordinate.ExecuteTxn;
-import accord.coordinate.PersistTxn;
import accord.impl.InMemoryCommandStores;
import accord.impl.SimpleProgressLog;
import accord.impl.SizeOfIntersectionSorter;
@@ -46,7 +45,6 @@
import accord.local.NodeTimeService;
import accord.local.ShardDistributor;
import accord.maelstrom.Packet.Type;
-import accord.messages.Apply;
import accord.messages.Callback;
import accord.messages.LocalRequest;
import accord.messages.Reply;
@@ -183,7 +181,7 @@
System::currentTimeMillis, NodeTimeService.unixWrapper(TimeUnit.MILLISECONDS, System::currentTimeMillis),
MaelstromStore::new, new ShardDistributor.EvenSplit(8, ignore -> new MaelstromKey.Splitter()),
MaelstromAgent.INSTANCE, new DefaultRandom(), scheduler, SizeOfIntersectionSorter.SUPPLIER,
- SimpleProgressLog::new, InMemoryCommandStores.SingleThread::new, ExecuteTxn.FACTORY, PersistTxn.FACTORY, Apply.FACTORY,
+ SimpleProgressLog::new, InMemoryCommandStores.SingleThread::new, new CoordinationAdapter.DefaultFactory(),
localConfig);
awaitUninterruptibly(on.unsafeStart());
err.println("Initialized node " + init.self);