From: tpantelis Date: Tue, 10 Feb 2015 06:07:28 +0000 (-0500) Subject: Raft behavior logging X-Git-Tag: release/lithium~544 X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=commitdiff_plain;h=5dd83186d64a4b10214f32ec15f007dae016f5e6 Raft behavior logging Added more logging in the raft behavior classes where appropriate. Reduced logging in some places to trace or removed redundant logging to avoid too much output when debug level is enabled. Removed isDebugEnabled() checks in various places where not needed to improve test line coverage. Also I found it useful to see the behavior's state in the log messages. So I added a logName field and method to AbstractRaftBehavior that is printed in each log message instead of just context.getId(). Also added/modified toString() in a few RaftRPC message classes. Change-Id: Ic4d3c93e25d5caf16319b8433d54cab69b26f5c0 Signed-off-by: tpantelis --- 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 353d0b4a24..854ceb23d0 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 @@ -31,16 +31,13 @@ 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.base.messages.SendInstallSnapshot; import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior; import org.opendaylight.controller.cluster.raft.behaviors.Follower; import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior; import org.opendaylight.controller.cluster.raft.client.messages.FindLeader; import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply; -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 org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -358,13 +355,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot()); } else { - if (!(message instanceof AppendEntriesMessages.AppendEntries) - && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: onReceiveCommand: message: {}", persistenceId(), message.getClass()); - } - } - RaftActorBehavior oldBehavior = currentBehavior; currentBehavior = currentBehavior.handleMessage(getSender(), message); 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 94c38f6108..b2bb127eab 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 @@ -92,7 +92,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private Optional snapshot; public AbstractLeader(RaftActorContext context) { - super(context); + super(context, RaftState.Leader); final Builder ftlBuilder = ImmutableMap.builder(); for (String followerId : context.getPeerAddresses().keySet()) { @@ -107,7 +107,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { leaderId = context.getId(); - LOG.debug("{}: Election: Leader has following peers: {}", context.getId(), getFollowerIds()); + LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds()); minReplicationCount = getMajorityVoteCount(getFollowerIds().size()); @@ -125,7 +125,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // Upon election: send initial empty AppendEntries RPCs // (heartbeat) to each server; repeat during idle periods to // prevent election timeouts (§5.2) - sendAppendEntries(0); + sendAppendEntries(0, false); } /** @@ -137,10 +137,6 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return followerToLog.keySet(); } - private Optional getSnapshot() { - return snapshot; - } - @VisibleForTesting void setSnapshot(Optional snapshot) { this.snapshot = snapshot; @@ -150,9 +146,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries); - } + LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries); return this; } @@ -161,10 +155,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { - if(! appendEntriesReply.isSuccess()) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: handleAppendEntriesReply: {}", context.getId(), appendEntriesReply); - } + if(LOG.isTraceEnabled()) { + LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply); + } else if(LOG.isDebugEnabled() && !appendEntriesReply.isSuccess()) { + LOG.debug("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply); } // Update the FollowerLogInformation @@ -173,7 +167,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { followerToLog.get(followerId); if(followerLogInformation == null){ - LOG.error("{}: handleAppendEntriesReply - unknown follower {}", context.getId(), followerId); + LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId); return this; } @@ -221,6 +215,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // Apply the change to the state machine if (context.getCommitIndex() > context.getLastApplied()) { + LOG.debug("{}: handleAppendEntriesReply: applying to log - commitIndex: {}, lastAppliedIndex: {}", + logName(), context.getCommitIndex(), context.getLastApplied()); + applyLogToStateMachine(context.getCommitIndex()); } @@ -229,7 +226,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event - sendUpdatesToFollower(followerId, followerLogInformation, false); + sendUpdatesToFollower(followerId, followerLogInformation, false, false); return this; } @@ -276,11 +273,6 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return this; } - @Override - public RaftState state() { - return RaftState.Leader; - } - @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { Preconditions.checkNotNull(sender, "sender should not be null"); @@ -293,8 +285,8 @@ 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()); + LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower", + logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); context.getTermInformation().updateAndPersist(rpc.getTerm(), null); @@ -327,12 +319,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } private void handleInstallSnapshotReply(InstallSnapshotReply reply) { + LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), 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); + logName(), followerId); return; } @@ -346,8 +340,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { //this was the last chunk reply if(LOG.isDebugEnabled()) { LOG.debug("{}: InstallSnapshotReply received, " + - "last chunk received, Chunk:{}. Follower:{} Setting nextIndex:{}", - context.getId(), reply.getChunkIndex(), followerId, + "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}", + logName(), reply.getChunkIndex(), followerId, context.getReplicatedLog().getSnapshotIndex() + 1 ); } @@ -358,10 +352,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { context.getReplicatedLog().getSnapshotIndex() + 1); mapFollowerToSnapshot.remove(followerId); - if(LOG.isDebugEnabled()) { - LOG.debug("{}: followerToLog.get(followerId).getNextIndex()=" + - context.getId(), followerToLog.get(followerId).getNextIndex()); - } + LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}", + logName(), followerId, followerLogInformation.getMatchIndex(), + followerLogInformation.getNextIndex()); if (mapFollowerToSnapshot.isEmpty()) { // once there are no pending followers receiving snapshots @@ -375,7 +368,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } } else { LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}", - context.getId(), reply.getChunkIndex()); + logName(), reply.getChunkIndex()); followerToSnapshot.markSendStatus(false); } @@ -389,7 +382,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } else { LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}", - context.getId(), reply.getChunkIndex(), followerId, + logName(), reply.getChunkIndex(), followerId, followerToSnapshot.getChunkIndex()); if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){ @@ -403,9 +396,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private void replicate(Replicate replicate) { long logIndex = replicate.getReplicatedLogEntry().getIndex(); - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Replicate message {}", context.getId(), logIndex); - } + LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(), + replicate.getIdentifier(), logIndex); // Create a tracker entry we will use this later to notify the // client actor @@ -419,11 +411,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { context.setCommitIndex(logIndex); applyLogToStateMachine(logIndex); } else { - sendAppendEntries(0); + sendAppendEntries(0, false); } } - private void sendAppendEntries(long timeSinceLastActivityInterval) { + private void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) { // Send an AppendEntries to all followers for (Entry e : followerToLog.entrySet()) { final String followerId = e.getKey(); @@ -431,7 +423,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // This checks helps not to send a repeat message to the follower if(!followerLogInformation.isFollowerActive() || followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) { - sendUpdatesToFollower(followerId, followerLogInformation, true); + sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat); } } } @@ -444,7 +436,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { */ private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation, - boolean sendHeartbeat) { + boolean sendHeartbeat, boolean isHeartbeat) { ActorSelection followerActor = context.getPeerActorSelection(followerId); if (followerActor != null) { @@ -463,8 +455,17 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } else { long leaderLastIndex = context.getReplicatedLog().lastIndex(); long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex(); - if (isFollowerActive && - context.getReplicatedLog().isPresent(followerNextIndex)) { + + if(!isHeartbeat || LOG.isTraceEnabled()) { + LOG.debug("{}: Checking sendAppendEntries for follower {}, leaderLastIndex: {}, leaderSnapShotIndex: {}", + logName(), followerId, leaderLastIndex, leaderSnapShotIndex); + } + + if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) { + + LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(), + followerNextIndex, followerId); + // FIXME : Sending one entry at a time final List entries = context.getReplicatedLog().getFrom(followerNextIndex, 1); @@ -477,11 +478,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: %d, leader-snapshot-index: %d, " + + "leader-last-index: %d", logName(), followerId, + followerNextIndex, leaderSnapShotIndex, leaderLastIndex)); } // Send heartbeat to follower whenever install snapshot is initiated. @@ -508,8 +508,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { prevLogTerm(followerNextIndex), entries, context.getCommitIndex(), super.getReplicatedToAllIndex()); - if(!entries.isEmpty()) { - LOG.debug("{}: Sending AppendEntries to follower {}: {}", context.getId(), followerId, + if(!entries.isEmpty() || LOG.isTraceEnabled()) { + LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId, appendEntries); } @@ -543,7 +543,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } else if (!context.isSnapshotCaptureInitiated()) { - LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", context.getId(), getLeaderId()); + LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", logName(), getLeaderId()); ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied()); long lastAppliedIndex = -1; long lastAppliedTerm = -1; @@ -570,6 +570,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private void sendInstallSnapshot() { + LOG.debug("{}: sendInstallSnapshot", logName()); for (Entry e : followerToLog.entrySet()) { ActorSelection followerActor = context.getPeerActorSelection(e.getKey()); @@ -609,12 +610,12 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { actor() ); LOG.info("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", - context.getId(), followerActor.path(), + logName(), followerActor.path(), followerToSnapshot.getChunkIndex(), followerToSnapshot.getTotalChunks()); } } catch (IOException e) { - LOG.error("{}: InstallSnapshot failed for Leader.", context.getId(), e); + LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e); } } @@ -629,15 +630,16 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { mapFollowerToSnapshot.put(followerId, followerToSnapshot); } ByteString nextChunk = followerToSnapshot.getNextChunk(); - if (LOG.isDebugEnabled()) { - LOG.debug("{}: Leader's snapshot nextChunk size:{}", context.getId(), nextChunk.size()); - } + + LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size()); + return nextChunk; } private void sendHeartBeat() { if (!followerToLog.isEmpty()) { - sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis()); + LOG.trace("{}: Sending heartbeat", logName()); + sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true); } } @@ -712,7 +714,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0); if(LOG.isDebugEnabled()) { LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}", - context.getId(), size, totalChunks); + logName(), size, totalChunks); } replyReceivedForOffset = -1; chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX; @@ -780,10 +782,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } } - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Next chunk: length={}, offset={},size={}", context.getId(), + + LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(), snapshotLength, start, size); - } + ByteString substring = getSnapshotBytes().substring(start, start + size); nextChunkHashCode = substring.hashCode(); return substring; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java index 8f433d529a..0b0b4c7cd6 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java @@ -14,6 +14,7 @@ import java.util.Random; import java.util.concurrent.TimeUnit; import org.opendaylight.controller.cluster.raft.ClientRequestTracker; 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.SerializationUtils; import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries; @@ -60,9 +61,25 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { private long replicatedToAllIndex = -1; - protected AbstractRaftActorBehavior(RaftActorContext context) { + private final String logName; + + private final RaftState state; + + protected AbstractRaftActorBehavior(RaftActorContext context, RaftState state) { this.context = context; + this.state = state; this.LOG = context.getLogger(); + + logName = String.format("%s (%s)", context.getId(), state); + } + + @Override + public RaftState state() { + return state; + } + + public String logName() { + return logName; } @Override @@ -106,7 +123,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { if (appendEntries.getTerm() < currentTerm()) { if(LOG.isDebugEnabled()) { LOG.debug("{}: Cannot append entries because sender term {} is less than {}", - context.getId(), appendEntries.getTerm(), currentTerm()); + logName(), appendEntries.getTerm(), currentTerm()); } sender.tell( @@ -143,12 +160,9 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { * @param requestVote * @return */ - protected RaftActorBehavior requestVote(ActorRef sender, - RequestVote requestVote) { + protected RaftActorBehavior requestVote(ActorRef sender, RequestVote requestVote) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Received {}", context.getId(), requestVote); - } + LOG.debug("{}: In requestVote: {}", logName(), requestVote); boolean grantVote = false; @@ -184,7 +198,11 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { } } - sender.tell(new RequestVoteReply(currentTerm(), grantVote), actor()); + RequestVoteReply reply = new RequestVoteReply(currentTerm(), grantVote); + + LOG.debug("{}: requestVote returning: {}", logName(), reply); + + sender.tell(reply, actor()); return this; } @@ -362,12 +380,12 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { // around as the rest wont be present either LOG.warn( "{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}", - context.getId(), i, i, index); + logName(), i, i, index); break; } } if(LOG.isDebugEnabled()) { - LOG.debug("{}: Setting last applied to {}", context.getId(), newLastApplied); + LOG.debug("{}: Setting last applied to {}", logName(), newLastApplied); } context.setLastApplied(newLastApplied); @@ -401,11 +419,11 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { } protected RaftActorBehavior switchBehavior(RaftActorBehavior behavior) { - LOG.info("{} :- Switching from behavior {} to {}", context.getId(), this.state(), behavior.state()); + LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), behavior.state()); try { close(); } catch (Exception e) { - LOG.error("{}: Failed to close behavior : {}", context.getId(), this.state(), e); + LOG.error("{}: Failed to close behavior : {}", logName(), this.state(), e); } return behavior; @@ -447,6 +465,9 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior { long tempMin = Math.min(snapshotCapturedIndex, (lastApplied > -1 ? lastApplied - 1 : -1)); if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin)) { + LOG.debug("{}: fakeSnapshot purging log to {} for term {}", logName(), tempMin, + context.getTermInformation().getCurrentTerm()); + //use the term of the temp-min, since we check for isPresent, entry will not be null ReplicatedLogEntry entry = context.getReplicatedLog().get(tempMin); context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm()); diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java index 09ffe056c3..b36c41abf2 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java @@ -47,12 +47,12 @@ public class Candidate extends AbstractRaftActorBehavior { private final Set peers; public Candidate(RaftActorContext context) { - super(context); + super(context, RaftState.Candidate); peers = context.getPeerAddresses().keySet(); if(LOG.isDebugEnabled()) { - LOG.debug("{}: Election: Candidate has following peers: {}", context.getId(), peers); + LOG.debug("{}: Election: Candidate has following peers: {}", logName(), peers); } votesRequired = getMajorityVoteCount(peers.size()); @@ -65,7 +65,7 @@ public class Candidate extends AbstractRaftActorBehavior { AppendEntries appendEntries) { if(LOG.isDebugEnabled()) { - LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries); + LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries); } return this; @@ -78,7 +78,10 @@ public class Candidate extends AbstractRaftActorBehavior { } @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, - RequestVoteReply requestVoteReply) { + RequestVoteReply requestVoteReply) { + + LOG.debug("{}: handleRequestVoteReply: {}, current voteCount: {}", logName(), requestVoteReply, + voteCount); if (requestVoteReply.isVoteGranted()) { voteCount++; @@ -91,10 +94,6 @@ public class Candidate extends AbstractRaftActorBehavior { return this; } - @Override public RaftState state() { - return RaftState.Candidate; - } - @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { @@ -105,7 +104,7 @@ public class Candidate extends AbstractRaftActorBehavior { RaftRPC rpc = (RaftRPC) message; if(LOG.isDebugEnabled()) { - LOG.debug("{}: RaftRPC message received {} my term is {}", context.getId(), rpc, + LOG.debug("{}: RaftRPC message received {}, my term is {}", logName(), rpc, context.getTermInformation().getCurrentTerm()); } @@ -120,6 +119,8 @@ public class Candidate extends AbstractRaftActorBehavior { } if (message instanceof ElectionTimeout) { + LOG.debug("{}: Received ElectionTimeout", logName()); + if (votesRequired == 0) { // If there are no peers then we should be a Leader // We wait for the election timeout to occur before declare @@ -146,12 +147,10 @@ public class Candidate extends AbstractRaftActorBehavior { // Increment the election term and vote for self long currentTerm = context.getTermInformation().getCurrentTerm(); - context.getTermInformation().updateAndPersist(currentTerm + 1, - context.getId()); + long newTerm = currentTerm + 1; + context.getTermInformation().updateAndPersist(newTerm, context.getId()); - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Starting new term {}", context.getId(), (currentTerm + 1)); - } + LOG.debug("{}: Starting new term {}", logName(), newTerm); // Request for a vote // TODO: Retry request for vote if replies do not arrive in a reasonable @@ -159,17 +158,17 @@ public class Candidate extends AbstractRaftActorBehavior { for (String peerId : peers) { ActorSelection peerActor = context.getPeerActorSelection(peerId); if(peerActor != null) { - peerActor.tell(new RequestVote( + RequestVote requestVote = new RequestVote( context.getTermInformation().getCurrentTerm(), context.getId(), context.getReplicatedLog().lastIndex(), - context.getReplicatedLog().lastTerm()), - context.getActor() - ); - } - } + context.getReplicatedLog().lastTerm()); + LOG.debug("{}: Sending {} to peer {}", logName(), requestVote, peerId); + peerActor.tell(requestVote, context.getActor()); + } + } } @Override public void close() throws Exception { diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java index 675543f2d1..1e4fcf7225 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java @@ -40,7 +40,7 @@ public class Follower extends AbstractRaftActorBehavior { private SnapshotTracker snapshotTracker = null; public Follower(RaftActorContext context) { - super(context); + super(context, RaftState.Follower); scheduleElection(electionDuration()); } @@ -75,10 +75,11 @@ public class Follower extends AbstractRaftActorBehavior { @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) { - if(appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries); - } + int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0; + if(LOG.isTraceEnabled()) { + LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries); + } else if(LOG.isDebugEnabled() && numLogEntries > 0) { + LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries); } // TODO : Refactor this method into a bunch of smaller methods @@ -101,44 +102,31 @@ public class Follower extends AbstractRaftActorBehavior { boolean outOfSync = true; // First check if the logs are in sync or not - if (lastIndex() == -1 - && appendEntries.getPrevLogIndex() != -1) { + long lastIndex = lastIndex(); + if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) { // The follower's log is out of sync because the leader does have // an entry at prevLogIndex and this follower has no entries in // it's log. - if(LOG.isDebugEnabled()) { - LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}", - context.getId(), appendEntries.getPrevLogIndex()); - } - - } else if (lastIndex() > -1 - && appendEntries.getPrevLogIndex() != -1 - && !prevEntryPresent) { + LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}", + logName(), appendEntries.getPrevLogIndex()); + } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) { // The follower's log is out of sync because the Leader's // prevLogIndex entry was not found in it's log - if(LOG.isDebugEnabled()) { - LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it", - context.getId(), appendEntries.getPrevLogIndex()); - } - - } else if (lastIndex() > -1 - && prevEntryPresent - && prevLogTerm != appendEntries.getPrevLogTerm()) { + LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it", + logName(), appendEntries.getPrevLogIndex()); + } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) { // The follower's log is out of sync because the Leader's // prevLogIndex entry does exist in the follower's log but it has // a different term in it - if (LOG.isDebugEnabled()) { - LOG.debug( - "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}" - , context.getId(), prevLogTerm - , appendEntries.getPrevLogTerm()); - } + LOG.debug( + "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}", + logName(), prevLogTerm, appendEntries.getPrevLogTerm()); } else { outOfSync = false; } @@ -146,25 +134,19 @@ public class Follower extends AbstractRaftActorBehavior { if (outOfSync) { // We found that the log was out of sync so just send a negative // reply and return - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Follower ({}) is out-of-sync, " + - "so sending negative reply, lastIndex():{}, lastTerm():{}", - context.getId(), context.getId(), lastIndex(), lastTerm() - ); - } - sender.tell( - new AppendEntriesReply(context.getId(), currentTerm(), false, - lastIndex(), lastTerm()), actor() - ); + + LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}", + logName(), lastIndex, lastTerm()); + + sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, + lastTerm()), actor()); return this; } - if (appendEntries.getEntries() != null - && appendEntries.getEntries().size() > 0) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Number of entries to be appended = {}", context.getId(), + if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) { + + LOG.debug("{}: Number of entries to be appended = {}", logName(), appendEntries.getEntries().size()); - } // 3. If an existing entry conflicts with a new one (same index // but different terms), delete the existing entry and all that @@ -182,77 +164,72 @@ public class Follower extends AbstractRaftActorBehavior { break; } - if (newEntry.getTerm() == matchEntry - .getTerm()) { + if (newEntry.getTerm() == matchEntry.getTerm()) { continue; } - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Removing entries from log starting at {}", context.getId(), + LOG.debug("{}: Removing entries from log starting at {}", logName(), matchEntry.getIndex()); - } // Entries do not match so remove all subsequent entries - context.getReplicatedLog() - .removeFromAndPersist(matchEntry.getIndex()); + context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex()); break; } } - if(LOG.isDebugEnabled()) { - LOG.debug("{}: After cleanup entries to be added from = {}", context.getId(), - (addEntriesFrom + lastIndex())); - } + lastIndex = lastIndex(); + LOG.debug("{}: After cleanup entries to be added from = {}", logName(), + (addEntriesFrom + lastIndex)); // 4. Append any new entries not already in the log - for (int i = addEntriesFrom; - i < appendEntries.getEntries().size(); i++) { + for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) { + ReplicatedLogEntry entry = appendEntries.getEntries().get(i); - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Append entry to log {}", context.getId(), - appendEntries.getEntries().get(i).getData()); - } - context.getReplicatedLog().appendAndPersist(appendEntries.getEntries().get(i)); - } + LOG.debug("{}: Append entry to log {}", logName(), entry.getData()); - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Log size is now {}", context.getId(), context.getReplicatedLog().size()); + context.getReplicatedLog().appendAndPersist(entry); } - } + LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); + } // 5. If leaderCommit > commitIndex, set commitIndex = // min(leaderCommit, index of last new entry) + lastIndex = lastIndex(); long prevCommitIndex = context.getCommitIndex(); - context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), - context.getReplicatedLog().lastIndex())); + context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex)); if (prevCommitIndex != context.getCommitIndex()) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Commit index set to {}", context.getId(), context.getCommitIndex()); - } + LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); } // If commitIndex > lastApplied: increment lastApplied, apply // log[lastApplied] to state machine (§5.3) // check if there are any entries to be applied. last-applied can be equal to last-index if (appendEntries.getLeaderCommit() > context.getLastApplied() && - context.getLastApplied() < lastIndex()) { + context.getLastApplied() < lastIndex) { if(LOG.isDebugEnabled()) { LOG.debug("{}: applyLogToStateMachine, " + - "appendEntries.getLeaderCommit():{}," + - "context.getLastApplied():{}, lastIndex():{}", context.getId(), - appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex() - ); + "appendEntries.getLeaderCommit(): {}," + + "context.getLastApplied(): {}, lastIndex(): {}", logName(), + appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex); } applyLogToStateMachine(appendEntries.getLeaderCommit()); } - sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), true, - lastIndex(), lastTerm()), actor()); + AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, + lastIndex, lastTerm()); + + if(LOG.isTraceEnabled()) { + LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply); + } else if(LOG.isDebugEnabled() && numLogEntries > 0) { + LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply); + } + + sender.tell(reply, actor()); if (!context.isSnapshotCaptureInitiated()) { super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex()); @@ -271,10 +248,6 @@ public class Follower extends AbstractRaftActorBehavior { return this; } - @Override public RaftState state() { - return RaftState.Follower; - } - @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { Object message = fromSerializableMessage(originalMessage); @@ -285,11 +258,15 @@ public class Follower 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 follower's term {} - updating term", + logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); + context.getTermInformation().updateAndPersist(rpc.getTerm(), null); } } if (message instanceof ElectionTimeout) { + LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); return switchBehavior(new Candidate(context)); } else if (message instanceof InstallSnapshot) { @@ -304,12 +281,10 @@ public class Follower extends AbstractRaftActorBehavior { private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: InstallSnapshot received by follower " + - "datasize:{} , Chunk:{}/{}", context.getId(), installSnapshot.getData().size(), - installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks() - ); - } + + LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}", + logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(), + installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks()); if(snapshotTracker == null){ snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks()); @@ -331,18 +306,23 @@ public class Follower extends AbstractRaftActorBehavior { } - sender.tell(new InstallSnapshotReply( - currentTerm(), context.getId(), installSnapshot.getChunkIndex(), - true), actor()); + InstallSnapshotReply reply = new InstallSnapshotReply( + currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true); + + LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); + + sender.tell(reply, actor()); } catch (SnapshotTracker.InvalidChunkException e) { + LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e); sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor()); snapshotTracker = null; } catch (Exception e){ - LOG.error("{}: Exception in InstallSnapshot of follower", context.getId(), e); + LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e); + //send reply with success as false. The chunk will be sent again on failure sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), installSnapshot.getChunkIndex(), false), actor()); @@ -358,6 +338,4 @@ public class Follower extends AbstractRaftActorBehavior { ByteString getSnapshotChunksCollected(){ return snapshotTracker != null ? snapshotTracker.getCollectedChunks() : ByteString.EMPTY; } - - } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java index 97bcd6a708..d2ea0c50cd 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java @@ -110,19 +110,15 @@ public class AppendEntries extends AbstractRaftRPC { return replicatedToAllIndex; } + @Override public String toString() { - final StringBuilder sb = - new StringBuilder("AppendEntries{"); - sb.append("term=").append(getTerm()); - sb.append("leaderId='").append(leaderId).append('\''); - sb.append(", prevLogIndex=").append(prevLogIndex); - sb.append(", prevLogTerm=").append(prevLogTerm); - sb.append(", entries=").append(entries); - sb.append(", leaderCommit=").append(leaderCommit); - sb.append(", replicatedToAllIndex=").append(replicatedToAllIndex); - sb.append('}'); - return sb.toString(); + StringBuilder builder = new StringBuilder(); + builder.append("AppendEntries [term=").append(term).append(", leaderId=").append(leaderId) + .append(", prevLogIndex=").append(prevLogIndex).append(", prevLogTerm=").append(prevLogTerm) + .append(", entries=").append(entries).append(", leaderCommit=").append(leaderCommit) + .append(", replicatedToAllIndex=").append(replicatedToAllIndex).append("]"); + return builder.toString(); } public Object toSerializable() { diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java index 32ed85b281..01fef006a9 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java @@ -59,15 +59,12 @@ public class AppendEntriesReply extends AbstractRaftRPC { return followerId; } - @Override public String toString() { - final StringBuilder sb = - new StringBuilder("AppendEntriesReply{"); - sb.append("term=").append(term); - sb.append(", success=").append(success); - sb.append(", logLastIndex=").append(logLastIndex); - sb.append(", logLastTerm=").append(logLastTerm); - sb.append(", followerId='").append(followerId).append('\''); - sb.append('}'); - return sb.toString(); + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("AppendEntriesReply [term=").append(term).append(", success=").append(success) + .append(", logLastIndex=").append(logLastIndex).append(", logLastTerm=").append(logLastTerm) + .append(", followerId=").append(followerId).append("]"); + return builder.toString(); } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java index 6337f8f6dc..119b43ce83 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java @@ -102,4 +102,15 @@ public class InstallSnapshot extends AbstractRaftRPC { return installSnapshot; } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("InstallSnapshot [term=").append(term).append(", leaderId=").append(leaderId) + .append(", lastIncludedIndex=").append(lastIncludedIndex).append(", lastIncludedTerm=") + .append(lastIncludedTerm).append(", data=").append(data).append(", chunkIndex=").append(chunkIndex) + .append(", totalChunks=").append(totalChunks).append(", lastChunkHashCode=").append(lastChunkHashCode) + .append("]"); + return builder.toString(); + } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java index 15621bf894..77efa53dfd 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java @@ -36,4 +36,12 @@ public class InstallSnapshotReply extends AbstractRaftRPC { public boolean isSuccess() { return success; } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("InstallSnapshotReply [term=").append(term).append(", followerId=").append(followerId) + .append(", chunkIndex=").append(chunkIndex).append(", success=").append(success).append("]"); + return builder.toString(); + } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java index 9ba5acb664..8f162ae254 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java @@ -64,14 +64,12 @@ public class RequestVote extends AbstractRaftRPC { this.lastLogTerm = lastLogTerm; } - @Override public String toString() { - final StringBuilder sb = - new StringBuilder("RequestVote{"); - sb.append("term='").append(getTerm()).append('\''); - sb.append("candidateId='").append(candidateId).append('\''); - sb.append(", lastLogIndex=").append(lastLogIndex); - sb.append(", lastLogTerm=").append(lastLogTerm); - sb.append('}'); - return sb.toString(); + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("RequestVote [term=").append(term).append(", candidateId=").append(candidateId) + .append(", lastLogIndex=").append(lastLogIndex).append(", lastLogTerm=").append(lastLogTerm) + .append("]"); + return builder.toString(); } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java index b3c95d6eca..865d4c287b 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java @@ -27,4 +27,11 @@ public class RequestVoteReply extends AbstractRaftRPC { public boolean isVoteGranted() { return voteGranted; } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("RequestVoteReply [term=").append(term).append(", voteGranted=").append(voteGranted).append("]"); + return builder.toString(); + } } diff --git a/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/cluster/common/actor/AbstractUntypedPersistentActor.java b/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/cluster/common/actor/AbstractUntypedPersistentActor.java index 95ee21674a..432c2d5615 100644 --- a/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/cluster/common/actor/AbstractUntypedPersistentActor.java +++ b/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/cluster/common/actor/AbstractUntypedPersistentActor.java @@ -20,8 +20,8 @@ public abstract class AbstractUntypedPersistentActor extends UntypedPersistentAc protected final Logger LOG = LoggerFactory.getLogger(getClass()); public AbstractUntypedPersistentActor() { - if(LOG.isDebugEnabled()) { - LOG.debug("Actor created {}", getSelf()); + if(LOG.isTraceEnabled()) { + LOG.trace("Actor created {}", getSelf()); } getContext(). system(). @@ -33,24 +33,24 @@ public abstract class AbstractUntypedPersistentActor extends UntypedPersistentAc @Override public void onReceiveCommand(Object message) throws Exception { final String messageType = message.getClass().getSimpleName(); - if(LOG.isDebugEnabled()) { - LOG.debug("Received message {}", messageType); + if(LOG.isTraceEnabled()) { + LOG.trace("Received message {}", messageType); } handleCommand(message); - if(LOG.isDebugEnabled()) { - LOG.debug("Done handling message {}", messageType); + if(LOG.isTraceEnabled()) { + LOG.trace("Done handling message {}", messageType); } } @Override public void onReceiveRecover(Object message) throws Exception { final String messageType = message.getClass().getSimpleName(); - if(LOG.isDebugEnabled()) { - LOG.debug("Received message {}", messageType); + if(LOG.isTraceEnabled()) { + LOG.trace("Received message {}", messageType); } handleRecover(message); - if(LOG.isDebugEnabled()) { - LOG.debug("Done handling message {}", messageType); + if(LOG.isTraceEnabled()) { + LOG.trace("Done handling message {}", messageType); } } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java index 87a0fb931e..21d74a6e1a 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java @@ -229,10 +229,6 @@ public class Shard extends RaftActor { @Override public void onReceiveCommand(final Object message) throws Exception { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: onReceiveCommand: Received message {} from {}", persistenceId(), message, getSender()); - } - if (message.getClass().equals(CreateTransaction.SERIALIZABLE_CLASS)) { handleCreateTransaction(message); } else if(message instanceof ForwardedReadyTransaction) {