X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2FRaftActor.java;h=15239795a8abc2aab0019439e2a64bbab2701afc;hb=ed693440aa741fee9b94447f8404d89b4020f616;hp=826faf7414395bdb1eefcc59906c3cd1f6ea0e16;hpb=fdab53ef9033fc83c812f7d3d6d3327d3d176f0f;p=controller.git 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 826faf7414..15239795a8 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 @@ -14,15 +14,23 @@ import akka.event.Logging; import akka.event.LoggingAdapter; import akka.japi.Procedure; import akka.persistence.RecoveryCompleted; +import akka.persistence.SaveSnapshotFailure; +import akka.persistence.SaveSnapshotSuccess; +import akka.persistence.SnapshotOffer; +import akka.persistence.SnapshotSelectionCriteria; import akka.persistence.UntypedPersistentActor; import org.opendaylight.controller.cluster.raft.behaviors.Candidate; import org.opendaylight.controller.cluster.raft.behaviors.Follower; import org.opendaylight.controller.cluster.raft.behaviors.Leader; 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.protobuff.client.messages.Payload; import java.io.Serializable; import java.util.ArrayList; @@ -33,50 +41,48 @@ import java.util.Map; * RaftActor encapsulates a state machine that needs to be kept synchronized * in a cluster. It implements the RAFT algorithm as described in the paper * - * In Search of an Understandable Consensus Algorithm - *

+ * In Search of an Understandable Consensus Algorithm + *

* RaftActor has 3 states and each state has a certain behavior associated * with it. A Raft actor can behave as, *

- * - *

+ *

+ *

* A RaftActor MUST be a Leader in order to accept requests from clients to * change the state of it's encapsulated state machine. Once a RaftActor becomes * a Leader it is also responsible for ensuring that all followers ultimately * have the same log and therefore the same state machine as itself. - * - *

+ *

+ *

* The current behavior of a RaftActor determines how election for leadership * is initiated and how peer RaftActors react to request for votes. - * - *

+ *

+ *

* Each RaftActor also needs to know the current election term. It uses this * information for a couple of things. One is to simply figure out who it * voted for in the last election. Another is to figure out if the message * it received to update it's state is stale. - * - *

+ *

+ *

* The RaftActor uses akka-persistence to store it's replicated log. * Furthermore through it's behaviors a Raft Actor determines - * + *

*

