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%2Fbehaviors%2FAbstractLeader.java;h=3336cbd9bec1bbc9feaac98610dd8bea85a85a6f;hp=0a31cde90fb2c0139b72ec805aa5e4f83483bcb4;hb=1a6cfbeec045f1f12ddf82b39345f00b3eeb69f8;hpb=989c3723e634bf4a0fb671246c31eadf255c462c diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java index 0a31cde90f..3336cbd9be 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java @@ -18,10 +18,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap.Builder; import com.google.protobuf.ByteString; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -84,7 +85,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private Cancellable heartbeatSchedule = null; - private List trackerList = new ArrayList<>(); + private final Collection trackerList = new LinkedList<>(); protected final int minReplicationCount; @@ -92,6 +93,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private Optional snapshot; + private long replicatedToAllIndex = -1; + public AbstractLeader(RaftActorContext context) { super(context); @@ -108,7 +111,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { leaderId = context.getId(); - LOG.debug("Election:Leader has following peers: {}", getFollowerIds()); + LOG.debug("{}: Election: Leader has following peers: {}", context.getId(), getFollowerIds()); minReplicationCount = getMajorityVoteCount(getFollowerIds().size()); @@ -152,7 +155,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { AppendEntries appendEntries) { if(LOG.isDebugEnabled()) { - LOG.debug(appendEntries.toString()); + LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries); } return this; @@ -164,7 +167,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { if(! appendEntriesReply.isSuccess()) { if(LOG.isDebugEnabled()) { - LOG.debug(appendEntriesReply.toString()); + LOG.debug("{}: handleAppendEntriesReply: {}", context.getId(), appendEntriesReply); } } @@ -174,7 +177,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { followerToLog.get(followerId); if(followerLogInformation == null){ - LOG.error("Unknown follower {}", followerId); + LOG.error("{}: handleAppendEntriesReply - unknown follower {}", context.getId(), followerId); return this; } @@ -225,18 +228,37 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { applyLogToStateMachine(context.getCommitIndex()); } + if (!context.isSnapshotCaptureInitiated()) { + purgeInMemoryLog(); + } + return this; } + private void purgeInMemoryLog() { + //find the lowest index across followers which has been replicated to all. -1 if there are no followers. + // we would delete the in-mem log from that index on, in-order to minimize mem usage + // we would also share this info thru AE with the followers so that they can delete their log entries as well. + long minReplicatedToAllIndex = followerToLog.isEmpty() ? -1 : Long.MAX_VALUE; + for (FollowerLogInformation info : followerToLog.values()) { + minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex()); + } + + replicatedToAllIndex = fakeSnapshot(minReplicatedToAllIndex, replicatedToAllIndex); + } + @Override protected ClientRequestTracker removeClientRequestTracker(long logIndex) { - - ClientRequestTracker toRemove = findClientRequestTracker(logIndex); - if(toRemove != null) { - trackerList.remove(toRemove); + final Iterator it = trackerList.iterator(); + while (it.hasNext()) { + final ClientRequestTracker t = it.next(); + if (t.getIndex() == logIndex) { + it.remove(); + return t; + } } - return toRemove; + return null; } @Override @@ -272,6 +294,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // set currentTerm = T, convert to follower (§5.1) // This applies to all RPC messages and responses if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) { + LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {}", context.getId(), + rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); + context.getTermInformation().updateAndPersist(rpc.getTerm(), null); return switchBehavior(new Follower(context)); @@ -308,19 +333,24 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private void handleInstallSnapshotReply(InstallSnapshotReply reply) { String followerId = reply.getFollowerId(); FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId); + + if (followerToSnapshot == null) { + LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader", + context.getId(), followerId); + return; + } + FollowerLogInformation followerLogInformation = followerToLog.get(followerId); followerLogInformation.markFollowerActive(); - if (followerToSnapshot != null && - followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) { - + if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) { if (reply.isSuccess()) { if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) { //this was the last chunk reply if(LOG.isDebugEnabled()) { - LOG.debug("InstallSnapshotReply received, " + + LOG.debug("{}: InstallSnapshotReply received, " + "last chunk received, Chunk:{}. Follower:{} Setting nextIndex:{}", - reply.getChunkIndex(), followerId, + context.getId(), reply.getChunkIndex(), followerId, context.getReplicatedLog().getSnapshotIndex() + 1 ); } @@ -332,8 +362,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { mapFollowerToSnapshot.remove(followerId); if(LOG.isDebugEnabled()) { - LOG.debug("followerToLog.get(followerId).getNextIndex()=" + - followerToLog.get(followerId).getNextIndex()); + LOG.debug("{}: followerToLog.get(followerId).getNextIndex()=" + + context.getId(), followerToLog.get(followerId).getNextIndex()); } if (mapFollowerToSnapshot.isEmpty()) { @@ -346,20 +376,15 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { followerToSnapshot.markSendStatus(true); } } else { - LOG.info("InstallSnapshotReply received, " + - "sending snapshot chunk failed, Will retry, Chunk:{}", - reply.getChunkIndex() - ); + LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}", + context.getId(), reply.getChunkIndex()); followerToSnapshot.markSendStatus(false); } - } else { - LOG.error("ERROR!!" + - "FollowerId in InstallSnapshotReply not known to Leader" + - " or Chunk Index in InstallSnapshotReply not matching {} != {}", - followerToSnapshot.getChunkIndex(), reply.getChunkIndex() - ); + LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}", + context.getId(), reply.getChunkIndex(), followerId, + followerToSnapshot.getChunkIndex()); if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){ // Since the Follower did not find this index to be valid we should reset the follower snapshot @@ -373,7 +398,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { long logIndex = replicate.getReplicatedLogEntry().getIndex(); if(LOG.isDebugEnabled()) { - LOG.debug("Replicate message {}", logIndex); + LOG.debug("{}: Replicate message {}", context.getId(), logIndex); } // Create a tracker entry we will use this later to notify the @@ -394,6 +419,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private void sendAppendEntries() { // Send an AppendEntries to all followers + for (Entry e : followerToLog.entrySet()) { final String followerId = e.getKey(); ActorSelection followerActor = context.getPeerActorSelection(followerId); @@ -402,24 +428,30 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { FollowerLogInformation followerLogInformation = followerToLog.get(followerId); long followerNextIndex = followerLogInformation.getNextIndex(); boolean isFollowerActive = followerLogInformation.isFollowerActive(); - List entries = null; - if (mapFollowerToSnapshot.get(followerId) != null) { + FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId); + if (followerToSnapshot != null) { // if install snapshot is in process , then sent next chunk if possible - if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) { + if (isFollowerActive && followerToSnapshot.canSendNextChunk()) { sendSnapshotChunk(followerActor, followerId); } else { // we send a heartbeat even if we have not received a reply for the last chunk sendAppendEntriesToFollower(followerActor, followerNextIndex, - Collections.emptyList()); + Collections.emptyList(), followerId); } } else { long leaderLastIndex = context.getReplicatedLog().lastIndex(); long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex(); + final List entries; + + LOG.debug("{}: Checking sendAppendEntries for {}, leaderLastIndex: {}, leaderSnapShotIndex: {}", + context.getId(), leaderLastIndex, leaderSnapShotIndex); + + if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) { + LOG.debug("{}: sendAppendEntries: {} is present for {}", context.getId(), + followerNextIndex, followerId); - if (isFollowerActive && - context.getReplicatedLog().isPresent(followerNextIndex)) { // FIXME : Sending one entry at a time entries = context.getReplicatedLog().getFrom(followerNextIndex, 1); @@ -430,11 +462,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // then snapshot should be sent if(LOG.isDebugEnabled()) { - LOG.debug("InitiateInstallSnapshot to follower:{}," + - "follower-nextIndex:{}, leader-snapshot-index:{}, " + - "leader-last-index:{}", followerId, - followerNextIndex, leaderSnapShotIndex, leaderLastIndex - ); + LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," + + "follower-nextIndex: %s, leader-snapshot-index: %s, " + + "leader-last-index: %s", context.getId(), followerId, + followerNextIndex, leaderSnapShotIndex, leaderLastIndex)); } actor().tell(new InitiateInstallSnapshot(), actor()); @@ -447,22 +478,25 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { entries = Collections.emptyList(); } - sendAppendEntriesToFollower(followerActor, followerNextIndex, entries); - + sendAppendEntriesToFollower(followerActor, followerNextIndex, entries, followerId); } } } } private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex, - List entries) { - followerActor.tell( - new AppendEntries(currentTerm(), context.getId(), - prevLogIndex(followerNextIndex), - prevLogTerm(followerNextIndex), entries, - context.getCommitIndex()).toSerializable(), - actor() - ); + List entries, String followerId) { + AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(), + prevLogIndex(followerNextIndex), + prevLogTerm(followerNextIndex), entries, + context.getCommitIndex(), replicatedToAllIndex); + + if(!entries.isEmpty()) { + LOG.debug("{}: Sending AppendEntries to follower {}: {}", context.getId(), followerId, + appendEntries); + } + + followerActor.tell(appendEntries.toSerializable(), actor()); } /** @@ -482,6 +516,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { * */ private void installSnapshotIfNeeded() { + if(LOG.isDebugEnabled()) { + LOG.debug("{}: installSnapshotIfNeeded, followers {}", context.getId(), followerToLog.keySet()); + } + for (Entry e : followerToLog.entrySet()) { final ActorSelection followerActor = context.getPeerActorSelection(e.getKey()); @@ -489,14 +527,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { long nextIndex = e.getValue().getNextIndex(); if (!context.getReplicatedLog().isPresent(nextIndex) && - context.getReplicatedLog().isInSnapshot(nextIndex)) { - LOG.info("{} follower needs a snapshot install", e.getKey()); + context.getReplicatedLog().isInSnapshot(nextIndex)) { + LOG.info("{}: {} follower needs a snapshot install", context.getId(), e.getKey()); if (snapshot.isPresent()) { // if a snapshot is present in the memory, most likely another install is in progress // no need to capture snapshot sendSnapshotChunk(followerActor, e.getKey()); - } else { + } else if (!context.isSnapshotCaptureInitiated()) { initiateCaptureSnapshot(); //we just need 1 follower who would need snapshot to be installed. // when we have the snapshot captured, we would again check (in SendInstallSnapshot) @@ -512,7 +550,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // on every install snapshot, we try to capture the snapshot. // Once a capture is going on, another one issued will get ignored by RaftActor. private void initiateCaptureSnapshot() { - LOG.info("Initiating Snapshot Capture to Install Snapshot, Leader:{}", getLeaderId()); + LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", context.getId(), getLeaderId()); ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied()); long lastAppliedIndex = -1; long lastAppliedTerm = -1; @@ -529,6 +567,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { actor().tell(new CaptureSnapshot(lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm, isInstallSnapshotInitiated), actor()); + context.setSnapshotCaptureInitiated(true); } @@ -554,23 +593,30 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private void sendSnapshotChunk(ActorSelection followerActor, String followerId) { try { if (snapshot.isPresent()) { + ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId,snapshot.get()); + + // Note: the previous call to getNextSnapshotChunk has the side-effect of adding + // followerId to the followerToSnapshot map. + FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId); + followerActor.tell( new InstallSnapshot(currentTerm(), context.getId(), context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().getSnapshotTerm(), - getNextSnapshotChunk(followerId,snapshot.get()), - mapFollowerToSnapshot.get(followerId).incrementChunkIndex(), - mapFollowerToSnapshot.get(followerId).getTotalChunks(), - Optional.of(mapFollowerToSnapshot.get(followerId).getLastChunkHashCode()) + nextSnapshotChunk, + followerToSnapshot.incrementChunkIndex(), + followerToSnapshot.getTotalChunks(), + Optional.of(followerToSnapshot.getLastChunkHashCode()) ).toSerializable(), actor() ); - LOG.info("InstallSnapshot sent to follower {}, Chunk: {}/{}", - followerActor.path(), mapFollowerToSnapshot.get(followerId).getChunkIndex(), - mapFollowerToSnapshot.get(followerId).getTotalChunks()); + LOG.info("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", + context.getId(), followerActor.path(), + followerToSnapshot.getChunkIndex(), + followerToSnapshot.getTotalChunks()); } } catch (IOException e) { - LOG.error(e, "InstallSnapshot failed for Leader."); + LOG.error(e, "{}: InstallSnapshot failed for Leader.", context.getId()); } } @@ -586,7 +632,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } ByteString nextChunk = followerToSnapshot.getNextChunk(); if (LOG.isDebugEnabled()) { - LOG.debug("Leader's snapshot nextChunk size:{}", nextChunk.size()); + LOG.debug("{}: Leader's snapshot nextChunk size:{}", context.getId(), nextChunk.size()); } return nextChunk; } @@ -650,14 +696,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { * snapshot chunks */ protected class FollowerToSnapshot { - private ByteString snapshotBytes; + private final ByteString snapshotBytes; private int offset = 0; // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset private int replyReceivedForOffset; // if replyStatus is false, the previous chunk is attempted private boolean replyStatus = false; private int chunkIndex; - private int totalChunks; + private final int totalChunks; private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE; private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE; @@ -667,8 +713,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) + ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0); if(LOG.isDebugEnabled()) { - LOG.debug("Snapshot {} bytes, total chunks to send:{}", - size, totalChunks); + LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}", + context.getId(), size, totalChunks); } replyReceivedForOffset = -1; chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX; @@ -737,7 +783,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } if(LOG.isDebugEnabled()) { - LOG.debug("length={}, offset={},size={}", + LOG.debug("{}: Next chunk: length={}, offset={},size={}", context.getId(), snapshotLength, start, size); } ByteString substring = getSnapshotBytes().substring(start, start + size);