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%2Fbehaviors%2FAbstractLeader.java;h=462c94ec8a40736cc005c994b74520d9111c3430;hb=f90815e94e325bb2f80c2f6228a2a8d04a2706ab;hp=d85ac8ef67ded21d6140bb42512ba5fa6eb2165d;hpb=0424501920e4d1c7691116deaaa8d26b213ab579;p=controller.git 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 d85ac8ef67..462c94ec8a 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 @@ -14,16 +14,19 @@ import akka.actor.Cancellable; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +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.Set; +import java.util.Map.Entry; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import org.opendaylight.controller.cluster.raft.ClientRequestTracker; import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl; import org.opendaylight.controller.cluster.raft.FollowerLogInformation; @@ -67,14 +70,22 @@ import scala.concurrent.duration.FiniteDuration; * set commitIndex = N (§5.3, §5.4). */ public abstract class AbstractLeader extends AbstractRaftActorBehavior { - protected final Map followerToLog = new HashMap<>(); - protected final Map mapFollowerToSnapshot = new HashMap<>(); - protected final Set followers; + // The index of the first chunk that is sent when installing a snapshot + public static final int FIRST_CHUNK_INDEX = 1; + + // The index that the follower should respond with if it needs the install snapshot to be reset + public static final int INVALID_CHUNK_INDEX = -1; + + // This would be passed as the hash code of the last chunk when sending the first chunk + public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1; + + private final Map followerToLog; + private final Map mapFollowerToSnapshot = new HashMap<>(); private Cancellable heartbeatSchedule = null; - private List trackerList = new ArrayList<>(); + private final Collection trackerList = new LinkedList<>(); protected final int minReplicationCount; @@ -85,25 +96,22 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { public AbstractLeader(RaftActorContext context) { super(context); - followers = context.getPeerAddresses().keySet(); - - for (String followerId : followers) { + final Builder ftlBuilder = ImmutableMap.builder(); + for (String followerId : context.getPeerAddresses().keySet()) { FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(followerId, - new AtomicLong(context.getCommitIndex()), - new AtomicLong(-1), + context.getCommitIndex(), -1, context.getConfigParams().getElectionTimeOutInterval()); - followerToLog.put(followerId, followerLogInformation); + ftlBuilder.put(followerId, followerLogInformation); } + followerToLog = ftlBuilder.build(); leaderId = context.getId(); - if(LOG.isDebugEnabled()) { - LOG.debug("Election:Leader has following peers: {}", followers); - } + LOG.debug("Election:Leader has following peers: {}", getFollowerIds()); - minReplicationCount = getMajorityVoteCount(followers.size()); + minReplicationCount = getMajorityVoteCount(getFollowerIds().size()); // the isolated Leader peer count will be 1 less than the majority vote count. // this is because the vote count has the self vote counted in it @@ -122,6 +130,15 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { scheduleHeartBeat(new FiniteDuration(0, TimeUnit.SECONDS)); } + /** + * Return an immutable collection of follower identifiers. + * + * @return Collection of follower IDs + */ + protected final Collection getFollowerIds() { + return followerToLog.keySet(); + } + private Optional getSnapshot() { return snapshot; } @@ -188,7 +205,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { int replicatedCount = 1; for (FollowerLogInformation info : followerToLog.values()) { - if (info.getMatchIndex().get() >= N) { + if (info.getMatchIndex() >= N) { replicatedCount++; } } @@ -212,16 +229,21 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return this; } + @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 protected ClientRequestTracker findClientRequestTracker(long logIndex) { for (ClientRequestTracker tracker : trackerList) { if (tracker.getIndex() == logIndex) { @@ -314,8 +336,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { mapFollowerToSnapshot.remove(followerId); if(LOG.isDebugEnabled()) { - LOG.debug("followerToLog.get(followerId).getNextIndex().get()=" + - followerToLog.get(followerId).getNextIndex().get()); + LOG.debug("followerToLog.get(followerId).getNextIndex()=" + + followerToLog.get(followerId).getNextIndex()); } if (mapFollowerToSnapshot.isEmpty()) { @@ -332,6 +354,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { "sending snapshot chunk failed, Will retry, Chunk:{}", reply.getChunkIndex() ); + followerToSnapshot.markSendStatus(false); } @@ -341,6 +364,12 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { " or Chunk Index in InstallSnapshotReply not matching {} != {}", followerToSnapshot.getChunkIndex(), reply.getChunkIndex() ); + + if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){ + // Since the Follower did not find this index to be valid we should reset the follower snapshot + // so that Installing the snapshot can resume from the beginning + followerToSnapshot.reset(); + } } } @@ -359,7 +388,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { logIndex) ); - if (followers.size() == 0) { + if (followerToLog.isEmpty()) { context.setCommitIndex(logIndex); applyLogToStateMachine(logIndex); } else { @@ -369,14 +398,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private void sendAppendEntries() { // Send an AppendEntries to all followers - for (String followerId : followers) { + for (Entry e : followerToLog.entrySet()) { + final String followerId = e.getKey(); ActorSelection followerActor = context.getPeerActorSelection(followerId); if (followerActor != null) { FollowerLogInformation followerLogInformation = followerToLog.get(followerId); - long followerNextIndex = followerLogInformation.getNextIndex().get(); + long followerNextIndex = followerLogInformation.getNextIndex(); boolean isFollowerActive = followerLogInformation.isFollowerActive(); - List entries = null; if (mapFollowerToSnapshot.get(followerId) != null) { // if install snapshot is in process , then sent next chunk if possible @@ -391,6 +420,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } else { long leaderLastIndex = context.getReplicatedLog().lastIndex(); long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex(); + final List entries; if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) { @@ -456,23 +486,19 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { * */ private void installSnapshotIfNeeded() { - for (String followerId : followers) { - ActorSelection followerActor = - context.getPeerActorSelection(followerId); + for (Entry e : followerToLog.entrySet()) { + final ActorSelection followerActor = context.getPeerActorSelection(e.getKey()); - if(followerActor != null) { - FollowerLogInformation followerLogInformation = - followerToLog.get(followerId); - - long nextIndex = followerLogInformation.getNextIndex().get(); + if (followerActor != null) { + long nextIndex = e.getValue().getNextIndex(); if (!context.getReplicatedLog().isPresent(nextIndex) && context.getReplicatedLog().isInSnapshot(nextIndex)) { - LOG.info("{} follower needs a snapshot install", followerId); + LOG.info("{} follower needs a snapshot install", 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, followerId); + sendSnapshotChunk(followerActor, e.getKey()); } else { initiateCaptureSnapshot(); @@ -511,16 +537,15 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private void sendInstallSnapshot() { - for (String followerId : followers) { - ActorSelection followerActor = context.getPeerActorSelection(followerId); + for (Entry e : followerToLog.entrySet()) { + ActorSelection followerActor = context.getPeerActorSelection(e.getKey()); - if(followerActor != null) { - FollowerLogInformation followerLogInformation = followerToLog.get(followerId); - long nextIndex = followerLogInformation.getNextIndex().get(); + if (followerActor != null) { + long nextIndex = e.getValue().getNextIndex(); if (!context.getReplicatedLog().isPresent(nextIndex) && context.getReplicatedLog().isInSnapshot(nextIndex)) { - sendSnapshotChunk(followerActor, followerId); + sendSnapshotChunk(followerActor, e.getKey()); } } } @@ -539,7 +564,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { context.getReplicatedLog().getSnapshotTerm(), getNextSnapshotChunk(followerId,snapshot.get()), mapFollowerToSnapshot.get(followerId).incrementChunkIndex(), - mapFollowerToSnapshot.get(followerId).getTotalChunks() + mapFollowerToSnapshot.get(followerId).getTotalChunks(), + Optional.of(mapFollowerToSnapshot.get(followerId).getLastChunkHashCode()) ).toSerializable(), actor() ); @@ -570,7 +596,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } private void sendHeartBeat() { - if (followers.size() > 0) { + if (!followerToLog.isEmpty()) { sendAppendEntries(); } } @@ -582,7 +608,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } private void scheduleHeartBeat(FiniteDuration interval) { - if(followers.size() == 0){ + if (followerToLog.isEmpty()) { // Optimization - do not bother scheduling a heartbeat as there are // no followers return; @@ -636,11 +662,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private boolean replyStatus = false; private int chunkIndex; private int totalChunks; + private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE; + private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE; public FollowerToSnapshot(ByteString snapshotBytes) { this.snapshotBytes = snapshotBytes; - replyReceivedForOffset = -1; - chunkIndex = 1; int size = snapshotBytes.size(); totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) + ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0); @@ -648,6 +674,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { LOG.debug("Snapshot {} bytes, total chunks to send:{}", size, totalChunks); } + replyReceivedForOffset = -1; + chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX; } public ByteString getSnapshotBytes() { @@ -692,6 +720,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // if the chunk sent was successful replyReceivedForOffset = offset; replyStatus = true; + lastChunkHashCode = nextChunkHashCode; } else { // if the chunk sent was failure replyReceivedForOffset = offset; @@ -715,24 +744,61 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { LOG.debug("length={}, offset={},size={}", snapshotLength, start, size); } - return getSnapshotBytes().substring(start, start + size); + ByteString substring = getSnapshotBytes().substring(start, start + size); + nextChunkHashCode = substring.hashCode(); + return substring; + } + /** + * reset should be called when the Follower needs to be sent the snapshot from the beginning + */ + public void reset(){ + offset = 0; + replyStatus = false; + replyReceivedForOffset = offset; + chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX; + lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE; + } + + public int getLastChunkHashCode() { + return lastChunkHashCode; } } // called from example-actor for printing the follower-states public String printFollowerStates() { - StringBuilder sb = new StringBuilder(); - for(FollowerLogInformation followerLogInformation : followerToLog.values()) { - boolean isFollowerActive = followerLogInformation.isFollowerActive(); - sb.append("{"+followerLogInformation.getId() + " state:" + isFollowerActive + "},"); + final StringBuilder sb = new StringBuilder(); + sb.append('['); + for (FollowerLogInformation followerLogInformation : followerToLog.values()) { + sb.append('{'); + sb.append(followerLogInformation.getId()); + sb.append(" state:"); + sb.append(followerLogInformation.isFollowerActive()); + sb.append("},"); } - return "[" + sb.toString() + "]"; + sb.append(']'); + + return sb.toString(); + } + + @VisibleForTesting + public FollowerLogInformation getFollower(String followerId) { + return followerToLog.get(followerId); + } + + @VisibleForTesting + protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) { + mapFollowerToSnapshot.put(followerId, snapshot); + } + + @VisibleForTesting + public int followerSnapshotSize() { + return mapFollowerToSnapshot.size(); } @VisibleForTesting - void markFollowerActive(String followerId) { - followerToLog.get(followerId).markFollowerActive(); + public int followerLogSize() { + return followerToLog.size(); } }