X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2FRaftActor.java;h=8135d837d3ad86563ad0246022941bfc6134b59d;hp=dd9572c9a73ddc6c1c5b85396c455cc3bf26d426;hb=b10d77375b5a290143106180f1583ea4e18f8478;hpb=b725909c696c8d40006a6297dc54a467fddcf6b3 diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java index dd9572c9a7..8135d837d3 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java @@ -19,6 +19,14 @@ import akka.persistence.SaveSnapshotSuccess; import akka.persistence.SnapshotOffer; import akka.persistence.SnapshotSelectionCriteria; import akka.persistence.UntypedPersistentActor; +import com.google.common.base.Optional; +import com.google.protobuf.ByteString; +import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; +import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; +import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot; +import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply; +import org.opendaylight.controller.cluster.raft.base.messages.Replicate; +import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat; import org.opendaylight.controller.cluster.raft.behaviors.Candidate; import org.opendaylight.controller.cluster.raft.behaviors.Follower; import org.opendaylight.controller.cluster.raft.behaviors.Leader; @@ -26,14 +34,12 @@ import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior; import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer; import org.opendaylight.controller.cluster.raft.client.messages.FindLeader; import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply; -import org.opendaylight.controller.cluster.raft.internal.messages.ApplySnapshot; import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer; -import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState; -import org.opendaylight.controller.cluster.raft.internal.messages.Replicate; +import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; +import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload; +import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages; import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; import java.util.Map; /** @@ -97,16 +103,27 @@ public abstract class RaftActor extends UntypedPersistentActor { */ private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl(); + private CaptureSnapshot captureSnapshot = null; + + private volatile boolean hasSnapshotCaptureInitiated = false; public RaftActor(String id, Map peerAddresses) { + this(id, peerAddresses, Optional.absent()); + } + + public RaftActor(String id, Map peerAddresses, + Optional configParams) { + context = new RaftActorContextImpl(this.getSelf(), - this.getContext(), - id, new ElectionTermImpl(getSelf().path().toString()), - -1, -1, replicatedLog, peerAddresses, LOG); + this.getContext(), id, new ElectionTermImpl(), + -1, -1, replicatedLog, peerAddresses, + (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()), + LOG); } @Override public void onReceiveRecover(Object message) { if (message instanceof SnapshotOffer) { + LOG.debug("SnapshotOffer called.."); SnapshotOffer offer = (SnapshotOffer) message; Snapshot snapshot = (Snapshot) offer.snapshot(); @@ -115,15 +132,32 @@ public abstract class RaftActor extends UntypedPersistentActor { // when we need to install it on a peer replicatedLog = new ReplicatedLogImpl(snapshot); + context.setReplicatedLog(replicatedLog); + context.setLastApplied(snapshot.getLastAppliedIndex()); + + LOG.debug("Applied snapshot to replicatedLog. " + + "snapshotIndex={}, snapshotTerm={}, journal-size={}", + replicatedLog.snapshotIndex, replicatedLog.snapshotTerm, + replicatedLog.size()); + // Apply the snapshot to the actors state - applySnapshot(snapshot.getState()); + applySnapshot(ByteString.copyFrom(snapshot.getState())); } else if (message instanceof ReplicatedLogEntry) { replicatedLog.append((ReplicatedLogEntry) message); + } else if (message instanceof DeleteEntries) { + replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex()); + } else if (message instanceof UpdateElectionTerm) { + context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(), ((UpdateElectionTerm) message).getVotedFor()); } else if (message instanceof RecoveryCompleted) { LOG.debug( - "Last index in log : " + replicatedLog.lastIndex()); + "RecoveryCompleted - Switching actor to Follower - " + + "Last index in log:{}, snapshotIndex={}, snapshotTerm={}, " + + "journal-size={}", + replicatedLog.lastIndex(), replicatedLog.snapshotIndex, + replicatedLog.snapshotTerm, replicatedLog.size()); currentBehavior = switchBehavior(RaftState.Follower); + onStateChanged(); } } @@ -131,46 +165,108 @@ public abstract class RaftActor extends UntypedPersistentActor { if (message instanceof ApplyState){ ApplyState applyState = (ApplyState) message; - LOG.debug("Applying state for log index {}", - applyState.getReplicatedLogEntry().getIndex()); + LOG.debug("Applying state for log index {} data {}", + applyState.getReplicatedLogEntry().getIndex(), + applyState.getReplicatedLogEntry().getData()); applyState(applyState.getClientActor(), applyState.getIdentifier(), applyState.getReplicatedLogEntry().getData()); + } else if(message instanceof ApplySnapshot ) { - applySnapshot(((ApplySnapshot) message).getSnapshot()); + Snapshot snapshot = ((ApplySnapshot) message).getSnapshot(); + + LOG.debug("ApplySnapshot called on Follower Actor " + + "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(), + snapshot.getLastAppliedTerm()); + applySnapshot(ByteString.copyFrom(snapshot.getState())); + + //clears the followers log, sets the snapshot index to ensure adjusted-index works + replicatedLog = new ReplicatedLogImpl(snapshot); + context.setReplicatedLog(replicatedLog); + context.setLastApplied(snapshot.getLastAppliedIndex()); + } else if (message instanceof FindLeader) { getSender().tell( - new FindLeaderReply( - context.getPeerAddress(currentBehavior.getLeaderId())), + new FindLeaderReply(getLeaderAddress()), getSelf() ); + } else if (message instanceof SaveSnapshotSuccess) { SaveSnapshotSuccess success = (SaveSnapshotSuccess) message; + LOG.info("SaveSnapshotSuccess received for snapshot"); + + context.getReplicatedLog().snapshotCommit(); // TODO: Not sure if we want to be this aggressive with trimming stuff trimPersistentData(success.metadata().sequenceNr()); } else if (message instanceof SaveSnapshotFailure) { - // TODO: Handle failure in saving the snapshot - } else if (message instanceof FindLeader){ - getSender().tell(new FindLeaderReply( - context.getPeerAddress(currentBehavior.getLeaderId())), - getSelf()); + SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message; + + LOG.info("saveSnapshotFailure.metadata():{}", saveSnapshotFailure.metadata().toString()); + LOG.error(saveSnapshotFailure.cause(), "SaveSnapshotFailure received for snapshot Cause:"); + + context.getReplicatedLog().snapshotRollback(); + + LOG.info("Replicated Log rollbacked. Snapshot will be attempted in the next cycle." + + "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", + context.getReplicatedLog().getSnapshotIndex(), + context.getReplicatedLog().getSnapshotTerm(), + context.getReplicatedLog().size()); } else if (message instanceof AddRaftPeer){ + + // FIXME : Do not add raft peers like this. + // When adding a new Peer we have to ensure that the a majority of + // the peers know about the new Peer. Doing it this way may cause + // a situation where multiple Leaders may emerge AddRaftPeer arp = (AddRaftPeer)message; context.addToPeers(arp.getName(), arp.getAddress()); } else if (message instanceof RemoveRaftPeer){ + RemoveRaftPeer rrp = (RemoveRaftPeer)message; context.removePeer(rrp.getName()); + + } else if (message instanceof CaptureSnapshot) { + LOG.debug("CaptureSnapshot received by actor"); + CaptureSnapshot cs = (CaptureSnapshot)message; + captureSnapshot = cs; + createSnapshot(); + + } else if (message instanceof CaptureSnapshotReply){ + LOG.debug("CaptureSnapshotReply received by actor"); + CaptureSnapshotReply csr = (CaptureSnapshotReply) message; + + ByteString stateInBytes = csr.getSnapshot(); + LOG.debug("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size()); + handleCaptureSnapshotReply(stateInBytes); + } else { + if (!(message instanceof AppendEntriesMessages.AppendEntries) + && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) { + LOG.debug("onReceiveCommand: message:" + message.getClass()); + } + RaftState state = currentBehavior.handleMessage(getSender(), message); + RaftActorBehavior oldBehavior = currentBehavior; currentBehavior = switchBehavior(state); + if(oldBehavior != currentBehavior){ + onStateChanged(); + } } } + public java.util.Set getPeers() { + return context.getPeerAddresses().keySet(); + } + + protected String getReplicatedLogState() { + return "snapshotIndex=" + context.getReplicatedLog().getSnapshotIndex() + + ", snapshotTerm=" + context.getReplicatedLog().getSnapshotTerm() + + ", im-mem journal size=" + context.getReplicatedLog().size(); + } /** @@ -182,12 +278,14 @@ public abstract class RaftActor extends UntypedPersistentActor { * @param data */ protected void persistData(ActorRef clientActor, String identifier, - Object data) { - LOG.debug("Persist data " + identifier); + Payload data) { + ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry( context.getReplicatedLog().lastIndex() + 1, context.getTermInformation().getCurrentTerm(), data); + LOG.debug("Persist data {}", replicatedLogEntry); + replicatedLog .appendAndPersist(clientActor, identifier, replicatedLogEntry); } @@ -214,20 +312,61 @@ public abstract class RaftActor extends UntypedPersistentActor { * @return A reference to the leader if known, null otherwise */ protected ActorSelection getLeader(){ - String leaderId = currentBehavior.getLeaderId(); - if (leaderId == null) { + String leaderAddress = getLeaderAddress(); + + if(leaderAddress == null){ return null; } - String peerAddress = context.getPeerAddress(leaderId); - LOG.debug("getLeader leaderId = " + leaderId + " peerAddress = " - + peerAddress); - return context.actorSelection(peerAddress); + + return context.actorSelection(leaderAddress); + } + + /** + * + * @return the current leader's id + */ + protected String getLeaderId(){ + return currentBehavior.getLeaderId(); } protected RaftState getRaftState() { return currentBehavior.state(); } + protected ReplicatedLogEntry getLastLogEntry() { + return replicatedLog.last(); + } + + protected Long getCurrentTerm(){ + return context.getTermInformation().getCurrentTerm(); + } + + protected Long getCommitIndex(){ + return context.getCommitIndex(); + } + + protected Long getLastApplied(){ + return context.getLastApplied(); + } + + /** + * setPeerAddress sets the address of a known peer at a later time. + *

+ * This is to account for situations where a we know that a peer + * exists but we do not know an address up-front. This may also be used in + * situations where a known peer starts off in a different location and we + * need to change it's address + *

+ * Note that if the peerId does not match the list of peers passed to + * this actor during construction an IllegalStateException will be thrown. + * + * @param peerId + * @param peerAddress + */ + protected void setPeerAddress(String peerId, String peerAddress){ + context.setPeerAddress(peerId, peerAddress); + } + /** @@ -259,7 +398,7 @@ public abstract class RaftActor extends UntypedPersistentActor { * * @return The current state of the actor */ - protected abstract Object createSnapshot(); + protected abstract void createSnapshot(); /** * This method will be called by the RaftActor during recovery to @@ -271,7 +410,14 @@ public abstract class RaftActor extends UntypedPersistentActor { * * @param snapshot A snapshot of the state of the actor */ - protected abstract void applySnapshot(Object snapshot); + protected abstract void applySnapshot(ByteString snapshot); + + /** + * This method will be called by the RaftActor when the state of the + * RaftActor changes. The derived actor can then use methods like + * isLeader or getLeader to do something useful + */ + protected abstract void onStateChanged(); private RaftActorBehavior switchBehavior(RaftState state) { if (currentBehavior != null) { @@ -299,103 +445,96 @@ public abstract class RaftActor extends UntypedPersistentActor { } else { behavior = new Leader(context); } + + + return behavior; } private void trimPersistentData(long sequenceNumber) { - // Trim snapshots + // Trim akka snapshots // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied // For now guessing that it is ANDed. deleteSnapshots(new SnapshotSelectionCriteria( - sequenceNumber - 100000, 43200000)); + sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000)); - // Trim journal + // Trim akka journal deleteMessages(sequenceNumber); } - private class ReplicatedLogImpl implements ReplicatedLog { - private final List journal; - private final Object snapshot; - private long snapshotIndex = -1; - private long snapshotTerm = -1; + private String getLeaderAddress(){ + if(isLeader()){ + return getSelf().path().toString(); + } + String leaderId = currentBehavior.getLeaderId(); + if (leaderId == null) { + return null; + } + String peerAddress = context.getPeerAddress(leaderId); + LOG.debug("getLeaderAddress leaderId = " + leaderId + " peerAddress = " + + peerAddress); - public ReplicatedLogImpl(Snapshot snapshot) { - this.snapshot = snapshot.getState(); - this.snapshotIndex = snapshot.getLastAppliedIndex(); - this.snapshotTerm = snapshot.getLastAppliedTerm(); + return peerAddress; + } - this.journal = new ArrayList<>(snapshot.getUnAppliedEntries()); - } + private void handleCaptureSnapshotReply(ByteString stateInBytes) { + // create a snapshot object from the state provided and save it + // when snapshot is saved async, SaveSnapshotSuccess is raised. - public ReplicatedLogImpl() { - this.snapshot = null; - this.journal = new ArrayList<>(); - } + Snapshot sn = Snapshot.create(stateInBytes.toByteArray(), + context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1), + captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), + captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); - @Override public ReplicatedLogEntry get(long index) { - int adjustedIndex = adjustedIndex(index); + saveSnapshot(sn); - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return null; - } + LOG.info("Persisting of snapshot done:{}", sn.getLogMessage()); - return journal.get(adjustedIndex); - } + //be greedy and remove entries from in-mem journal which are in the snapshot + // and update snapshotIndex and snapshotTerm without waiting for the success, - @Override public ReplicatedLogEntry last() { - if (journal.size() == 0) { - return null; - } - return get(journal.size() - 1); - } + context.getReplicatedLog().snapshotPreCommit(stateInBytes, + captureSnapshot.getLastAppliedIndex(), + captureSnapshot.getLastAppliedTerm()); - @Override public long lastIndex() { - if (journal.size() == 0) { - return -1; - } + LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " + + "and term:{}", captureSnapshot.getLastAppliedIndex(), + captureSnapshot.getLastAppliedTerm()); - return last().getIndex(); - } + captureSnapshot = null; + hasSnapshotCaptureInitiated = false; + } - @Override public long lastTerm() { - if (journal.size() == 0) { - return -1; - } - return last().getTerm(); + private class ReplicatedLogImpl extends AbstractReplicatedLogImpl { + + public ReplicatedLogImpl(Snapshot snapshot) { + super(ByteString.copyFrom(snapshot.getState()), + snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(), + snapshot.getUnAppliedEntries()); } + public ReplicatedLogImpl() { + super(); + } - @Override public void removeFrom(long index) { - int adjustedIndex = adjustedIndex(index); + @Override public void removeFromAndPersist(long logEntryIndex) { + int adjustedIndex = adjustedIndex(logEntryIndex); - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { + if (adjustedIndex < 0) { return; } - for (int i = adjustedIndex; - i < journal.size(); i++) { - deleteMessage(i); - journal.remove(i); - } - } - @Override public void append( - final ReplicatedLogEntry replicatedLogEntry) { - journal.add(replicatedLogEntry); - } + // FIXME: Maybe this should be done after the command is saved + journal.subList(adjustedIndex , journal.size()).clear(); - @Override public List getFrom(long index) { - int adjustedIndex = adjustedIndex(index); + persist(new DeleteEntries(adjustedIndex), new Procedure(){ - List entries = new ArrayList<>(100); - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return entries; - } - for (int i = adjustedIndex; - i < journal.size(); i++) { - entries.add(journal.get(i)); - } - return entries; + @Override public void apply(DeleteEntries param) + throws Exception { + //FIXME : Doing nothing for now + } + }); } @Override public void appendAndPersist( @@ -407,7 +546,7 @@ public abstract class RaftActor extends UntypedPersistentActor { final String identifier, final ReplicatedLogEntry replicatedLogEntry) { context.getLogger().debug( - "Append log entry and persist " + replicatedLogEntry.getIndex()); + "Append log entry and persist {} ", replicatedLogEntry); // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs journal.add(replicatedLogEntry); @@ -419,21 +558,30 @@ public abstract class RaftActor extends UntypedPersistentActor { persist(replicatedLogEntry, new Procedure() { public void apply(ReplicatedLogEntry evt) throws Exception { - // FIXME : Tentatively create a snapshot every hundred thousand entries. To be tuned. - if (size() > 100000) { - ReplicatedLogEntry lastAppliedEntry = - get(context.getLastApplied()); + // when a snaphsot is being taken, captureSnapshot != null + if (hasSnapshotCaptureInitiated == false && + journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0) { + + LOG.info("Initiating Snapshot Capture.."); long lastAppliedIndex = -1; long lastAppliedTerm = -1; + + ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied()); if (lastAppliedEntry != null) { lastAppliedIndex = lastAppliedEntry.getIndex(); lastAppliedTerm = lastAppliedEntry.getTerm(); } - saveSnapshot(Snapshot.create(createSnapshot(), - getFrom(context.getLastApplied() + 1), - lastIndex(), lastTerm(), lastAppliedIndex, - lastAppliedTerm)); + LOG.debug("Snapshot Capture logSize: {}", journal.size()); + LOG.debug("Snapshot Capture lastApplied:{} ", context.getLastApplied()); + LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex); + LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm); + + // send a CaptureSnapshot to self to make the expensive operation async. + getSelf().tell(new CaptureSnapshot( + lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm), + null); + hasSnapshotCaptureInitiated = true; } // Send message for replication if (clientActor != null) { @@ -447,119 +595,74 @@ public abstract class RaftActor extends UntypedPersistentActor { ); } - @Override public long size() { - return journal.size() + snapshotIndex; - } - - @Override public boolean isPresent(long index) { - int adjustedIndex = adjustedIndex(index); + } - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return false; - } - return true; - } + private static class DeleteEntries implements Serializable { + private final int fromIndex; - @Override public boolean isInSnapshot(long index) { - return index <= snapshotIndex; - } - @Override public Object getSnapshot() { - return snapshot; + public DeleteEntries(int fromIndex) { + this.fromIndex = fromIndex; } - @Override public long getSnapshotIndex() { - return snapshotIndex; - } - - @Override public long getSnapshotTerm() { - return snapshotTerm; - } - - private int adjustedIndex(long index) { - if(snapshotIndex < 0){ - return (int) index; - } - return (int) (index - snapshotIndex); + public int getFromIndex() { + return fromIndex; } } - private static class ReplicatedLogImplEntry implements ReplicatedLogEntry, - Serializable { - - private final long index; - private final long term; - private final Object payload; - - public ReplicatedLogImplEntry(long index, long term, Object payload) { - - this.index = index; - this.term = term; - this.payload = payload; - } - - @Override public Object getData() { - return payload; - } + private class ElectionTermImpl implements ElectionTerm { + /** + * Identifier of the actor whose election term information this is + */ + private long currentTerm = 0; + private String votedFor = null; - @Override public long getTerm() { - return term; + public long getCurrentTerm() { + return currentTerm; } - @Override public long getIndex() { - return index; + public String getVotedFor() { + return votedFor; } - } + @Override public void update(long currentTerm, String votedFor) { + LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor); - private static class Snapshot implements Serializable { - private final Object state; - private final List unAppliedEntries; - private final long lastIndex; - private final long lastTerm; - private final long lastAppliedIndex; - private final long lastAppliedTerm; - - private Snapshot(Object state, - List unAppliedEntries, long lastIndex, - long lastTerm, long lastAppliedIndex, long lastAppliedTerm) { - this.state = state; - this.unAppliedEntries = unAppliedEntries; - this.lastIndex = lastIndex; - this.lastTerm = lastTerm; - this.lastAppliedIndex = lastAppliedIndex; - this.lastAppliedTerm = lastAppliedTerm; + this.currentTerm = currentTerm; + this.votedFor = votedFor; } + @Override + public void updateAndPersist(long currentTerm, String votedFor){ + update(currentTerm, votedFor); + // FIXME : Maybe first persist then update the state + persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure(){ - public static Snapshot create(Object state, - List entries, long lastIndex, long lastTerm, - long lastAppliedIndex, long lastAppliedTerm) { - return new Snapshot(state, entries, lastIndex, lastTerm, - lastAppliedIndex, lastAppliedTerm); - } + @Override public void apply(UpdateElectionTerm param) + throws Exception { - public Object getState() { - return state; + } + }); } + } - public List getUnAppliedEntries() { - return unAppliedEntries; - } + private static class UpdateElectionTerm implements Serializable { + private final long currentTerm; + private final String votedFor; - public long getLastTerm() { - return lastTerm; + public UpdateElectionTerm(long currentTerm, String votedFor) { + this.currentTerm = currentTerm; + this.votedFor = votedFor; } - public long getLastAppliedIndex() { - return lastAppliedIndex; + public long getCurrentTerm() { + return currentTerm; } - public long getLastAppliedTerm() { - return lastAppliedTerm; + public String getVotedFor() { + return votedFor; } } - }