- * - * UntypeEventSourceProcessor */ public abstract class RaftActor extends UntypedPersistentActor { protected final LoggingAdapter LOG = Logging.getLogger(getContext().system(), this); /** - * The current state determines the current behavior of a RaftActor + * The current state determines the current behavior of a RaftActor * A Raft Actor always starts off in the Follower State */ private RaftActorBehavior currentBehavior; @@ -93,70 +99,99 @@ public abstract class RaftActor extends UntypedPersistentActor { private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl(); - - public RaftActor(String id, Map peerAddresses){ + public RaftActor(String id, Map peerAddresses) { context = new RaftActorContextImpl(this.getSelf(), this.getContext(), - id, new ElectionTermImpl(getSelf().path().toString()), + id, new ElectionTermImpl(), -1, -1, replicatedLog, peerAddresses, LOG); - currentBehavior = switchBehavior(RaftState.Follower); } @Override public void onReceiveRecover(Object message) { - if(message instanceof ReplicatedLogEntry) { + if (message instanceof SnapshotOffer) { + SnapshotOffer offer = (SnapshotOffer) message; + Snapshot snapshot = (Snapshot) offer.snapshot(); + + // Create a replicated log with the snapshot information + // The replicated log can be used later on to retrieve this snapshot + // when we need to install it on a peer + replicatedLog = new ReplicatedLogImpl(snapshot); + + // Apply the snapshot to the actors state + applySnapshot(snapshot.getState()); + + } else if (message instanceof ReplicatedLogEntry) { replicatedLog.append((ReplicatedLogEntry) message); - } else if(message instanceof RecoveryCompleted){ - LOG.debug("Log now has messages to index : " + replicatedLog.lastIndex()); + } 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()); + currentBehavior = switchBehavior(RaftState.Follower); } } @Override public void onReceiveCommand(Object message) { - if(message instanceof ApplyState){ - - ApplyState applyState = (ApplyState) message; + 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 FindLeader){ + + } else if(message instanceof ApplySnapshot ) { + applySnapshot(((ApplySnapshot) message).getSnapshot()); + + } else if (message instanceof FindLeader) { + getSender().tell( + new FindLeaderReply( + context.getPeerAddress(currentBehavior.getLeaderId())), + getSelf() + ); + + } else if (message instanceof SaveSnapshotSuccess) { + SaveSnapshotSuccess success = (SaveSnapshotSuccess) message; + + // 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()); + + } 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 { + RaftState state = currentBehavior.handleMessage(getSender(), message); currentBehavior = switchBehavior(state); } } - private RaftActorBehavior switchBehavior(RaftState state){ - if(currentBehavior != null) { - if (currentBehavior.state() == state) { - return currentBehavior; - } - LOG.info("Switching from state " + currentBehavior.state() + " to " - + state); - try { - currentBehavior.close(); - } catch (Exception e) { - LOG.error(e, "Failed to close behavior : " + currentBehavior.state()); - } - - } else { - LOG.info("Switching behavior to " + state); - } - RaftActorBehavior behavior = null; - if(state == RaftState.Candidate){ - behavior = new Candidate(context); - } else if(state == RaftState.Follower){ - behavior = new Follower(context); - } else { - behavior = new Leader(context); - } - return behavior; - } /** * When a derived RaftActor needs to persist something it must call @@ -166,55 +201,179 @@ public abstract class RaftActor extends UntypedPersistentActor { * @param identifier * @param data */ - protected void persistData(ActorRef clientActor, String identifier, Object data){ - LOG.debug("Persist data " + identifier); + protected void persistData(ActorRef clientActor, String identifier, + Payload data) { + ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry( context.getReplicatedLog().lastIndex() + 1, context.getTermInformation().getCurrentTerm(), data); - replicatedLog.appendAndPersist(clientActor, identifier, replicatedLogEntry); - } + LOG.debug("Persist data {}", replicatedLogEntry); - protected abstract void applyState(ActorRef clientActor, String identifier, Object data); + replicatedLog + .appendAndPersist(clientActor, identifier, replicatedLogEntry); + } - protected String getId(){ + protected String getId() { return context.getId(); } - protected boolean isLeader(){ + /** + * Derived actors can call the isLeader method to check if the current + * RaftActor is the Leader or not + * + * @return true it this RaftActor is a Leader false otherwise + */ + protected boolean isLeader() { return context.getId().equals(currentBehavior.getLeaderId()); } + /** + * Derived actor can call getLeader if they need a reference to the Leader. + * This would be useful for example in forwarding a request to an actor + * which is the leader + * + * @return A reference to the leader if known, null otherwise + */ protected ActorSelection getLeader(){ String leaderId = currentBehavior.getLeaderId(); + if (leaderId == null) { + return null; + } String peerAddress = context.getPeerAddress(leaderId); - LOG.debug("getLeader leaderId = " + leaderId + " peerAddress = " + peerAddress); + LOG.debug("getLeader leaderId = " + leaderId + " peerAddress = " + + peerAddress); return context.actorSelection(peerAddress); } + protected RaftState getRaftState() { + return currentBehavior.state(); + } + + + + /** + * The applyState method will be called by the RaftActor when some data + * needs to be applied to the actor's state + * + * @param clientActor A reference to the client who sent this message. This + * is the same reference that was passed to persistData + * by the derived actor. clientActor may be null when + * the RaftActor is behaving as a follower or during + * recovery. + * @param identifier The identifier of the persisted data. This is also + * the same identifier that was passed to persistData by + * the derived actor. identifier may be null when + * the RaftActor is behaving as a follower or during + * recovery + * @param data A piece of data that was persisted by the persistData call. + * This should NEVER be null. + */ + protected abstract void applyState(ActorRef clientActor, String identifier, + Object data); + + /** + * This method will be called by the RaftActor when a snapshot needs to be + * created. The derived actor should respond with its current state. + *

+ * During recovery the state that is returned by the derived actor will + * be passed back to it by calling the applySnapshot method + * + * @return The current state of the actor + */ + protected abstract Object createSnapshot(); + + /** + * This method will be called by the RaftActor during recovery to + * reconstruct the state of the actor. + *

+ * This method may also be called at any other point during normal + * operations when the derived actor is out of sync with it's peers + * and the only way to bring it in sync is by applying a snapshot + * + * @param snapshot A snapshot of the state of the actor + */ + protected abstract void applySnapshot(Object snapshot); + + private RaftActorBehavior switchBehavior(RaftState state) { + if (currentBehavior != null) { + if (currentBehavior.state() == state) { + return currentBehavior; + } + LOG.info("Switching from state " + currentBehavior.state() + " to " + + state); + + try { + currentBehavior.close(); + } catch (Exception e) { + LOG.error(e, + "Failed to close behavior : " + currentBehavior.state()); + } + + } else { + LOG.info("Switching behavior to " + state); + } + RaftActorBehavior behavior = null; + if (state == RaftState.Candidate) { + behavior = new Candidate(context); + } else if (state == RaftState.Follower) { + behavior = new Follower(context); + } else { + behavior = new Leader(context); + } + return behavior; + } + + private void trimPersistentData(long sequenceNumber) { + // Trim snapshots + // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied + // For now guessing that it is ANDed. + deleteSnapshots(new SnapshotSelectionCriteria( + sequenceNumber - 100000, 43200000)); + + // Trim journal + deleteMessages(sequenceNumber); + } + + private class ReplicatedLogImpl implements ReplicatedLog { - private final List journal = new ArrayList(); - private long snapshotIndex = 0; - private Object snapShot = null; + private final List journal; + private final Object snapshot; + private long snapshotIndex = -1; + private long snapshotTerm = -1; + public ReplicatedLogImpl(Snapshot snapshot) { + this.snapshot = snapshot.getState(); + this.snapshotIndex = snapshot.getLastAppliedIndex(); + this.snapshotTerm = snapshot.getLastAppliedTerm(); + + this.journal = new ArrayList<>(snapshot.getUnAppliedEntries()); + } + + public ReplicatedLogImpl() { + this.snapshot = null; + this.journal = new ArrayList<>(); + } @Override public ReplicatedLogEntry get(long index) { - if(index < 0 || index >= journal.size()){ + int adjustedIndex = adjustedIndex(index); + + if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { return null; } - return journal.get((int) (index - snapshotIndex)); + return journal.get(adjustedIndex); } @Override public ReplicatedLogEntry last() { - if(journal.size() == 0){ + if (journal.size() == 0) { return null; } return get(journal.size() - 1); } @Override public long lastIndex() { - if(journal.size() == 0){ + if (journal.size() == 0) { return -1; } @@ -222,7 +381,7 @@ public abstract class RaftActor extends UntypedPersistentActor { } @Override public long lastTerm() { - if(journal.size() == 0){ + if (journal.size() == 0) { return -1; } @@ -231,82 +390,261 @@ public abstract class RaftActor extends UntypedPersistentActor { @Override public void removeFrom(long index) { - if(index < 0 || index >= journal.size()){ + int adjustedIndex = adjustedIndex(index); + + if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { return; } - for(int i= (int) (index - snapshotIndex) ; i < journal.size() ; i++){ - deleteMessage(i); - journal.remove(i); + + journal.subList(adjustedIndex , journal.size()).clear(); + } + + + @Override public void removeFromAndPersist(long index) { + int adjustedIndex = adjustedIndex(index); + + if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { + return; } + + // FIXME: Maybe this should be done after the command is saved + journal.subList(adjustedIndex , journal.size()).clear(); + + persist(new DeleteEntries(adjustedIndex), new Procedure(){ + + @Override public void apply(DeleteEntries param) + throws Exception { + //FIXME : Doing nothing for now + } + }); + + } - @Override public void append(final ReplicatedLogEntry replicatedLogEntry) { + @Override public void append( + final ReplicatedLogEntry replicatedLogEntry) { journal.add(replicatedLogEntry); } @Override public List getFrom(long index) { + int adjustedIndex = adjustedIndex(index); + List entries = new ArrayList<>(100); - if(index < 0 || index >= journal.size()){ + if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { return entries; } - for(int i= (int) (index - snapshotIndex); i < journal.size() ; i++){ + + + for (int i = adjustedIndex; + i < journal.size(); i++) { entries.add(journal.get(i)); } return entries; } - @Override public void appendAndPersist(final ReplicatedLogEntry replicatedLogEntry){ + @Override public void appendAndPersist( + final ReplicatedLogEntry replicatedLogEntry) { appendAndPersist(null, null, replicatedLogEntry); } - public void appendAndPersist(final ActorRef clientActor, final String identifier, final ReplicatedLogEntry replicatedLogEntry){ - context.getLogger().debug("Append log entry and persist" + replicatedLogEntry.getIndex()); + public void appendAndPersist(final ActorRef clientActor, + final String identifier, + final ReplicatedLogEntry replicatedLogEntry) { + context.getLogger().debug( + "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); + + // When persisting events with persist it is guaranteed that the + // persistent actor will not receive further commands between the + // persist call and the execution(s) of the associated event + // handler. This also holds for multiple persist calls in context + // of a single command. 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()); + long lastAppliedIndex = -1; + long lastAppliedTerm = -1; + if (lastAppliedEntry != null) { + lastAppliedIndex = lastAppliedEntry.getIndex(); + lastAppliedTerm = lastAppliedEntry.getTerm(); + } + + saveSnapshot(Snapshot.create(createSnapshot(), + getFrom(context.getLastApplied() + 1), + lastIndex(), lastTerm(), lastAppliedIndex, + lastAppliedTerm)); + } // Send message for replication - if(clientActor != null) { + if (clientActor != null) { currentBehavior.handleMessage(getSelf(), new Replicate(clientActor, identifier, - replicatedLogEntry)); + replicatedLogEntry) + ); } } - }); + } + ); } @Override public long size() { - return journal.size() + snapshotIndex; + return journal.size() + snapshotIndex + 1; + } + + @Override public boolean isPresent(long index) { + int adjustedIndex = adjustedIndex(index); + + if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { + return false; + } + return true; + } + + @Override public boolean isInSnapshot(long index) { + return index <= snapshotIndex; + } + + @Override public Object getSnapshot() { + return snapshot; + } + + @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); + } + } + + + + + private static class DeleteEntries implements Serializable { + private final int fromIndex; + + + public DeleteEntries(int fromIndex) { + this.fromIndex = fromIndex; + } + + public int getFromIndex() { + return fromIndex; } } - private static class ReplicatedLogImplEntry implements ReplicatedLogEntry, - Serializable { - private final long index; - private final long term; - private final Object payload; + 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; + } - public ReplicatedLogImplEntry(long index, long term, Object payload){ - this.index = index; - this.term = term; - this.payload = payload; + 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 Object getData() { - return payload; + public Object getState() { + return state; } - @Override public long getTerm() { - return term; + public List getUnAppliedEntries() { + return unAppliedEntries; } - @Override public long getIndex() { - return index; + public long getLastTerm() { + return lastTerm; + } + + public long getLastAppliedIndex() { + return lastAppliedIndex; + } + + public long getLastAppliedTerm() { + return lastAppliedTerm; } } + private class ElectionTermImpl implements ElectionTerm { + /** + * Identifier of the actor whose election term information this is + */ + private long currentTerm = 0; + private String votedFor = null; + + public long getCurrentTerm() { + return currentTerm; + } + + public String getVotedFor() { + return votedFor; + } + + @Override public void update(long currentTerm, String votedFor) { + LOG.info("Set currentTerm={}, votedFor={}", currentTerm, votedFor); + + 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(){ + + @Override public void apply(UpdateElectionTerm param) + throws Exception { + + } + }); + } + } + + private static class UpdateElectionTerm implements Serializable { + private final long currentTerm; + private final String votedFor; + + public UpdateElectionTerm(long currentTerm, String votedFor) { + this.currentTerm = currentTerm; + this.votedFor = votedFor; + } + + public long getCurrentTerm() { + return currentTerm; + } + + public String getVotedFor() { + return votedFor; + } + } }