From: Tom Pantelis Date: Wed, 27 Jan 2016 07:33:32 +0000 (-0500) Subject: Deprecate InstallSnapshot protobuff messages X-Git-Tag: release/boron~377 X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=commitdiff_plain;h=252ba03242407ee584c38fafdbfa1c322e66151d;hp=384479f0181763f3202f2f7ad681e90182bcc820 Deprecate InstallSnapshot protobuff messages Deprecated the associated InstallSnapshot protobuff message and changed InstallSnapshot to Externalizable. Backwards compatibility with pre-boron is maintained. Related code was modified accordingly. Previously InstallSnapshot took a ByteString. I changed this to byte[] to avoid the extra copy overhead with ByteString with converting to and from byte[]. Change-Id: I532d062983e76e63c685e6df1d48e0ae38197a5c Signed-off-by: Tom Pantelis --- diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/SerializationUtils.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/SerializationUtils.java index 9dd46e6f43..3e5d3fa071 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/SerializationUtils.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/SerializationUtils.java @@ -13,7 +13,7 @@ import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot; public class SerializationUtils { public static Object fromSerializable(Object serializable){ - if (serializable.getClass().equals(InstallSnapshot.SERIALIZABLE_CLASS)) { + if (InstallSnapshot.isSerializedType(serializable)) { return InstallSnapshot.fromSerializable(serializable); } return serializable; 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 7f3589c82f..fd78431d16 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; @@ -690,7 +691,7 @@ 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. @@ -704,7 +705,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { followerToSnapshot.incrementChunkIndex(), followerToSnapshot.getTotalChunks(), Optional.of(followerToSnapshot.getLastChunkHashCode()) - ).toSerializable(), + ).toSerializable(followerToLog.get(followerId).getRaftVersion()), actor() ); @@ -723,15 +724,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; } @@ -870,25 +871,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; } /** 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 ec1642ec2a..bcc2480f45 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 @@ -353,9 +353,7 @@ public class Follower extends AbstractRaftActorBehavior { private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) { - LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}", - logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(), - installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks()); + LOG.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot); leaderId = installSnapshot.getLeaderId(); diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/SnapshotTracker.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/SnapshotTracker.java index d26837f180..bb7a242481 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/SnapshotTracker.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/SnapshotTracker.java @@ -10,6 +10,7 @@ package org.opendaylight.controller.cluster.raft.behaviors; import com.google.common.base.Optional; import com.google.protobuf.ByteString; +import java.util.Arrays; import org.slf4j.Logger; /** @@ -36,7 +37,10 @@ public class SnapshotTracker { * @return true when the lastChunk is received * @throws InvalidChunkException */ - boolean addChunk(int chunkIndex, ByteString chunk, Optional lastChunkHashCode) throws InvalidChunkException{ + boolean addChunk(int chunkIndex, byte[] chunk, Optional lastChunkHashCode) throws InvalidChunkException{ + LOG.debug("addChunk: chunkIndex={}, lastChunkIndex={}, collectedChunks.size={}, lastChunkHashCode={}", + chunkIndex, lastChunkIndex, collectedChunks.size(), this.lastChunkHashCode); + if(sealed){ throw new InvalidChunkException("Invalid chunk received with chunkIndex " + chunkIndex + " all chunks already received"); } @@ -48,19 +52,14 @@ public class SnapshotTracker { if(lastChunkHashCode.isPresent()){ if(lastChunkHashCode.get() != this.lastChunkHashCode){ throw new InvalidChunkException("The hash code of the recorded last chunk does not match " + - "the senders hash code expected " + lastChunkHashCode + " was " + lastChunkHashCode.get()); + "the senders hash code, expected " + this.lastChunkHashCode + " was " + lastChunkHashCode.get()); } } - if(LOG.isDebugEnabled()) { - LOG.debug("Chunk={},collectedChunks.size:{}", - chunkIndex, collectedChunks.size()); - } - sealed = (chunkIndex == totalChunks); lastChunkIndex = chunkIndex; - collectedChunks = collectedChunks.concat(chunk); - this.lastChunkHashCode = chunk.hashCode(); + collectedChunks = collectedChunks.concat(ByteString.copyFrom(chunk)); + this.lastChunkHashCode = Arrays.hashCode(chunk); return sealed; } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AbstractRaftRPC.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AbstractRaftRPC.java index bb34389720..0aabb49f6b 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AbstractRaftRPC.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AbstractRaftRPC.java @@ -27,7 +27,7 @@ public class AbstractRaftRPC implements RaftRPC { return term; } - public void setTerm(long term) { + protected void setTerm(long term) { this.term = term; } } 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 9757a0b8bd..f2f9cd39d8 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 @@ -10,23 +10,33 @@ package org.opendaylight.controller.cluster.raft.messages; import com.google.common.base.Optional; import com.google.protobuf.ByteString; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.opendaylight.controller.cluster.raft.RaftVersions; import org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages; -public class InstallSnapshot extends AbstractRaftRPC { - - public static final Class SERIALIZABLE_CLASS = InstallSnapshotMessages.InstallSnapshot.class; +public class InstallSnapshot extends AbstractRaftRPC implements Externalizable { private static final long serialVersionUID = 1L; + public static final Class SERIALIZABLE_CLASS = InstallSnapshotMessages.InstallSnapshot.class; - private final String leaderId; - private final long lastIncludedIndex; - private final long lastIncludedTerm; - private final ByteString data; - private final int chunkIndex; - private final int totalChunks; - private final Optional lastChunkHashCode; + private String leaderId; + private long lastIncludedIndex; + private long lastIncludedTerm; + private byte[] data; + private int chunkIndex; + private int totalChunks; + private Optional lastChunkHashCode; + + /** + * Empty constructor to satisfy Externalizable. + */ + public InstallSnapshot() { + } public InstallSnapshot(long term, String leaderId, long lastIncludedIndex, - long lastIncludedTerm, ByteString data, int chunkIndex, int totalChunks, Optional lastChunkHashCode) { + long lastIncludedTerm, byte[] data, int chunkIndex, int totalChunks, Optional lastChunkHashCode) { super(term); this.leaderId = leaderId; this.lastIncludedIndex = lastIncludedIndex; @@ -38,11 +48,10 @@ public class InstallSnapshot extends AbstractRaftRPC { } public InstallSnapshot(long term, String leaderId, long lastIncludedIndex, - long lastIncludedTerm, ByteString data, int chunkIndex, int totalChunks) { + long lastIncludedTerm, byte[] data, int chunkIndex, int totalChunks) { this(term, leaderId, lastIncludedIndex, lastIncludedTerm, data, chunkIndex, totalChunks, Optional.absent()); } - public String getLeaderId() { return leaderId; } @@ -55,7 +64,7 @@ public class InstallSnapshot extends AbstractRaftRPC { return lastIncludedTerm; } - public ByteString getData() { + public byte[] getData() { return data; } @@ -71,47 +80,92 @@ public class InstallSnapshot extends AbstractRaftRPC { return lastChunkHashCode; } - public Object toSerializable(){ - InstallSnapshotMessages.InstallSnapshot.Builder builder = InstallSnapshotMessages.InstallSnapshot.newBuilder() - .setTerm(this.getTerm()) - .setLeaderId(this.getLeaderId()) - .setChunkIndex(this.getChunkIndex()) - .setData(this.getData()) - .setLastIncludedIndex(this.getLastIncludedIndex()) - .setLastIncludedTerm(this.getLastIncludedTerm()) - .setTotalChunks(this.getTotalChunks()); - - if(lastChunkHashCode.isPresent()){ - builder.setLastChunkHashCode(lastChunkHashCode.get()); + @Override + public void writeExternal(ObjectOutput out) throws IOException { + out.writeShort(RaftVersions.CURRENT_VERSION); + out.writeLong(getTerm()); + out.writeUTF(leaderId); + out.writeLong(lastIncludedIndex); + out.writeLong(lastIncludedTerm); + out.writeInt(chunkIndex); + out.writeInt(totalChunks); + + out.writeByte(lastChunkHashCode.isPresent() ? 1 : 0); + if(lastChunkHashCode.isPresent()) { + out.writeInt(lastChunkHashCode.get().intValue()); } - return builder.build(); - } - public static InstallSnapshot fromSerializable (Object o) { - InstallSnapshotMessages.InstallSnapshot from = - (InstallSnapshotMessages.InstallSnapshot) o; + out.writeObject(data); + } - Optional lastChunkHashCode = Optional.absent(); - if(from.hasLastChunkHashCode()){ - lastChunkHashCode = Optional.of(from.getLastChunkHashCode()); + @Override + public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + in.readShort(); // raft version - not currently used + setTerm(in.readLong()); + leaderId = in.readUTF(); + lastIncludedIndex = in.readLong(); + lastIncludedTerm = in.readLong(); + chunkIndex = in.readInt(); + totalChunks = in.readInt(); + + lastChunkHashCode = Optional.absent(); + boolean chunkHashCodePresent = in.readByte() == 1; + if(chunkHashCodePresent) { + lastChunkHashCode = Optional.of(in.readInt()); } - InstallSnapshot installSnapshot = new InstallSnapshot(from.getTerm(), - from.getLeaderId(), from.getLastIncludedIndex(), - from.getLastIncludedTerm(), from.getData(), - from.getChunkIndex(), from.getTotalChunks(), lastChunkHashCode); + data = (byte[])in.readObject(); + } - return installSnapshot; + public Object toSerializable(short version) { + if(version >= RaftVersions.BORON_VERSION) { + return this; + } else { + InstallSnapshotMessages.InstallSnapshot.Builder builder = InstallSnapshotMessages.InstallSnapshot.newBuilder() + .setTerm(this.getTerm()) + .setLeaderId(this.getLeaderId()) + .setChunkIndex(this.getChunkIndex()) + .setData(ByteString.copyFrom(getData())) + .setLastIncludedIndex(this.getLastIncludedIndex()) + .setLastIncludedTerm(this.getLastIncludedTerm()) + .setTotalChunks(this.getTotalChunks()); + + if(lastChunkHashCode.isPresent()){ + builder.setLastChunkHashCode(lastChunkHashCode.get()); + } + return builder.build(); + } } @Override public String toString() { - StringBuilder builder = new StringBuilder(); - builder.append("InstallSnapshot [term=").append(getTerm()).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(); + return "InstallSnapshot [term=" + getTerm() + ", leaderId=" + leaderId + ", lastIncludedIndex=" + + lastIncludedIndex + ", lastIncludedTerm=" + lastIncludedTerm + ", datasize=" + data.length + + ", Chunk=" + chunkIndex + "/" + totalChunks + ", lastChunkHashCode=" + lastChunkHashCode + "]"; + } + + public static InstallSnapshot fromSerializable (Object o) { + if(o instanceof InstallSnapshot) { + return (InstallSnapshot)o; + } else { + InstallSnapshotMessages.InstallSnapshot from = + (InstallSnapshotMessages.InstallSnapshot) o; + + Optional lastChunkHashCode = Optional.absent(); + if(from.hasLastChunkHashCode()){ + lastChunkHashCode = Optional.of(from.getLastChunkHashCode()); + } + + InstallSnapshot installSnapshot = new InstallSnapshot(from.getTerm(), + from.getLeaderId(), from.getLastIncludedIndex(), + from.getLastIncludedTerm(), from.getData().toByteArray(), + from.getChunkIndex(), from.getTotalChunks(), lastChunkHashCode); + + return installSnapshot; + } + } + + public static boolean isSerializedType(Object message) { + return message instanceof InstallSnapshot || message instanceof InstallSnapshotMessages.InstallSnapshot; } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/AbstractRaftActorIntegrationTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/AbstractRaftActorIntegrationTest.java index bf55fa7aca..8536528c74 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/AbstractRaftActorIntegrationTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/AbstractRaftActorIntegrationTest.java @@ -158,6 +158,8 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest } } + protected static final int SNAPSHOT_CHUNK_SIZE = 100; + protected final Logger testLog = LoggerFactory.getLogger(getClass()); protected final TestActorFactory factory = new TestActorFactory(getSystem()); @@ -204,6 +206,7 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest configParams.setSnapshotBatchCount(snapshotBatchCount); configParams.setSnapshotDataThresholdPercentage(70); configParams.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS)); + configParams.setSnapshotChunkSize(SNAPSHOT_CHUNK_SIZE); return configParams; } diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java index 89d69886ed..03387abcd5 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java @@ -498,27 +498,52 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A List unAppliedEntry; ApplySnapshot applySnapshot; InstallSnapshot installSnapshot; - InstallSnapshotReply installSnapshotReply; testLog.info("testInstallSnapshotToLaggingFollower starting"); + MessageCollectorActor.clearMessages(leaderCollectorActor); + // Now stop dropping AppendEntries in follower 2. follower2Actor.underlyingActor().stopDropMessages(AppendEntries.class); + + MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class); + + // Verify the leader's persisted snapshot. The previous snapshot (currently) won't be deleted from + // the snapshot store because the second snapshot was initiated by the follower install snapshot and + // not because the batch count was reached so the persisted journal sequence number wasn't advanced + // far enough to cause the previous snapshot to be deleted. This is because + // RaftActor#trimPersistentData subtracts the snapshotBatchCount from the snapshot's sequence number. + // This is OK - the next snapshot should delete it. In production, even if the system restarted + // before another snapshot, they would both get applied which wouldn't hurt anything. + persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class); + Assert.assertTrue("Expected at least 1 persisted snapshots", persistedSnapshots.size() > 0); + Snapshot persistedSnapshot = persistedSnapshots.get(persistedSnapshots.size() - 1); + verifySnapshot("Persisted", persistedSnapshot, currentTerm, lastAppliedIndex, currentTerm, lastAppliedIndex); + unAppliedEntry = persistedSnapshot.getUnAppliedEntries(); + assertEquals("Persisted Snapshot getUnAppliedEntries size", 0, unAppliedEntry.size()); + + int snapshotSize = persistedSnapshot.getState().length; + int expTotalChunks = (snapshotSize / SNAPSHOT_CHUNK_SIZE) + ((snapshotSize % SNAPSHOT_CHUNK_SIZE) > 0 ? 1 : 0); + installSnapshot = MessageCollectorActor.expectFirstMatching(follower2CollectorActor, InstallSnapshot.class); assertEquals("InstallSnapshot getTerm", currentTerm, installSnapshot.getTerm()); assertEquals("InstallSnapshot getLeaderId", leaderId, installSnapshot.getLeaderId()); assertEquals("InstallSnapshot getChunkIndex", 1, installSnapshot.getChunkIndex()); - assertEquals("InstallSnapshot getTotalChunks", 1, installSnapshot.getTotalChunks()); + assertEquals("InstallSnapshot getTotalChunks", expTotalChunks, installSnapshot.getTotalChunks()); assertEquals("InstallSnapshot getLastIncludedTerm", currentTerm, installSnapshot.getLastIncludedTerm()); assertEquals("InstallSnapshot getLastIncludedIndex", lastAppliedIndex, installSnapshot.getLastIncludedIndex()); //assertArrayEquals("InstallSnapshot getData", snapshot, installSnapshot.getData().toByteArray()); - installSnapshotReply = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, InstallSnapshotReply.class); - assertEquals("InstallSnapshotReply getTerm", currentTerm, installSnapshotReply.getTerm()); - assertEquals("InstallSnapshotReply getChunkIndex", 1, installSnapshotReply.getChunkIndex()); - assertEquals("InstallSnapshotReply getFollowerId", follower2Id, installSnapshotReply.getFollowerId()); - assertEquals("InstallSnapshotReply isSuccess", true, installSnapshotReply.isSuccess()); + List installSnapshotReplies = MessageCollectorActor.expectMatching( + leaderCollectorActor, InstallSnapshotReply.class, expTotalChunks); + int index = 1; + for(InstallSnapshotReply installSnapshotReply: installSnapshotReplies) { + assertEquals("InstallSnapshotReply getTerm", currentTerm, installSnapshotReply.getTerm()); + assertEquals("InstallSnapshotReply getChunkIndex", index++, installSnapshotReply.getChunkIndex()); + assertEquals("InstallSnapshotReply getFollowerId", follower2Id, installSnapshotReply.getFollowerId()); + assertEquals("InstallSnapshotReply isSuccess", true, installSnapshotReply.isSuccess()); + } // Verify follower 2 applies the snapshot. applySnapshot = MessageCollectorActor.expectFirstMatching(follower2CollectorActor, ApplySnapshot.class); @@ -536,20 +561,6 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A // the log. In addition replicatedToAllIndex should've advanced. verifyLeadersTrimmedLog(lastAppliedIndex); - // Verify the leader's persisted snapshot. The previous snapshot (currently) won't be deleted from - // the snapshot store because the second snapshot was initiated by the follower install snapshot and - // not because the batch count was reached so the persisted journal sequence number wasn't advanced - // far enough to cause the previous snapshot to be deleted. This is because - // RaftActor#trimPersistentData subtracts the snapshotBatchCount from the snapshot's sequence number. - // This is OK - the next snapshot should delete it. In production, even if the system restarted - // before another snapshot, they would both get applied which wouldn't hurt anything. - persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class); - Assert.assertTrue("Expected at least 1 persisted snapshots", persistedSnapshots.size() > 0); - Snapshot persistedSnapshot = persistedSnapshots.get(persistedSnapshots.size() - 1); - verifySnapshot("Persisted", persistedSnapshot, currentTerm, lastAppliedIndex, currentTerm, lastAppliedIndex); - unAppliedEntry = persistedSnapshot.getUnAppliedEntries(); - assertEquals("Persisted Snapshot getUnAppliedEntries size", 0, unAppliedEntry.size()); - MessageCollectorActor.clearMessages(leaderCollectorActor); MessageCollectorActor.clearMessages(follower1CollectorActor); MessageCollectorActor.clearMessages(follower2CollectorActor); diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java index 1b15ecb135..537101e8dc 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java @@ -763,7 +763,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { InstallSnapshot lastInstallSnapshot = null; for(int i = 0; i < totalChunks; i++) { - ByteString chunkData = getNextChunk(bsSnapshot, offset, chunkSize); + byte[] chunkData = getNextChunk(bsSnapshot, offset, chunkSize); lastInstallSnapshot = new InstallSnapshot(1, "leader", lastIncludedIndex, 1, chunkData, chunkIndex, totalChunks); follower.handleMessage(leaderActor, lastInstallSnapshot); @@ -830,7 +830,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { assertTrue(totalChunks > 1); // Send an install snapshot with the first chunk to start the process of installing a snapshot - ByteString chunkData = getNextChunk(bsSnapshot, 0, chunkSize); + byte[] chunkData = getNextChunk(bsSnapshot, 0, chunkSize); follower.handleMessage(leaderActor, new InstallSnapshot(1, "leader", lastIncludedIndex, 1, chunkData, 1, totalChunks)); @@ -871,7 +871,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { InstallSnapshot lastInstallSnapshot = null; for(int i = 0; i < totalChunks; i++) { - ByteString chunkData = getNextChunk(bsSnapshot, offset, chunkSize); + byte[] chunkData = getNextChunk(bsSnapshot, offset, chunkSize); lastInstallSnapshot = new InstallSnapshot(1, "leader", lastIncludedIndex, 1, chunkData, chunkIndex, totalChunks); follower.handleMessage(leaderActor, lastInstallSnapshot); @@ -983,7 +983,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { assertEquals("schedule election", 0, getElectionTimeoutCount(follower)); } - public ByteString getNextChunk (ByteString bs, int offset, int chunkSize){ + public byte[] getNextChunk (ByteString bs, int offset, int chunkSize){ int snapshotLength = bs.size(); int start = offset; int size = chunkSize; @@ -994,7 +994,10 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { size = snapshotLength - start; } } - return bs.substring(start, start + size); + + byte[] nextChunk = new byte[size]; + bs.copyTo(nextChunk, start, 0, size); + return nextChunk; } private void expectAndVerifyAppendEntriesReply(int expTerm, boolean expSuccess, diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java index 8529e1926b..479ba8fbc5 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java @@ -23,6 +23,7 @@ import akka.testkit.TestActorRef; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; import com.google.protobuf.ByteString; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -1126,7 +1127,7 @@ public class LeaderTest extends AbstractLeaderTest { assertEquals(3, installSnapshot.getTotalChunks()); assertEquals(AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE, installSnapshot.getLastChunkHashCode().get().intValue()); - int hashCode = installSnapshot.getData().hashCode(); + int hashCode = Arrays.hashCode(installSnapshot.getData()); followerActor.underlyingActor().clear(); @@ -1177,8 +1178,8 @@ public class LeaderTest extends AbstractLeaderTest { j = barray.length; } - ByteString chunk = fts.getNextChunk(); - assertEquals("bytestring size not matching for chunk:"+ chunkIndex, j-i, chunk.size()); + byte[] chunk = fts.getNextChunk(); + assertEquals("bytestring size not matching for chunk:"+ chunkIndex, j-i, chunk.length); assertEquals("chunkindex not matching", chunkIndex, fts.getChunkIndex()); fts.markSendStatus(true); diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/SnapshotTrackerTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/SnapshotTrackerTest.java index 80348263c8..c7d98b083c 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/SnapshotTrackerTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/SnapshotTrackerTest.java @@ -14,6 +14,7 @@ import com.google.protobuf.ByteString; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.ObjectOutputStream; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import org.junit.Assert; @@ -28,9 +29,9 @@ public class SnapshotTrackerTest { Map data; ByteString byteString; - ByteString chunk1; - ByteString chunk2; - ByteString chunk3; + byte[] chunk1; + byte[] chunk2; + byte[] chunk3; @Before public void setup(){ @@ -128,9 +129,9 @@ public class SnapshotTrackerTest { SnapshotTracker tracker2 = new SnapshotTracker(logger, 3); - tracker2.addChunk(1, chunk1, Optional.absent()); - tracker2.addChunk(2, chunk2, Optional.absent()); - tracker2.addChunk(3, chunk3, Optional.absent()); + tracker2.addChunk(1, chunk1, Optional.of(AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE)); + tracker2.addChunk(2, chunk2, Optional.of(Arrays.hashCode(chunk1))); + tracker2.addChunk(3, chunk3, Optional.of(Arrays.hashCode(chunk2))); byte[] snapshot = tracker2.getSnapshot(); @@ -141,15 +142,15 @@ public class SnapshotTrackerTest { public void testGetCollectedChunks() throws SnapshotTracker.InvalidChunkException { SnapshotTracker tracker1 = new SnapshotTracker(logger, 5); - ByteString chunks = chunk1.concat(chunk2); + ByteString chunks = ByteString.copyFrom(chunk1).concat(ByteString.copyFrom(chunk2)); - tracker1.addChunk(1, chunk1, Optional.absent()); - tracker1.addChunk(2, chunk2, Optional.absent()); + tracker1.addChunk(1, chunk1, Optional.of(AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE)); + tracker1.addChunk(2, chunk2, Optional.of(Arrays.hashCode(chunk1))); assertEquals(chunks, tracker1.getCollectedChunks()); } - public ByteString getNextChunk (ByteString bs, int offset, int size){ + public byte[] getNextChunk (ByteString bs, int offset, int size){ int snapshotLength = bs.size(); int start = offset; if (size > snapshotLength) { @@ -159,7 +160,10 @@ public class SnapshotTrackerTest { size = snapshotLength - start; } } - return bs.substring(start, start + size); + + byte[] nextChunk = new byte[size]; + bs.copyTo(nextChunk, start, 0, size); + return nextChunk; } private static ByteString toByteString(Map state) { diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotTest.java new file mode 100644 index 0000000000..b2d518e1ed --- /dev/null +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotTest.java @@ -0,0 +1,87 @@ +/* + * Copyright (c) 2015 Brocade Communications Systems, Inc. and others. All rights reserved. + * + * This program and the accompanying materials are made available under the + * terms of the Eclipse Public License v1.0 which accompanies this distribution, + * and is available at http://www.eclipse.org/legal/epl-v10.html + */ +package org.opendaylight.controller.cluster.raft.messages; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import com.google.common.base.Optional; +import java.io.Serializable; +import org.apache.commons.lang.SerializationUtils; +import org.junit.Test; +import org.opendaylight.controller.cluster.raft.RaftVersions; +import org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages; + +/** + * Unit tests for InstallSnapshot. + * + * @author Thomas Pantelis + */ +public class InstallSnapshotTest { + + @Test + public void testSerialization() { + byte[] data = new byte[1000]; + int j = 0; + for(int i = 0; i < data.length; i++) { + data[i] = (byte)j; + if(++j >= 255) { + j = 0; + } + } + + InstallSnapshot expected = new InstallSnapshot(3L, "leaderId", 11L, 2L, data, 5, 6, Optional.of(54321)); + + Object serialized = expected.toSerializable(RaftVersions.CURRENT_VERSION); + assertEquals("Serialized type", InstallSnapshot.class, serialized.getClass()); + + InstallSnapshot actual = InstallSnapshot.fromSerializable(SerializationUtils.clone((Serializable) serialized)); + verifyInstallSnapshot(expected, actual); + + expected = new InstallSnapshot(3L, "leaderId", 11L, 2L, data, 5, 6); + actual = InstallSnapshot.fromSerializable(SerializationUtils.clone( + (Serializable) expected.toSerializable(RaftVersions.CURRENT_VERSION))); + verifyInstallSnapshot(expected, actual); + } + + @Test + public void testSerializationWithPreBoronVersion() { + byte[] data = {0,1,2,3,4,5,7,8,9}; + InstallSnapshot expected = new InstallSnapshot(3L, "leaderId", 11L, 2L, data, 5, 6, Optional.of(54321)); + + Object serialized = expected.toSerializable(RaftVersions.LITHIUM_VERSION); + assertEquals("Serialized type", InstallSnapshot.SERIALIZABLE_CLASS, serialized.getClass()); + + InstallSnapshot actual = InstallSnapshot.fromSerializable(SerializationUtils.clone((Serializable) serialized)); + verifyInstallSnapshot(expected, actual); + } + + @Test + public void testIsSerializedType() { + assertEquals("isSerializedType", true, InstallSnapshot.isSerializedType( + InstallSnapshotMessages.InstallSnapshot.newBuilder().build())); + assertEquals("isSerializedType", true, InstallSnapshot.isSerializedType(new InstallSnapshot())); + assertEquals("isSerializedType", false, InstallSnapshot.isSerializedType(new Object())); + } + + private void verifyInstallSnapshot(InstallSnapshot expected, InstallSnapshot actual) { + assertEquals("getTerm", expected.getTerm(), actual.getTerm()); + assertEquals("getChunkIndex", expected.getChunkIndex(), actual.getChunkIndex()); + assertEquals("getTotalChunks", expected.getTotalChunks(), actual.getTotalChunks()); + assertEquals("getLastIncludedTerm", expected.getLastIncludedTerm(), actual.getLastIncludedTerm()); + assertEquals("getLastIncludedIndex", expected.getLastIncludedIndex(), actual.getLastIncludedIndex()); + assertEquals("getLeaderId", expected.getLeaderId(), actual.getLeaderId()); + assertEquals("getChunkIndex", expected.getChunkIndex(), actual.getChunkIndex()); + assertArrayEquals("getData", expected.getData(), actual.getData()); + assertEquals("getLastChunkHashCode present", expected.getLastChunkHashCode().isPresent(), + actual.getLastChunkHashCode().isPresent()); + if(expected.getLastChunkHashCode().isPresent()) { + assertEquals("getLastChunkHashCode", expected.getLastChunkHashCode().get(), + actual.getLastChunkHashCode().get()); + } + } +} diff --git a/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/protobuff/messages/cluster/raft/InstallSnapshotMessages.java b/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/protobuff/messages/cluster/raft/InstallSnapshotMessages.java index de7f44e213..03c381ab3e 100644 --- a/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/protobuff/messages/cluster/raft/InstallSnapshotMessages.java +++ b/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/protobuff/messages/cluster/raft/InstallSnapshotMessages.java @@ -3,6 +3,7 @@ package org.opendaylight.controller.protobuff.messages.cluster.raft; +@Deprecated public final class InstallSnapshotMessages { private InstallSnapshotMessages() {} public static void registerAllExtensions( @@ -114,6 +115,7 @@ public final class InstallSnapshotMessages { return defaultInstance; } + @Override public InstallSnapshot getDefaultInstanceForType() { return defaultInstance; } @@ -204,6 +206,7 @@ public final class InstallSnapshotMessages { return org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_descriptor; } + @Override protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { return org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_fieldAccessorTable @@ -213,7 +216,8 @@ public final class InstallSnapshotMessages { public static com.google.protobuf.Parser PARSER = new com.google.protobuf.AbstractParser() { - public InstallSnapshot parsePartialFrom( + @Override + public InstallSnapshot parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { @@ -233,12 +237,14 @@ public final class InstallSnapshotMessages { /** * optional int64 term = 1; */ + @Override public boolean hasTerm() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** * optional int64 term = 1; */ + @Override public long getTerm() { return term_; } @@ -249,12 +255,14 @@ public final class InstallSnapshotMessages { /** * optional string leaderId = 2; */ + @Override public boolean hasLeaderId() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** * optional string leaderId = 2; */ + @Override public java.lang.String getLeaderId() { java.lang.Object ref = leaderId_; if (ref instanceof java.lang.String) { @@ -272,6 +280,7 @@ public final class InstallSnapshotMessages { /** * optional string leaderId = 2; */ + @Override public com.google.protobuf.ByteString getLeaderIdBytes() { java.lang.Object ref = leaderId_; @@ -292,12 +301,14 @@ public final class InstallSnapshotMessages { /** * optional int64 lastIncludedIndex = 3; */ + @Override public boolean hasLastIncludedIndex() { return ((bitField0_ & 0x00000004) == 0x00000004); } /** * optional int64 lastIncludedIndex = 3; */ + @Override public long getLastIncludedIndex() { return lastIncludedIndex_; } @@ -308,12 +319,14 @@ public final class InstallSnapshotMessages { /** * optional int64 lastIncludedTerm = 4; */ + @Override public boolean hasLastIncludedTerm() { return ((bitField0_ & 0x00000008) == 0x00000008); } /** * optional int64 lastIncludedTerm = 4; */ + @Override public long getLastIncludedTerm() { return lastIncludedTerm_; } @@ -324,12 +337,14 @@ public final class InstallSnapshotMessages { /** * optional bytes data = 5; */ + @Override public boolean hasData() { return ((bitField0_ & 0x00000010) == 0x00000010); } /** * optional bytes data = 5; */ + @Override public com.google.protobuf.ByteString getData() { return data_; } @@ -340,12 +355,14 @@ public final class InstallSnapshotMessages { /** * optional int32 chunkIndex = 6; */ + @Override public boolean hasChunkIndex() { return ((bitField0_ & 0x00000020) == 0x00000020); } /** * optional int32 chunkIndex = 6; */ + @Override public int getChunkIndex() { return chunkIndex_; } @@ -356,12 +373,14 @@ public final class InstallSnapshotMessages { /** * optional int32 totalChunks = 7; */ + @Override public boolean hasTotalChunks() { return ((bitField0_ & 0x00000040) == 0x00000040); } /** * optional int32 totalChunks = 7; */ + @Override public int getTotalChunks() { return totalChunks_; } @@ -372,12 +391,14 @@ public final class InstallSnapshotMessages { /** * optional int32 lastChunkHashCode = 8; */ + @Override public boolean hasLastChunkHashCode() { return ((bitField0_ & 0x00000080) == 0x00000080); } /** * optional int32 lastChunkHashCode = 8; */ + @Override public int getLastChunkHashCode() { return lastChunkHashCode_; } @@ -393,14 +414,18 @@ public final class InstallSnapshotMessages { lastChunkHashCode_ = 0; } private byte memoizedIsInitialized = -1; + @Override public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; + if (isInitialized != -1) { + return isInitialized == 1; + } memoizedIsInitialized = 1; return true; } + @Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -432,9 +457,12 @@ public final class InstallSnapshotMessages { } private int memoizedSerializedSize = -1; + @Override public int getSerializedSize() { int size = memoizedSerializedSize; - if (size != -1) return size; + if (size != -1) { + return size; + } size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { @@ -535,10 +563,12 @@ public final class InstallSnapshotMessages { } public static Builder newBuilder() { return Builder.create(); } + @Override public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot prototype) { return newBuilder().mergeFrom(prototype); } + @Override public Builder toBuilder() { return newBuilder(this); } @java.lang.Override @@ -558,7 +588,8 @@ public final class InstallSnapshotMessages { return org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_descriptor; } - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + @Override + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { return org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -583,7 +614,8 @@ public final class InstallSnapshotMessages { return new Builder(); } - public Builder clear() { + @Override + public Builder clear() { super.clear(); term_ = 0L; bitField0_ = (bitField0_ & ~0x00000001); @@ -604,20 +636,24 @@ public final class InstallSnapshotMessages { return this; } - public Builder clone() { + @Override + public Builder clone() { return create().mergeFrom(buildPartial()); } - public com.google.protobuf.Descriptors.Descriptor + @Override + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_descriptor; } - public org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot getDefaultInstanceForType() { + @Override + public org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot getDefaultInstanceForType() { return org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot.getDefaultInstance(); } - public org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot build() { + @Override + public org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot build() { org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); @@ -625,7 +661,8 @@ public final class InstallSnapshotMessages { return result; } - public org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot buildPartial() { + @Override + public org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot buildPartial() { org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot result = new org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; @@ -666,7 +703,8 @@ public final class InstallSnapshotMessages { return result; } - public Builder mergeFrom(com.google.protobuf.Message other) { + @Override + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot) { return mergeFrom((org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot)other); } else { @@ -676,7 +714,9 @@ public final class InstallSnapshotMessages { } public Builder mergeFrom(org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot other) { - if (other == org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot.getDefaultInstance()) return this; + if (other == org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot.getDefaultInstance()) { + return this; + } if (other.hasTerm()) { setTerm(other.getTerm()); } @@ -707,11 +747,13 @@ public final class InstallSnapshotMessages { return this; } - public final boolean isInitialized() { + @Override + public final boolean isInitialized() { return true; } - public Builder mergeFrom( + @Override + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -735,13 +777,15 @@ public final class InstallSnapshotMessages { /** * optional int64 term = 1; */ - public boolean hasTerm() { + @Override + public boolean hasTerm() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** * optional int64 term = 1; */ - public long getTerm() { + @Override + public long getTerm() { return term_; } /** @@ -768,13 +812,15 @@ public final class InstallSnapshotMessages { /** * optional string leaderId = 2; */ - public boolean hasLeaderId() { + @Override + public boolean hasLeaderId() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** * optional string leaderId = 2; */ - public java.lang.String getLeaderId() { + @Override + public java.lang.String getLeaderId() { java.lang.Object ref = leaderId_; if (!(ref instanceof java.lang.String)) { java.lang.String s = ((com.google.protobuf.ByteString) ref) @@ -788,7 +834,8 @@ public final class InstallSnapshotMessages { /** * optional string leaderId = 2; */ - public com.google.protobuf.ByteString + @Override + public com.google.protobuf.ByteString getLeaderIdBytes() { java.lang.Object ref = leaderId_; if (ref instanceof String) { @@ -842,13 +889,15 @@ public final class InstallSnapshotMessages { /** * optional int64 lastIncludedIndex = 3; */ - public boolean hasLastIncludedIndex() { + @Override + public boolean hasLastIncludedIndex() { return ((bitField0_ & 0x00000004) == 0x00000004); } /** * optional int64 lastIncludedIndex = 3; */ - public long getLastIncludedIndex() { + @Override + public long getLastIncludedIndex() { return lastIncludedIndex_; } /** @@ -875,13 +924,15 @@ public final class InstallSnapshotMessages { /** * optional int64 lastIncludedTerm = 4; */ - public boolean hasLastIncludedTerm() { + @Override + public boolean hasLastIncludedTerm() { return ((bitField0_ & 0x00000008) == 0x00000008); } /** * optional int64 lastIncludedTerm = 4; */ - public long getLastIncludedTerm() { + @Override + public long getLastIncludedTerm() { return lastIncludedTerm_; } /** @@ -908,13 +959,15 @@ public final class InstallSnapshotMessages { /** * optional bytes data = 5; */ - public boolean hasData() { + @Override + public boolean hasData() { return ((bitField0_ & 0x00000010) == 0x00000010); } /** * optional bytes data = 5; */ - public com.google.protobuf.ByteString getData() { + @Override + public com.google.protobuf.ByteString getData() { return data_; } /** @@ -944,13 +997,15 @@ public final class InstallSnapshotMessages { /** * optional int32 chunkIndex = 6; */ - public boolean hasChunkIndex() { + @Override + public boolean hasChunkIndex() { return ((bitField0_ & 0x00000020) == 0x00000020); } /** * optional int32 chunkIndex = 6; */ - public int getChunkIndex() { + @Override + public int getChunkIndex() { return chunkIndex_; } /** @@ -977,13 +1032,15 @@ public final class InstallSnapshotMessages { /** * optional int32 totalChunks = 7; */ - public boolean hasTotalChunks() { + @Override + public boolean hasTotalChunks() { return ((bitField0_ & 0x00000040) == 0x00000040); } /** * optional int32 totalChunks = 7; */ - public int getTotalChunks() { + @Override + public int getTotalChunks() { return totalChunks_; } /** @@ -1010,13 +1067,15 @@ public final class InstallSnapshotMessages { /** * optional int32 lastChunkHashCode = 8; */ - public boolean hasLastChunkHashCode() { + @Override + public boolean hasLastChunkHashCode() { return ((bitField0_ & 0x00000080) == 0x00000080); } /** * optional int32 lastChunkHashCode = 8; */ - public int getLastChunkHashCode() { + @Override + public int getLastChunkHashCode() { return lastChunkHashCode_; } /** @@ -1075,6 +1134,7 @@ public final class InstallSnapshotMessages { }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + @Override public com.google.protobuf.ExtensionRegistry assignDescriptors( com.google.protobuf.Descriptors.FileDescriptor root) { descriptor = root; diff --git a/opendaylight/md-sal/sal-dummy-distributed-datastore/src/main/java/org/opendaylight/controller/dummy/datastore/DummyShard.java b/opendaylight/md-sal/sal-dummy-distributed-datastore/src/main/java/org/opendaylight/controller/dummy/datastore/DummyShard.java index 3f62b864f0..ad7836347d 100644 --- a/opendaylight/md-sal/sal-dummy-distributed-datastore/src/main/java/org/opendaylight/controller/dummy/datastore/DummyShard.java +++ b/opendaylight/md-sal/sal-dummy-distributed-datastore/src/main/java/org/opendaylight/controller/dummy/datastore/DummyShard.java @@ -45,7 +45,7 @@ public class DummyShard extends UntypedActor{ sender().tell(new RequestVoteReply(req.getTerm(), true), self()); } else if(o instanceof AppendEntries) { handleAppendEntries((AppendEntries)o); - } else if(InstallSnapshot.SERIALIZABLE_CLASS.equals(o.getClass())) { + } else if(InstallSnapshot.isSerializedType(o)) { InstallSnapshot req = InstallSnapshot.fromSerializable(o); handleInstallSnapshot(req); } else if(o instanceof InstallSnapshot){