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=5219ebb3107c1a62f36a3dc8b285be72dad88532;hp=2b94eb312b153867361878222001d932ad048d08;hb=ad860a3b51c31b740aabb297727e15aa45756777;hpb=d066fa07902c7fe2c137616f18dac81fccefe6af 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 2b94eb312b..5219ebb310 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 @@ -16,6 +16,7 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; import java.io.IOException; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -24,6 +25,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Queue; import javax.annotation.Nullable; import org.opendaylight.controller.cluster.raft.ClientRequestTracker; import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl; @@ -33,6 +35,7 @@ import org.opendaylight.controller.cluster.raft.PeerInfo; import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.RaftState; import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry; +import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload; import org.opendaylight.controller.cluster.raft.Snapshot; import org.opendaylight.controller.cluster.raft.VotingState; import org.opendaylight.controller.cluster.raft.base.messages.Replicate; @@ -83,26 +86,27 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private final Map followerToLog = new HashMap<>(); private final Map mapFollowerToSnapshot = new HashMap<>(); - private Cancellable heartbeatSchedule = null; - - private final Collection trackerList = new LinkedList<>(); - - private int minReplicationCount; + /** + * Lookup table for request contexts based on journal index. We could use a {@link Map} here, but we really + * expect the entries to be modified in sequence, hence we open-code the lookup. + * + * TODO: Evaluate the use of ArrayDeque(), as that has lower memory overhead. Non-head removals are more costly, + * but we already expect those to be far from frequent. + */ + private final Queue trackers = new LinkedList<>(); + private Cancellable heartbeatSchedule = null; private Optional snapshot; + private int minReplicationCount; - public AbstractLeader(RaftActorContext context) { - super(context, RaftState.Leader); - - setLeaderPayloadVersion(context.getPayloadVersion()); + protected AbstractLeader(RaftActorContext context, RaftState state) { + super(context, state); for(PeerInfo peerInfo: context.getPeers()) { FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(peerInfo, -1, context); followerToLog.put(peerInfo.getId(), followerLogInformation); } - leaderId = context.getId(); - LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds()); updateMinReplicaCount(); @@ -211,6 +215,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { followerLogInformation.markFollowerActive(); followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion()); + followerLogInformation.setRaftVersion(appendEntriesReply.getRaftVersion()); boolean updated = false; if (appendEntriesReply.isSuccess()) { @@ -251,19 +256,34 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // If there exists an N such that N > commitIndex, a majority // of matchIndex[i] ≥ N, and log[N].term == currentTerm: // set commitIndex = N (§5.3, §5.4). + if(LOG.isTraceEnabled()) { + LOG.trace("{}: handleAppendEntriesReply from {}: commitIndex: {}, lastAppliedIndex: {}, currentTerm: {}", + logName(), followerId, context.getCommitIndex(), context.getLastApplied(), currentTerm()); + } + for (long N = context.getCommitIndex() + 1; ; N++) { int replicatedCount = 1; + LOG.trace("{}: checking Nth index {}", logName(), N); for (FollowerLogInformation info : followerToLog.values()) { final PeerInfo peerInfo = context.getPeerInfo(info.getId()); if(info.getMatchIndex() >= N && (peerInfo != null && peerInfo.isVoting())) { replicatedCount++; + } else if(LOG.isTraceEnabled()) { + LOG.trace("{}: Not counting follower {} - matchIndex: {}, {}", logName(), info.getId(), + info.getMatchIndex(), peerInfo); } } + if(LOG.isTraceEnabled()) { + LOG.trace("{}: replicatedCount {}, minReplicationCount: {}", logName(), replicatedCount, minReplicationCount); + } + if (replicatedCount >= minReplicationCount) { ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N); if (replicatedLogEntry == null) { + LOG.debug("{}: ReplicatedLogEntry not found for index {} - snapshotIndex: {}, journal size: {}", + logName(), N, context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().size()); break; } @@ -273,9 +293,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // reach consensus, as per §5.4.1: "once an entry from the current term is committed by // counting replicas, then all prior entries are committed indirectly". if (replicatedLogEntry.getTerm() == currentTerm()) { + LOG.trace("{}: Setting commit index to {}", logName(), N); context.setCommitIndex(N); + } else { + LOG.debug("{}: Not updating commit index to {} - retrieved log entry with index {}, term {} does not match the current term {}", + logName(), N, replicatedLogEntry.getIndex(), replicatedLogEntry.getTerm(), currentTerm()); } } else { + LOG.trace("{}: minReplicationCount not reached, actual {} - breaking", logName(), replicatedCount); break; } } @@ -327,7 +352,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { @Override protected ClientRequestTracker removeClientRequestTracker(long logIndex) { - final Iterator it = trackerList.iterator(); + final Iterator it = trackers.iterator(); while (it.hasNext()) { final ClientRequestTracker t = it.next(); if (t.getIndex() == logIndex) { @@ -339,16 +364,6 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return null; } - @Override - protected ClientRequestTracker findClientRequestTracker(long logIndex) { - for (ClientRequestTracker tracker : trackerList) { - if (tracker.getIndex() == logIndex) { - return tracker; - } - } - return null; - } - @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, RequestVoteReply requestVoteReply) { @@ -382,23 +397,19 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { beforeSendHeartbeat(); sendHeartBeat(); scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval()); - return this; - } else if(message instanceof SendInstallSnapshot) { // received from RaftActor setSnapshot(((SendInstallSnapshot) message).getSnapshot()); sendInstallSnapshot(); - } else if (message instanceof Replicate) { replicate((Replicate) message); - - } else if (message instanceof InstallSnapshotReply){ + } else if (message instanceof InstallSnapshotReply) { handleInstallSnapshotReply((InstallSnapshotReply) message); - + } else { + return super.handleMessage(sender, message); } - - return super.handleMessage(sender, message); + return this; } private void handleInstallSnapshotReply(InstallSnapshotReply reply) { @@ -499,7 +510,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // Create a tracker entry we will use this later to notify the // client actor - trackerList.add( + trackers.add( new ClientRequestTrackerImpl(replicate.getClientActor(), replicate.getIdentifier(), logIndex) @@ -689,21 +700,28 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private void sendSnapshotChunk(ActorSelection followerActor, String followerId) { try { if (snapshot.isPresent()) { - ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes()); + byte[] nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes()); // Note: the previous call to getNextSnapshotChunk has the side-effect of adding // followerId to the followerToSnapshot map. FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId); + int nextChunkIndex = followerToSnapshot.incrementChunkIndex(); + Optional serverConfig = Optional.absent(); + if(followerToSnapshot.isLastChunk(nextChunkIndex)) { + serverConfig = Optional.fromNullable(context.getPeerServerInfo(true)); + } + followerActor.tell( new InstallSnapshot(currentTerm(), context.getId(), snapshot.get().getLastIncludedIndex(), snapshot.get().getLastIncludedTerm(), nextSnapshotChunk, - followerToSnapshot.incrementChunkIndex(), + nextChunkIndex, followerToSnapshot.getTotalChunks(), - Optional.of(followerToSnapshot.getLastChunkHashCode()) - ).toSerializable(), + Optional.of(followerToSnapshot.getLastChunkHashCode()), + serverConfig + ).toSerializable(followerToLog.get(followerId).getRaftVersion()), actor() ); @@ -722,15 +740,15 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { * Acccepts snaphot as ByteString, enters into map for future chunks * creates and return a ByteString chunk */ - private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException { + private byte[] getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException { FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId); if (followerToSnapshot == null) { followerToSnapshot = new FollowerToSnapshot(snapshotBytes); mapFollowerToSnapshot.put(followerId, followerToSnapshot); } - ByteString nextChunk = followerToSnapshot.getNextChunk(); + byte[] nextChunk = followerToSnapshot.getNextChunk(); - LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size()); + LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.length); return nextChunk; } @@ -763,27 +781,33 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // need to be sent if there are other messages being sent to the remote // actor. heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce( - interval, context.getActor(), new SendHeartBeat(), + interval, context.getActor(), SendHeartBeat.INSTANCE, context.getActorSystem().dispatcher(), context.getActor()); } @Override - public void close() throws Exception { + public void close() { stopHeartBeat(); } @Override - public String getLeaderId() { + public final String getLeaderId() { return context.getId(); } + @Override + public final short getLeaderPayloadVersion() { + return context.getPayloadVersion(); + } + protected boolean isLeaderIsolated() { int minPresent = getMinIsolatedLeaderPeerCount(); for (FollowerLogInformation followerLogInformation : followerToLog.values()) { - if (followerLogInformation.isFollowerActive()) { + final PeerInfo peerInfo = context.getPeerInfo(followerLogInformation.getId()); + if(peerInfo != null && peerInfo.isVoting() && followerLogInformation.isFollowerActive()) { --minPresent; if (minPresent == 0) { - break; + return false; } } } @@ -869,25 +893,23 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } } - public ByteString getNextChunk() { + public byte[] getNextChunk() { int snapshotLength = getSnapshotBytes().size(); int start = incrementOffset(); int size = context.getConfigParams().getSnapshotChunkSize(); if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) { size = snapshotLength; - } else { - if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) { - size = snapshotLength - start; - } + } else if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) { + size = snapshotLength - start; } + byte[] nextChunk = new byte[size]; + getSnapshotBytes().copyTo(nextChunk, start, 0, size); + nextChunkHashCode = Arrays.hashCode(nextChunk); - LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(), - snapshotLength, start, size); - - ByteString substring = getSnapshotBytes().substring(start, start + size); - nextChunkHashCode = substring.hashCode(); - return substring; + LOG.debug("{}: Next chunk: total length={}, offset={}, size={}, hashCode={}", logName(), + snapshotLength, start, size, nextChunkHashCode); + return nextChunk; } /**