Change InstallSnapshot and reply to use Externalizable Proxy 38/42638/4
authorTom Pantelis <tpanteli@brocade.com>
Tue, 26 Jul 2016 22:36:06 +0000 (18:36 -0400)
committerTom Pantelis <tpanteli@brocade.com>
Thu, 11 Aug 2016 01:28:26 +0000 (21:28 -0400)
This makes InstallSnapshot cleaner with no public no-arg constructor.

I also removed the InstallSnapshot protobuff message. In addition,
SerializableUtils is no longer needed as there's no more protobuff
messages.

Change-Id: I17aa4f7195cf09b798daee5587bbf50ccbc4bff0
Signed-off-by: Tom Pantelis <tpanteli@brocade.com>
18 files changed:
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/SerializationUtils.java [deleted file]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/RaftActorServerConfigurationSupportTest.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehaviorTest.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReplyTest.java [new file with mode: 0644]
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotTest.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/utils/MessageCollectorActor.java
opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/protobuff/messages/cluster/raft/AppendEntriesMessages.java [deleted file]
opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/protobuff/messages/cluster/raft/InstallSnapshotMessages.java [deleted file]
opendaylight/md-sal/sal-clustering-commons/src/main/resources/AppendEntriesMessages.proto [deleted file]
opendaylight/md-sal/sal-clustering-commons/src/main/resources/InstallSnapshot.proto [deleted file]
opendaylight/md-sal/sal-dummy-distributed-datastore/src/main/java/org/opendaylight/controller/dummy/datastore/DummyShard.java

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
deleted file mode 100644 (file)
index 3e5d3fa..0000000
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Copyright (c) 2014 Cisco 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;
-
-import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
-
-public class SerializationUtils {
-
-    public static Object fromSerializable(Object serializable){
-        if (InstallSnapshot.isSerializedType(serializable)) {
-            return InstallSnapshot.fromSerializable(serializable);
-        }
-        return serializable;
-    }
-}
index deb10da94187b340d6c4a82a2e7d57e0071e3ca3..befc6d412962a7395608c4ee1981b02014f5986d 100644 (file)
@@ -406,11 +406,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     protected void beforeSendHeartbeat(){}
 
     @Override
-    public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
+    public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
         Preconditions.checkNotNull(sender, "sender should not be null");
 
-        Object message = fromSerializableMessage(originalMessage);
-
         if (message instanceof RaftRPC) {
             RaftRPC rpc = (RaftRPC) message;
             // If RPC request or response contains term T > currentTerm:
index 943c4f97d5769f7fbc0835ec9999ab057b1e7577..d8b23f946e0f889b69a109d0d34463a7abb43f40 100644 (file)
@@ -17,7 +17,6 @@ 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.ApplyJournalEntries;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
@@ -399,10 +398,6 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor());
     }
 
-    protected Object fromSerializableMessage(Object serializable){
-        return SerializationUtils.fromSerializable(serializable);
-    }
-
     @Override
     public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
         if (message instanceof AppendEntries) {
index 90120e9ac36dae88b4b8a9cb39a1f251ba0b7070..52ed26758ee26b6f9949fa1202f1a44628b82364 100644 (file)
@@ -128,8 +128,8 @@ public class Candidate extends AbstractRaftActorBehavior {
     }
 
     @Override
-    public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
-        if (originalMessage instanceof ElectionTimeout) {
+    public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
+        if (message instanceof ElectionTimeout) {
             LOG.debug("{}: Received ElectionTimeout", logName());
 
             if (votesRequired == 0) {
@@ -147,7 +147,6 @@ public class Candidate extends AbstractRaftActorBehavior {
             return this;
         }
 
-        final Object message = fromSerializableMessage(originalMessage);
         if (message instanceof RaftRPC) {
 
             RaftRPC rpc = (RaftRPC) message;
index 8650d96f60b5b9b64f3bb69e4f8f649cdccab617..5f37af6e9d828482d16aac95178fc47418f27d6d 100644 (file)
@@ -354,12 +354,11 @@ public class Follower extends AbstractRaftActorBehavior {
     }
 
     @Override
-    public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
-        if (originalMessage instanceof ElectionTimeout || originalMessage instanceof TimeoutNow) {
-            return handleElectionTimeout(originalMessage);
+    public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
+        if (message instanceof ElectionTimeout || message instanceof TimeoutNow) {
+            return handleElectionTimeout(message);
         }
 
-        final Object message = fromSerializableMessage(originalMessage);
         if (!(message instanceof RaftRPC)) {
             // The rest of the processing requires the message to be a RaftRPC
             return null;
index d2a6a0d0824b53895e860491c7958aa50132c65d..5b6f030431e217016f7156b3c56172ba9e09fae1 100644 (file)
@@ -9,33 +9,23 @@
 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.cluster.raft.persisted.ServerConfigurationPayload;
-import org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages;
 
-public class InstallSnapshot extends AbstractRaftRPC implements Externalizable {
+public class InstallSnapshot extends AbstractRaftRPC {
     private static final long serialVersionUID = 1L;
-    public static final Class<InstallSnapshotMessages.InstallSnapshot> SERIALIZABLE_CLASS = InstallSnapshotMessages.InstallSnapshot.class;
-
-    private String leaderId;
-    private long lastIncludedIndex;
-    private long lastIncludedTerm;
-    private byte[] data;
-    private int chunkIndex;
-    private int totalChunks;
-    private Optional<Integer> lastChunkHashCode;
-    private Optional<ServerConfigurationPayload> serverConfig;
-
-    /**
-     * Empty constructor to satisfy Externalizable.
-     */
-    public InstallSnapshot() {
-    }
+
+    private final String leaderId;
+    private final long lastIncludedIndex;
+    private final long lastIncludedTerm;
+    private final byte[] data;
+    private final int chunkIndex;
+    private final int totalChunks;
+    private final Optional<Integer> lastChunkHashCode;
+    private final Optional<ServerConfigurationPayload> serverConfig;
 
     public InstallSnapshot(long term, String leaderId, long lastIncludedIndex, long lastIncludedTerm, byte[] data,
             int chunkIndex, int totalChunks, Optional<Integer> lastChunkHashCode, Optional<ServerConfigurationPayload> serverConfig) {
@@ -88,72 +78,9 @@ public class InstallSnapshot extends AbstractRaftRPC implements Externalizable {
         return serverConfig;
     }
 
-    @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());
-        }
-
-        out.writeByte(serverConfig.isPresent() ? 1 : 0);
-        if(serverConfig.isPresent()) {
-            out.writeObject(serverConfig.get());
-        }
-
-        out.writeObject(data);
-    }
-
-    @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());
-        }
-
-        serverConfig = Optional.absent();
-        boolean serverConfigPresent = in.readByte() == 1;
-        if(serverConfigPresent) {
-            serverConfig = Optional.of((ServerConfigurationPayload)in.readObject());
-        }
-
-        data = (byte[])in.readObject();
-    }
 
     public <T extends Object> 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();
-        }
+        return this;
     }
 
     @Override
@@ -164,29 +91,73 @@ public class InstallSnapshot extends AbstractRaftRPC implements Externalizable {
                 + ", serverConfig=" + serverConfig.orNull() + "]";
     }
 
-    public static InstallSnapshot fromSerializable (Object o) {
-        if(o instanceof InstallSnapshot) {
-            return (InstallSnapshot)o;
-        } else {
-            InstallSnapshotMessages.InstallSnapshot from =
-                    (InstallSnapshotMessages.InstallSnapshot) o;
+    private Object writeReplace() {
+        return new Proxy(this);
+    }
+
+    private static class Proxy implements Externalizable {
+        private static final long serialVersionUID = 1L;
+
+        private InstallSnapshot installSnapshot;
+
+        public Proxy() {
+        }
+
+        Proxy(InstallSnapshot installSnapshot) {
+            this.installSnapshot = installSnapshot;
+        }
+
+        @Override
+        public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeLong(installSnapshot.getTerm());
+            out.writeObject(installSnapshot.leaderId);
+            out.writeLong(installSnapshot.lastIncludedIndex);
+            out.writeLong(installSnapshot.lastIncludedTerm);
+            out.writeInt(installSnapshot.chunkIndex);
+            out.writeInt(installSnapshot.totalChunks);
+
+            out.writeByte(installSnapshot.lastChunkHashCode.isPresent() ? 1 : 0);
+            if(installSnapshot.lastChunkHashCode.isPresent()) {
+                out.writeInt(installSnapshot.lastChunkHashCode.get().intValue());
+            }
+
+            out.writeByte(installSnapshot.serverConfig.isPresent() ? 1 : 0);
+            if(installSnapshot.serverConfig.isPresent()) {
+                out.writeObject(installSnapshot.serverConfig.get());
+            }
+
+            out.writeObject(installSnapshot.data);
+        }
+
+        @Override
+        public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            long term = in.readLong();
+            String leaderId = (String) in.readObject();
+            long lastIncludedIndex = in.readLong();
+            long lastIncludedTerm = in.readLong();
+            int chunkIndex = in.readInt();
+            int totalChunks = in.readInt();
 
             Optional<Integer> lastChunkHashCode = Optional.absent();
-            if(from.hasLastChunkHashCode()){
-                lastChunkHashCode = Optional.of(from.getLastChunkHashCode());
+            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().toByteArray(),
-                    from.getChunkIndex(), from.getTotalChunks(), lastChunkHashCode,
-                    Optional.<ServerConfigurationPayload>absent());
+            Optional<ServerConfigurationPayload> serverConfig = Optional.absent();
+            boolean serverConfigPresent = in.readByte() == 1;
+            if(serverConfigPresent) {
+                serverConfig = Optional.of((ServerConfigurationPayload)in.readObject());
+            }
 
-            return installSnapshot;
+            byte[] data = (byte[])in.readObject();
+
+            installSnapshot = new InstallSnapshot(term, leaderId, lastIncludedIndex, lastIncludedTerm, data,
+                    chunkIndex, totalChunks, lastChunkHashCode, serverConfig);
         }
-    }
 
-    public static boolean isSerializedType(Object message) {
-        return message instanceof InstallSnapshot || message instanceof InstallSnapshotMessages.InstallSnapshot;
+        private Object readResolve() {
+            return installSnapshot;
+        }
     }
 }
index 1d502cc6f4a460d7b94beceb158b8826bedd40b4..41d2ae68c90e8e2f3846d6c3806693843f2b091c 100644 (file)
@@ -8,6 +8,11 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
 public class InstallSnapshotReply extends AbstractRaftRPC {
     private static final long serialVersionUID = 642227896390779503L;
 
@@ -17,8 +22,7 @@ public class InstallSnapshotReply extends AbstractRaftRPC {
     private final int chunkIndex;
     private final boolean success;
 
-    public InstallSnapshotReply(long term, String followerId, int chunkIndex,
-        boolean success) {
+    public InstallSnapshotReply(long term, String followerId, int chunkIndex, boolean success) {
         super(term);
         this.followerId = followerId;
         this.chunkIndex = chunkIndex;
@@ -44,4 +48,43 @@ public class InstallSnapshotReply extends AbstractRaftRPC {
                 .append(", chunkIndex=").append(chunkIndex).append(", success=").append(success).append("]");
         return builder.toString();
     }
+
+    private Object writeReplace() {
+        return new Proxy(this);
+    }
+
+    private static class Proxy implements Externalizable {
+        private static final long serialVersionUID = 1L;
+
+        private InstallSnapshotReply installSnapshotReply;
+
+        public Proxy() {
+        }
+
+        Proxy(InstallSnapshotReply installSnapshotReply) {
+            this.installSnapshotReply = installSnapshotReply;
+        }
+
+        @Override
+        public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeLong(installSnapshotReply.getTerm());
+            out.writeObject(installSnapshotReply.followerId);
+            out.writeInt(installSnapshotReply.chunkIndex);
+            out.writeBoolean(installSnapshotReply.success);
+        }
+
+        @Override
+        public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            long term = in.readLong();
+            String followerId = (String) in.readObject();
+            int chunkIndex = in.readInt();
+            boolean success = in.readBoolean();
+
+            installSnapshotReply = new InstallSnapshotReply(term, followerId, chunkIndex, success);
+        }
+
+        private Object readResolve() {
+            return installSnapshotReply;
+        }
+    }
 }
index dc02af012545a7337451d1d518ef66f363dad260..630ec7fb930302b5d0a98e1b20517207e246ec93 100644 (file)
@@ -375,7 +375,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         followerActor.underlyingActor().setBehavior(newFollower2);
 
         MockNewFollowerRaftActor newFollowerRaftActorInstance = newFollowerRaftActor.underlyingActor();
-        newFollowerRaftActorInstance.setDropMessageOfType(InstallSnapshot.SERIALIZABLE_CLASS);
+        newFollowerRaftActorInstance.setDropMessageOfType(InstallSnapshot.class);
 
         leaderActor.tell(new AddServer(NEW_SERVER_ID, newFollowerRaftActor.path().toString(), true), testKit.getRef());
 
@@ -606,7 +606,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ((DefaultConfigParamsImpl)leaderActorContext.getConfigParams()).setElectionTimeoutFactor(1);
 
         // Drop the InstallSnapshot message so it times out
-        newFollowerRaftActor.underlyingActor().setDropMessageOfType(InstallSnapshot.SERIALIZABLE_CLASS);
+        newFollowerRaftActor.underlyingActor().setDropMessageOfType(InstallSnapshot.class);
 
         leaderActor.tell(new AddServer(NEW_SERVER_ID, newFollowerRaftActor.path().toString(), true), testKit.getRef());
 
index b5be834fec064d21d0c40a10000c413c9fd64f60..1cee5e0d88347cf629f2412c534f90e26b852b7d 100644 (file)
@@ -29,7 +29,6 @@ import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
 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.TestActorFactory;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
@@ -342,10 +341,6 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
         return new RequestVoteReply(100, false);
     }
 
-    protected Object fromSerializableMessage(Object serializable){
-        return SerializationUtils.fromSerializable(serializable);
-    }
-
     protected ByteString toByteString(Map<String, String> state) {
         ByteArrayOutputStream bos = new ByteArrayOutputStream();
         try(ObjectOutputStream oos = new ObjectOutputStream(bos)) {
index 292562c92f8b9dc97ed2fd32734e9b18c357309c..0e882368a2949f82594664cc0f7f07ab1b166b5f 100644 (file)
@@ -42,7 +42,6 @@ import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.RaftVersions;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogImplEntry;
-import org.opendaylight.controller.cluster.raft.SerializationUtils;
 import org.opendaylight.controller.cluster.raft.Snapshot;
 import org.opendaylight.controller.cluster.raft.VotingState;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
@@ -598,9 +597,7 @@ public class LeaderTest extends AbstractLeaderTest<Leader> {
 
         leader.handleMessage(leaderActor, SendHeartBeat.INSTANCE);
 
-        AppendEntries aeproto = MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
-
-        AppendEntries ae = (AppendEntries) SerializationUtils.fromSerializable(aeproto);
+        AppendEntries ae = MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
 
         assertTrue("AppendEntries should be sent with empty entries", ae.getEntries().isEmpty());
 
diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReplyTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReplyTest.java
new file mode 100644 (file)
index 0000000..cf05173
--- /dev/null
@@ -0,0 +1,31 @@
+/*
+ * Copyright (c) 2016 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.assertEquals;
+import org.apache.commons.lang.SerializationUtils;
+import org.junit.Test;
+
+/**
+ * Unit tests for InstallSnapshotReply.
+ *
+ * @author Thomas Pantelis
+ */
+public class InstallSnapshotReplyTest {
+
+    @Test
+    public void testSerialization() {
+        InstallSnapshotReply expected = new InstallSnapshotReply(5L, "follower", 1, true);
+        InstallSnapshotReply cloned = (InstallSnapshotReply) SerializationUtils.clone(expected);
+
+        assertEquals("getTerm", expected.getTerm(), cloned.getTerm());
+        assertEquals("getFollowerId", expected.getFollowerId(), cloned.getFollowerId());
+        assertEquals("getChunkIndex", expected.getChunkIndex(), cloned.getChunkIndex());
+        assertEquals("isSuccess", expected.isSuccess(), cloned.isSuccess());
+    }
+}
index cde180796b8eba6634971ffd22238513a6bb602e..70eece61636f90a0472625deee3569d8028d4f11 100644 (file)
@@ -17,7 +17,6 @@ import org.junit.Test;
 import org.opendaylight.controller.cluster.raft.RaftVersions;
 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
 import org.opendaylight.controller.cluster.raft.persisted.ServerInfo;
-import org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages;
 
 /**
  * Unit tests for InstallSnapshot.
@@ -45,36 +44,14 @@ public class InstallSnapshotTest {
         Object serialized = expected.toSerializable(RaftVersions.CURRENT_VERSION);
         assertEquals("Serialized type", InstallSnapshot.class, serialized.getClass());
 
-        InstallSnapshot actual = InstallSnapshot.fromSerializable(SerializationUtils.clone((Serializable) serialized));
+        InstallSnapshot actual = (InstallSnapshot) 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)));
+        actual = (InstallSnapshot) 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.<Integer>of(54321),
-                Optional.<ServerConfigurationPayload>absent());
-
-        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 static void verifyInstallSnapshot(InstallSnapshot expected, InstallSnapshot actual) {
         assertEquals("getTerm", expected.getTerm(), actual.getTerm());
         assertEquals("getChunkIndex", expected.getChunkIndex(), actual.getChunkIndex());
index 7e3e344908aa83fe973a666beb84a5701b08a13c..f7caf0f4a57ad683dbea5559ce4775136e865e2d 100644 (file)
@@ -22,7 +22,6 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import org.junit.Assert;
-import org.opendaylight.controller.cluster.raft.SerializationUtils;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.Duration;
@@ -47,7 +46,7 @@ public class MessageCollectorActor extends UntypedActor {
         } else if(CLEAR_MESSAGES.equals(message)) {
             clear();
         } else if(message != null) {
-            messages.add(SerializationUtils.fromSerializable(message));
+            messages.add(message);
         }
     }
 
diff --git a/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/protobuff/messages/cluster/raft/AppendEntriesMessages.java b/opendaylight/md-sal/sal-clustering-commons/src/main/java/org/opendaylight/controller/protobuff/messages/cluster/raft/AppendEntriesMessages.java
deleted file mode 100644 (file)
index 215c265..0000000
+++ /dev/null
@@ -1,2420 +0,0 @@
-// Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: AppendEntriesMessages.proto
-
-package org.opendaylight.controller.protobuff.messages.cluster.raft;
-
-public final class AppendEntriesMessages {
-  private AppendEntriesMessages() {}
-  public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
-  }
-  public interface AppendEntriesOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // optional int64 term = 1;
-    /**
-     * <code>optional int64 term = 1;</code>
-     */
-    boolean hasTerm();
-    /**
-     * <code>optional int64 term = 1;</code>
-     */
-    long getTerm();
-
-    // optional string leaderId = 2;
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    boolean hasLeaderId();
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    java.lang.String getLeaderId();
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    com.google.protobuf.ByteString
-        getLeaderIdBytes();
-
-    // optional int64 prevLogIndex = 3;
-    /**
-     * <code>optional int64 prevLogIndex = 3;</code>
-     */
-    boolean hasPrevLogIndex();
-    /**
-     * <code>optional int64 prevLogIndex = 3;</code>
-     */
-    long getPrevLogIndex();
-
-    // optional int64 prevLogTerm = 4;
-    /**
-     * <code>optional int64 prevLogTerm = 4;</code>
-     */
-    boolean hasPrevLogTerm();
-    /**
-     * <code>optional int64 prevLogTerm = 4;</code>
-     */
-    long getPrevLogTerm();
-
-    // repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    java.util.List<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry>
-        getLogEntriesList();
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry getLogEntries(int index);
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    int getLogEntriesCount();
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder>
-        getLogEntriesOrBuilderList();
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder getLogEntriesOrBuilder(
-        int index);
-
-    // optional int64 leaderCommit = 6;
-    /**
-     * <code>optional int64 leaderCommit = 6;</code>
-     */
-    boolean hasLeaderCommit();
-    /**
-     * <code>optional int64 leaderCommit = 6;</code>
-     */
-    long getLeaderCommit();
-  }
-  /**
-   * Protobuf type {@code org.opendaylight.controller.cluster.raft.AppendEntries}
-   */
-  public static final class AppendEntries extends
-      com.google.protobuf.GeneratedMessage
-      implements AppendEntriesOrBuilder {
-    // Use AppendEntries.newBuilder() to construct.
-    private AppendEntries(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private AppendEntries(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final AppendEntries defaultInstance;
-    public static AppendEntries getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public AppendEntries getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private AppendEntries(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 8: {
-              bitField0_ |= 0x00000001;
-              term_ = input.readInt64();
-              break;
-            }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              leaderId_ = input.readBytes();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              prevLogIndex_ = input.readInt64();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              prevLogTerm_ = input.readInt64();
-              break;
-            }
-            case 42: {
-              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-                logEntries_ = new java.util.ArrayList<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry>();
-                mutable_bitField0_ |= 0x00000010;
-              }
-              logEntries_.add(input.readMessage(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.PARSER, extensionRegistry));
-              break;
-            }
-            case 48: {
-              bitField0_ |= 0x00000010;
-              leaderCommit_ = input.readInt64();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-          logEntries_ = java.util.Collections.unmodifiableList(logEntries_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.class, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<AppendEntries> PARSER =
-        new com.google.protobuf.AbstractParser<AppendEntries>() {
-      public AppendEntries parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new AppendEntries(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<AppendEntries> getParserForType() {
-      return PARSER;
-    }
-
-    public interface ReplicatedLogEntryOrBuilder
-        extends com.google.protobuf.MessageOrBuilder {
-
-      // optional int64 term = 1;
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      boolean hasTerm();
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      long getTerm();
-
-      // optional int64 index = 2;
-      /**
-       * <code>optional int64 index = 2;</code>
-       */
-      boolean hasIndex();
-      /**
-       * <code>optional int64 index = 2;</code>
-       */
-      long getIndex();
-
-      // optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;
-      /**
-       * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-       */
-      boolean hasData();
-      /**
-       * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-       */
-      org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload getData();
-      /**
-       * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-       */
-      org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.PayloadOrBuilder getDataOrBuilder();
-    }
-    /**
-     * Protobuf type {@code org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry}
-     */
-    public static final class ReplicatedLogEntry extends
-        com.google.protobuf.GeneratedMessage
-        implements ReplicatedLogEntryOrBuilder {
-      // Use ReplicatedLogEntry.newBuilder() to construct.
-      private ReplicatedLogEntry(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-        super(builder);
-        this.unknownFields = builder.getUnknownFields();
-      }
-      private ReplicatedLogEntry(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-      private static final ReplicatedLogEntry defaultInstance;
-      public static ReplicatedLogEntry getDefaultInstance() {
-        return defaultInstance;
-      }
-
-      public ReplicatedLogEntry getDefaultInstanceForType() {
-        return defaultInstance;
-      }
-
-      private final com.google.protobuf.UnknownFieldSet unknownFields;
-      @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
-          getUnknownFields() {
-        return this.unknownFields;
-      }
-      private ReplicatedLogEntry(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        initFields();
-        int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
-        try {
-          boolean done = false;
-          while (!done) {
-            int tag = input.readTag();
-            switch (tag) {
-              case 0:
-                done = true;
-                break;
-              default: {
-                if (!parseUnknownField(input, unknownFields,
-                                       extensionRegistry, tag)) {
-                  done = true;
-                }
-                break;
-              }
-              case 8: {
-                bitField0_ |= 0x00000001;
-                term_ = input.readInt64();
-                break;
-              }
-              case 16: {
-                bitField0_ |= 0x00000002;
-                index_ = input.readInt64();
-                break;
-              }
-              case 26: {
-                org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.Builder subBuilder = null;
-                if (((bitField0_ & 0x00000004) == 0x00000004)) {
-                  subBuilder = data_.toBuilder();
-                }
-                data_ = input.readMessage(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.PARSER, extensionRegistry);
-                if (subBuilder != null) {
-                  subBuilder.mergeFrom(data_);
-                  data_ = subBuilder.buildPartial();
-                }
-                bitField0_ |= 0x00000004;
-                break;
-              }
-            }
-          }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          throw e.setUnfinishedMessage(this);
-        } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
-              e.getMessage()).setUnfinishedMessage(this);
-        } finally {
-          this.unknownFields = unknownFields.build();
-          makeExtensionsImmutable();
-        }
-      }
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.class, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder.class);
-      }
-
-      public static com.google.protobuf.Parser<ReplicatedLogEntry> PARSER =
-          new com.google.protobuf.AbstractParser<ReplicatedLogEntry>() {
-        public ReplicatedLogEntry parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
-          return new ReplicatedLogEntry(input, extensionRegistry);
-        }
-      };
-
-      @java.lang.Override
-      public com.google.protobuf.Parser<ReplicatedLogEntry> getParserForType() {
-        return PARSER;
-      }
-
-      public interface PayloadOrBuilder extends
-          com.google.protobuf.GeneratedMessage.
-              ExtendableMessageOrBuilder<Payload> {
-
-        // optional string clientPayloadClassName = 1;
-        /**
-         * <code>optional string clientPayloadClassName = 1;</code>
-         */
-        boolean hasClientPayloadClassName();
-        /**
-         * <code>optional string clientPayloadClassName = 1;</code>
-         */
-        java.lang.String getClientPayloadClassName();
-        /**
-         * <code>optional string clientPayloadClassName = 1;</code>
-         */
-        com.google.protobuf.ByteString
-            getClientPayloadClassNameBytes();
-      }
-      /**
-       * Protobuf type {@code org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload}
-       */
-      public static final class Payload extends
-          com.google.protobuf.GeneratedMessage.ExtendableMessage<
-            Payload> implements PayloadOrBuilder {
-        // Use Payload.newBuilder() to construct.
-        private Payload(com.google.protobuf.GeneratedMessage.ExtendableBuilder<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload, ?> builder) {
-          super(builder);
-          this.unknownFields = builder.getUnknownFields();
-        }
-        private Payload(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-        private static final Payload defaultInstance;
-        public static Payload getDefaultInstance() {
-          return defaultInstance;
-        }
-
-        public Payload getDefaultInstanceForType() {
-          return defaultInstance;
-        }
-
-        private final com.google.protobuf.UnknownFieldSet unknownFields;
-        @java.lang.Override
-        public final com.google.protobuf.UnknownFieldSet
-            getUnknownFields() {
-          return this.unknownFields;
-        }
-        private Payload(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
-          initFields();
-          int mutable_bitField0_ = 0;
-          com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-              com.google.protobuf.UnknownFieldSet.newBuilder();
-          try {
-            boolean done = false;
-            while (!done) {
-              int tag = input.readTag();
-              switch (tag) {
-                case 0:
-                  done = true;
-                  break;
-                default: {
-                  if (!parseUnknownField(input, unknownFields,
-                                         extensionRegistry, tag)) {
-                    done = true;
-                  }
-                  break;
-                }
-                case 10: {
-                  bitField0_ |= 0x00000001;
-                  clientPayloadClassName_ = input.readBytes();
-                  break;
-                }
-              }
-            }
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-            throw e.setUnfinishedMessage(this);
-          } catch (java.io.IOException e) {
-            throw new com.google.protobuf.InvalidProtocolBufferException(
-                e.getMessage()).setUnfinishedMessage(this);
-          } finally {
-            this.unknownFields = unknownFields.build();
-            makeExtensionsImmutable();
-          }
-        }
-        public static final com.google.protobuf.Descriptors.Descriptor
-            getDescriptor() {
-          return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_descriptor;
-        }
-
-        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-            internalGetFieldAccessorTable() {
-          return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_fieldAccessorTable
-              .ensureFieldAccessorsInitialized(
-                  org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.class, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.Builder.class);
-        }
-
-        public static com.google.protobuf.Parser<Payload> PARSER =
-            new com.google.protobuf.AbstractParser<Payload>() {
-          public Payload parsePartialFrom(
-              com.google.protobuf.CodedInputStream input,
-              com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-              throws com.google.protobuf.InvalidProtocolBufferException {
-            return new Payload(input, extensionRegistry);
-          }
-        };
-
-        @java.lang.Override
-        public com.google.protobuf.Parser<Payload> getParserForType() {
-          return PARSER;
-        }
-
-        private int bitField0_;
-        // optional string clientPayloadClassName = 1;
-        public static final int CLIENTPAYLOADCLASSNAME_FIELD_NUMBER = 1;
-        private java.lang.Object clientPayloadClassName_;
-        /**
-         * <code>optional string clientPayloadClassName = 1;</code>
-         */
-        public boolean hasClientPayloadClassName() {
-          return ((bitField0_ & 0x00000001) == 0x00000001);
-        }
-        /**
-         * <code>optional string clientPayloadClassName = 1;</code>
-         */
-        public java.lang.String getClientPayloadClassName() {
-          java.lang.Object ref = clientPayloadClassName_;
-          if (ref instanceof java.lang.String) {
-            return (java.lang.String) ref;
-          } else {
-            com.google.protobuf.ByteString bs =
-                (com.google.protobuf.ByteString) ref;
-            java.lang.String s = bs.toStringUtf8();
-            if (bs.isValidUtf8()) {
-              clientPayloadClassName_ = s;
-            }
-            return s;
-          }
-        }
-        /**
-         * <code>optional string clientPayloadClassName = 1;</code>
-         */
-        public com.google.protobuf.ByteString
-            getClientPayloadClassNameBytes() {
-          java.lang.Object ref = clientPayloadClassName_;
-          if (ref instanceof java.lang.String) {
-            com.google.protobuf.ByteString b =
-                com.google.protobuf.ByteString.copyFromUtf8(
-                    (java.lang.String) ref);
-            clientPayloadClassName_ = b;
-            return b;
-          } else {
-            return (com.google.protobuf.ByteString) ref;
-          }
-        }
-
-        private void initFields() {
-          clientPayloadClassName_ = "";
-        }
-        private byte memoizedIsInitialized = -1;
-        public final boolean isInitialized() {
-          byte isInitialized = memoizedIsInitialized;
-          if (isInitialized != -1) return isInitialized == 1;
-
-          if (!extensionsAreInitialized()) {
-            memoizedIsInitialized = 0;
-            return false;
-          }
-          memoizedIsInitialized = 1;
-          return true;
-        }
-
-        public void writeTo(com.google.protobuf.CodedOutputStream output)
-                            throws java.io.IOException {
-          getSerializedSize();
-          com.google.protobuf.GeneratedMessage
-            .ExtendableMessage<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload>.ExtensionWriter extensionWriter =
-              newExtensionWriter();
-          if (((bitField0_ & 0x00000001) == 0x00000001)) {
-            output.writeBytes(1, getClientPayloadClassNameBytes());
-          }
-          extensionWriter.writeUntil(101, output);
-          getUnknownFields().writeTo(output);
-        }
-
-        private int memoizedSerializedSize = -1;
-        public int getSerializedSize() {
-          int size = memoizedSerializedSize;
-          if (size != -1) return size;
-
-          size = 0;
-          if (((bitField0_ & 0x00000001) == 0x00000001)) {
-            size += com.google.protobuf.CodedOutputStream
-              .computeBytesSize(1, getClientPayloadClassNameBytes());
-          }
-          size += extensionsSerializedSize();
-          size += getUnknownFields().getSerializedSize();
-          memoizedSerializedSize = size;
-          return size;
-        }
-
-        private static final long serialVersionUID = 0L;
-        @java.lang.Override
-        protected java.lang.Object writeReplace()
-            throws java.io.ObjectStreamException {
-          return super.writeReplace();
-        }
-
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseFrom(
-            com.google.protobuf.ByteString data)
-            throws com.google.protobuf.InvalidProtocolBufferException {
-          return PARSER.parseFrom(data);
-        }
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseFrom(
-            com.google.protobuf.ByteString data,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
-          return PARSER.parseFrom(data, extensionRegistry);
-        }
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseFrom(byte[] data)
-            throws com.google.protobuf.InvalidProtocolBufferException {
-          return PARSER.parseFrom(data);
-        }
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseFrom(
-            byte[] data,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
-          return PARSER.parseFrom(data, extensionRegistry);
-        }
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseFrom(java.io.InputStream input)
-            throws java.io.IOException {
-          return PARSER.parseFrom(input);
-        }
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseFrom(
-            java.io.InputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws java.io.IOException {
-          return PARSER.parseFrom(input, extensionRegistry);
-        }
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseDelimitedFrom(java.io.InputStream input)
-            throws java.io.IOException {
-          return PARSER.parseDelimitedFrom(input);
-        }
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseDelimitedFrom(
-            java.io.InputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws java.io.IOException {
-          return PARSER.parseDelimitedFrom(input, extensionRegistry);
-        }
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseFrom(
-            com.google.protobuf.CodedInputStream input)
-            throws java.io.IOException {
-          return PARSER.parseFrom(input);
-        }
-        public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parseFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws java.io.IOException {
-          return PARSER.parseFrom(input, extensionRegistry);
-        }
-
-        public static Builder newBuilder() { return Builder.create(); }
-        public Builder newBuilderForType() { return newBuilder(); }
-        public static Builder newBuilder(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload prototype) {
-          return newBuilder().mergeFrom(prototype);
-        }
-        public Builder toBuilder() { return newBuilder(this); }
-
-        @java.lang.Override
-        protected Builder newBuilderForType(
-            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-          Builder builder = new Builder(parent);
-          return builder;
-        }
-        /**
-         * Protobuf type {@code org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload}
-         */
-        public static final class Builder extends
-            com.google.protobuf.GeneratedMessage.ExtendableBuilder<
-              org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload, Builder> implements org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.PayloadOrBuilder {
-          public static final com.google.protobuf.Descriptors.Descriptor
-              getDescriptor() {
-            return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_descriptor;
-          }
-
-          protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-              internalGetFieldAccessorTable() {
-            return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_fieldAccessorTable
-                .ensureFieldAccessorsInitialized(
-                    org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.class, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.Builder.class);
-          }
-
-          // Construct using org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.newBuilder()
-          private Builder() {
-            maybeForceBuilderInitialization();
-          }
-
-          private Builder(
-              com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-            super(parent);
-            maybeForceBuilderInitialization();
-          }
-          private void maybeForceBuilderInitialization() {
-            if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-            }
-          }
-          private static Builder create() {
-            return new Builder();
-          }
-
-          public Builder clear() {
-            super.clear();
-            clientPayloadClassName_ = "";
-            bitField0_ = (bitField0_ & ~0x00000001);
-            return this;
-          }
-
-          public Builder clone() {
-            return create().mergeFrom(buildPartial());
-          }
-
-          public com.google.protobuf.Descriptors.Descriptor
-              getDescriptorForType() {
-            return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_descriptor;
-          }
-
-          public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload getDefaultInstanceForType() {
-            return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.getDefaultInstance();
-          }
-
-          public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload build() {
-            org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload result = buildPartial();
-            if (!result.isInitialized()) {
-              throw newUninitializedMessageException(result);
-            }
-            return result;
-          }
-
-          public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload buildPartial() {
-            org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload result = new org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload(this);
-            int from_bitField0_ = bitField0_;
-            int to_bitField0_ = 0;
-            if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-              to_bitField0_ |= 0x00000001;
-            }
-            result.clientPayloadClassName_ = clientPayloadClassName_;
-            result.bitField0_ = to_bitField0_;
-            onBuilt();
-            return result;
-          }
-
-          public Builder mergeFrom(com.google.protobuf.Message other) {
-            if (other instanceof org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload) {
-              return mergeFrom((org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload)other);
-            } else {
-              super.mergeFrom(other);
-              return this;
-            }
-          }
-
-          public Builder mergeFrom(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload other) {
-            if (other == org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.getDefaultInstance()) return this;
-            if (other.hasClientPayloadClassName()) {
-              bitField0_ |= 0x00000001;
-              clientPayloadClassName_ = other.clientPayloadClassName_;
-              onChanged();
-            }
-            this.mergeExtensionFields(other);
-            this.mergeUnknownFields(other.getUnknownFields());
-            return this;
-          }
-
-          public final boolean isInitialized() {
-            if (!extensionsAreInitialized()) {
-
-              return false;
-            }
-            return true;
-          }
-
-          public Builder mergeFrom(
-              com.google.protobuf.CodedInputStream input,
-              com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-              throws java.io.IOException {
-            org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload parsedMessage = null;
-            try {
-              parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-            } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-              parsedMessage = (org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload) e.getUnfinishedMessage();
-              throw e;
-            } finally {
-              if (parsedMessage != null) {
-                mergeFrom(parsedMessage);
-              }
-            }
-            return this;
-          }
-          private int bitField0_;
-
-          // optional string clientPayloadClassName = 1;
-          private java.lang.Object clientPayloadClassName_ = "";
-          /**
-           * <code>optional string clientPayloadClassName = 1;</code>
-           */
-          public boolean hasClientPayloadClassName() {
-            return ((bitField0_ & 0x00000001) == 0x00000001);
-          }
-          /**
-           * <code>optional string clientPayloadClassName = 1;</code>
-           */
-          public java.lang.String getClientPayloadClassName() {
-            java.lang.Object ref = clientPayloadClassName_;
-            if (!(ref instanceof java.lang.String)) {
-              java.lang.String s = ((com.google.protobuf.ByteString) ref)
-                  .toStringUtf8();
-              clientPayloadClassName_ = s;
-              return s;
-            } else {
-              return (java.lang.String) ref;
-            }
-          }
-          /**
-           * <code>optional string clientPayloadClassName = 1;</code>
-           */
-          public com.google.protobuf.ByteString
-              getClientPayloadClassNameBytes() {
-            java.lang.Object ref = clientPayloadClassName_;
-            if (ref instanceof String) {
-              com.google.protobuf.ByteString b =
-                  com.google.protobuf.ByteString.copyFromUtf8(
-                      (java.lang.String) ref);
-              clientPayloadClassName_ = b;
-              return b;
-            } else {
-              return (com.google.protobuf.ByteString) ref;
-            }
-          }
-          /**
-           * <code>optional string clientPayloadClassName = 1;</code>
-           */
-          public Builder setClientPayloadClassName(
-              java.lang.String value) {
-            if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-            clientPayloadClassName_ = value;
-            onChanged();
-            return this;
-          }
-          /**
-           * <code>optional string clientPayloadClassName = 1;</code>
-           */
-          public Builder clearClientPayloadClassName() {
-            bitField0_ = (bitField0_ & ~0x00000001);
-            clientPayloadClassName_ = getDefaultInstance().getClientPayloadClassName();
-            onChanged();
-            return this;
-          }
-          /**
-           * <code>optional string clientPayloadClassName = 1;</code>
-           */
-          public Builder setClientPayloadClassNameBytes(
-              com.google.protobuf.ByteString value) {
-            if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-            clientPayloadClassName_ = value;
-            onChanged();
-            return this;
-          }
-
-          // @@protoc_insertion_point(builder_scope:org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload)
-        }
-
-        static {
-          defaultInstance = new Payload(true);
-          defaultInstance.initFields();
-        }
-
-        // @@protoc_insertion_point(class_scope:org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload)
-      }
-
-      private int bitField0_;
-      // optional int64 term = 1;
-      public static final int TERM_FIELD_NUMBER = 1;
-      private long term_;
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      public boolean hasTerm() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      public long getTerm() {
-        return term_;
-      }
-
-      // optional int64 index = 2;
-      public static final int INDEX_FIELD_NUMBER = 2;
-      private long index_;
-      /**
-       * <code>optional int64 index = 2;</code>
-       */
-      public boolean hasIndex() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional int64 index = 2;</code>
-       */
-      public long getIndex() {
-        return index_;
-      }
-
-      // optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;
-      public static final int DATA_FIELD_NUMBER = 3;
-      private org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload data_;
-      /**
-       * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-       */
-      public boolean hasData() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-       */
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload getData() {
-        return data_;
-      }
-      /**
-       * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-       */
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.PayloadOrBuilder getDataOrBuilder() {
-        return data_;
-      }
-
-      private void initFields() {
-        term_ = 0L;
-        index_ = 0L;
-        data_ = org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.getDefaultInstance();
-      }
-      private byte memoizedIsInitialized = -1;
-      public final boolean isInitialized() {
-        byte isInitialized = memoizedIsInitialized;
-        if (isInitialized != -1) return isInitialized == 1;
-
-        if (hasData()) {
-          if (!getData().isInitialized()) {
-            memoizedIsInitialized = 0;
-            return false;
-          }
-        }
-        memoizedIsInitialized = 1;
-        return true;
-      }
-
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
-                          throws java.io.IOException {
-        getSerializedSize();
-        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          output.writeInt64(1, term_);
-        }
-        if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          output.writeInt64(2, index_);
-        }
-        if (((bitField0_ & 0x00000004) == 0x00000004)) {
-          output.writeMessage(3, data_);
-        }
-        getUnknownFields().writeTo(output);
-      }
-
-      private int memoizedSerializedSize = -1;
-      public int getSerializedSize() {
-        int size = memoizedSerializedSize;
-        if (size != -1) return size;
-
-        size = 0;
-        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
-            .computeInt64Size(1, term_);
-        }
-        if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          size += com.google.protobuf.CodedOutputStream
-            .computeInt64Size(2, index_);
-        }
-        if (((bitField0_ & 0x00000004) == 0x00000004)) {
-          size += com.google.protobuf.CodedOutputStream
-            .computeMessageSize(3, data_);
-        }
-        size += getUnknownFields().getSerializedSize();
-        memoizedSerializedSize = size;
-        return size;
-      }
-
-      private static final long serialVersionUID = 0L;
-      @java.lang.Override
-      protected java.lang.Object writeReplace()
-          throws java.io.ObjectStreamException {
-        return super.writeReplace();
-      }
-
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data);
-      }
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data, extensionRegistry);
-      }
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data);
-      }
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseFrom(
-          byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return PARSER.parseFrom(data, extensionRegistry);
-      }
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseFrom(java.io.InputStream input)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input);
-      }
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseFrom(
-          java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input, extensionRegistry);
-      }
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseDelimitedFrom(java.io.InputStream input)
-          throws java.io.IOException {
-        return PARSER.parseDelimitedFrom(input);
-      }
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseDelimitedFrom(
-          java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        return PARSER.parseDelimitedFrom(input, extensionRegistry);
-      }
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseFrom(
-          com.google.protobuf.CodedInputStream input)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input);
-      }
-      public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        return PARSER.parseFrom(input, extensionRegistry);
-      }
-
-      public static Builder newBuilder() { return Builder.create(); }
-      public Builder newBuilderForType() { return newBuilder(); }
-      public static Builder newBuilder(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry prototype) {
-        return newBuilder().mergeFrom(prototype);
-      }
-      public Builder toBuilder() { return newBuilder(this); }
-
-      @java.lang.Override
-      protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        Builder builder = new Builder(parent);
-        return builder;
-      }
-      /**
-       * Protobuf type {@code org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry}
-       */
-      public static final class Builder extends
-          com.google.protobuf.GeneratedMessage.Builder<Builder>
-         implements org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
-            getDescriptor() {
-          return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_descriptor;
-        }
-
-        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-            internalGetFieldAccessorTable() {
-          return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_fieldAccessorTable
-              .ensureFieldAccessorsInitialized(
-                  org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.class, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder.class);
-        }
-
-        // Construct using org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.newBuilder()
-        private Builder() {
-          maybeForceBuilderInitialization();
-        }
-
-        private Builder(
-            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-          super(parent);
-          maybeForceBuilderInitialization();
-        }
-        private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-            getDataFieldBuilder();
-          }
-        }
-        private static Builder create() {
-          return new Builder();
-        }
-
-        public Builder clear() {
-          super.clear();
-          term_ = 0L;
-          bitField0_ = (bitField0_ & ~0x00000001);
-          index_ = 0L;
-          bitField0_ = (bitField0_ & ~0x00000002);
-          if (dataBuilder_ == null) {
-            data_ = org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.getDefaultInstance();
-          } else {
-            dataBuilder_.clear();
-          }
-          bitField0_ = (bitField0_ & ~0x00000004);
-          return this;
-        }
-
-        public Builder clone() {
-          return create().mergeFrom(buildPartial());
-        }
-
-        public com.google.protobuf.Descriptors.Descriptor
-            getDescriptorForType() {
-          return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_descriptor;
-        }
-
-        public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry getDefaultInstanceForType() {
-          return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.getDefaultInstance();
-        }
-
-        public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry build() {
-          org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry result = buildPartial();
-          if (!result.isInitialized()) {
-            throw newUninitializedMessageException(result);
-          }
-          return result;
-        }
-
-        public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry buildPartial() {
-          org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry result = new org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry(this);
-          int from_bitField0_ = bitField0_;
-          int to_bitField0_ = 0;
-          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-            to_bitField0_ |= 0x00000001;
-          }
-          result.term_ = term_;
-          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-            to_bitField0_ |= 0x00000002;
-          }
-          result.index_ = index_;
-          if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-            to_bitField0_ |= 0x00000004;
-          }
-          if (dataBuilder_ == null) {
-            result.data_ = data_;
-          } else {
-            result.data_ = dataBuilder_.build();
-          }
-          result.bitField0_ = to_bitField0_;
-          onBuilt();
-          return result;
-        }
-
-        public Builder mergeFrom(com.google.protobuf.Message other) {
-          if (other instanceof org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry) {
-            return mergeFrom((org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry)other);
-          } else {
-            super.mergeFrom(other);
-            return this;
-          }
-        }
-
-        public Builder mergeFrom(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry other) {
-          if (other == org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.getDefaultInstance()) return this;
-          if (other.hasTerm()) {
-            setTerm(other.getTerm());
-          }
-          if (other.hasIndex()) {
-            setIndex(other.getIndex());
-          }
-          if (other.hasData()) {
-            mergeData(other.getData());
-          }
-          this.mergeUnknownFields(other.getUnknownFields());
-          return this;
-        }
-
-        public final boolean isInitialized() {
-          if (hasData()) {
-            if (!getData().isInitialized()) {
-
-              return false;
-            }
-          }
-          return true;
-        }
-
-        public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws java.io.IOException {
-          org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry parsedMessage = null;
-          try {
-            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-            parsedMessage = (org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry) e.getUnfinishedMessage();
-            throw e;
-          } finally {
-            if (parsedMessage != null) {
-              mergeFrom(parsedMessage);
-            }
-          }
-          return this;
-        }
-        private int bitField0_;
-
-        // optional int64 term = 1;
-        private long term_ ;
-        /**
-         * <code>optional int64 term = 1;</code>
-         */
-        public boolean hasTerm() {
-          return ((bitField0_ & 0x00000001) == 0x00000001);
-        }
-        /**
-         * <code>optional int64 term = 1;</code>
-         */
-        public long getTerm() {
-          return term_;
-        }
-        /**
-         * <code>optional int64 term = 1;</code>
-         */
-        public Builder setTerm(long value) {
-          bitField0_ |= 0x00000001;
-          term_ = value;
-          onChanged();
-          return this;
-        }
-        /**
-         * <code>optional int64 term = 1;</code>
-         */
-        public Builder clearTerm() {
-          bitField0_ = (bitField0_ & ~0x00000001);
-          term_ = 0L;
-          onChanged();
-          return this;
-        }
-
-        // optional int64 index = 2;
-        private long index_ ;
-        /**
-         * <code>optional int64 index = 2;</code>
-         */
-        public boolean hasIndex() {
-          return ((bitField0_ & 0x00000002) == 0x00000002);
-        }
-        /**
-         * <code>optional int64 index = 2;</code>
-         */
-        public long getIndex() {
-          return index_;
-        }
-        /**
-         * <code>optional int64 index = 2;</code>
-         */
-        public Builder setIndex(long value) {
-          bitField0_ |= 0x00000002;
-          index_ = value;
-          onChanged();
-          return this;
-        }
-        /**
-         * <code>optional int64 index = 2;</code>
-         */
-        public Builder clearIndex() {
-          bitField0_ = (bitField0_ & ~0x00000002);
-          index_ = 0L;
-          onChanged();
-          return this;
-        }
-
-        // optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;
-        private org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload data_ = org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.getDefaultInstance();
-        private com.google.protobuf.SingleFieldBuilder<
-            org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.Builder, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.PayloadOrBuilder> dataBuilder_;
-        /**
-         * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-         */
-        public boolean hasData() {
-          return ((bitField0_ & 0x00000004) == 0x00000004);
-        }
-        /**
-         * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-         */
-        public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload getData() {
-          if (dataBuilder_ == null) {
-            return data_;
-          } else {
-            return dataBuilder_.getMessage();
-          }
-        }
-        /**
-         * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-         */
-        public Builder setData(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload value) {
-          if (dataBuilder_ == null) {
-            if (value == null) {
-              throw new NullPointerException();
-            }
-            data_ = value;
-            onChanged();
-          } else {
-            dataBuilder_.setMessage(value);
-          }
-          bitField0_ |= 0x00000004;
-          return this;
-        }
-        /**
-         * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-         */
-        public Builder setData(
-            org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.Builder builderForValue) {
-          if (dataBuilder_ == null) {
-            data_ = builderForValue.build();
-            onChanged();
-          } else {
-            dataBuilder_.setMessage(builderForValue.build());
-          }
-          bitField0_ |= 0x00000004;
-          return this;
-        }
-        /**
-         * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-         */
-        public Builder mergeData(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload value) {
-          if (dataBuilder_ == null) {
-            if (((bitField0_ & 0x00000004) == 0x00000004) &&
-                data_ != org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.getDefaultInstance()) {
-              data_ =
-                org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.newBuilder(data_).mergeFrom(value).buildPartial();
-            } else {
-              data_ = value;
-            }
-            onChanged();
-          } else {
-            dataBuilder_.mergeFrom(value);
-          }
-          bitField0_ |= 0x00000004;
-          return this;
-        }
-        /**
-         * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-         */
-        public Builder clearData() {
-          if (dataBuilder_ == null) {
-            data_ = org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.getDefaultInstance();
-            onChanged();
-          } else {
-            dataBuilder_.clear();
-          }
-          bitField0_ = (bitField0_ & ~0x00000004);
-          return this;
-        }
-        /**
-         * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-         */
-        public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.Builder getDataBuilder() {
-          bitField0_ |= 0x00000004;
-          onChanged();
-          return getDataFieldBuilder().getBuilder();
-        }
-        /**
-         * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-         */
-        public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.PayloadOrBuilder getDataOrBuilder() {
-          if (dataBuilder_ != null) {
-            return dataBuilder_.getMessageOrBuilder();
-          } else {
-            return data_;
-          }
-        }
-        /**
-         * <code>optional .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload data = 3;</code>
-         */
-        private com.google.protobuf.SingleFieldBuilder<
-            org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.Builder, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.PayloadOrBuilder>
-            getDataFieldBuilder() {
-          if (dataBuilder_ == null) {
-            dataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-                org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload.Builder, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.PayloadOrBuilder>(
-                    data_,
-                    getParentForChildren(),
-                    isClean());
-            data_ = null;
-          }
-          return dataBuilder_;
-        }
-
-        // @@protoc_insertion_point(builder_scope:org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry)
-      }
-
-      static {
-        defaultInstance = new ReplicatedLogEntry(true);
-        defaultInstance.initFields();
-      }
-
-      // @@protoc_insertion_point(class_scope:org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry)
-    }
-
-    private int bitField0_;
-    // optional int64 term = 1;
-    public static final int TERM_FIELD_NUMBER = 1;
-    private long term_;
-    /**
-     * <code>optional int64 term = 1;</code>
-     */
-    public boolean hasTerm() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>optional int64 term = 1;</code>
-     */
-    public long getTerm() {
-      return term_;
-    }
-
-    // optional string leaderId = 2;
-    public static final int LEADERID_FIELD_NUMBER = 2;
-    private java.lang.Object leaderId_;
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    public boolean hasLeaderId() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    public java.lang.String getLeaderId() {
-      java.lang.Object ref = leaderId_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs =
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          leaderId_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    public com.google.protobuf.ByteString
-        getLeaderIdBytes() {
-      java.lang.Object ref = leaderId_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        leaderId_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional int64 prevLogIndex = 3;
-    public static final int PREVLOGINDEX_FIELD_NUMBER = 3;
-    private long prevLogIndex_;
-    /**
-     * <code>optional int64 prevLogIndex = 3;</code>
-     */
-    public boolean hasPrevLogIndex() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional int64 prevLogIndex = 3;</code>
-     */
-    public long getPrevLogIndex() {
-      return prevLogIndex_;
-    }
-
-    // optional int64 prevLogTerm = 4;
-    public static final int PREVLOGTERM_FIELD_NUMBER = 4;
-    private long prevLogTerm_;
-    /**
-     * <code>optional int64 prevLogTerm = 4;</code>
-     */
-    public boolean hasPrevLogTerm() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>optional int64 prevLogTerm = 4;</code>
-     */
-    public long getPrevLogTerm() {
-      return prevLogTerm_;
-    }
-
-    // repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;
-    public static final int LOGENTRIES_FIELD_NUMBER = 5;
-    private java.util.List<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry> logEntries_;
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    public java.util.List<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry> getLogEntriesList() {
-      return logEntries_;
-    }
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder>
-        getLogEntriesOrBuilderList() {
-      return logEntries_;
-    }
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    public int getLogEntriesCount() {
-      return logEntries_.size();
-    }
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry getLogEntries(int index) {
-      return logEntries_.get(index);
-    }
-    /**
-     * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-     */
-    public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder getLogEntriesOrBuilder(
-        int index) {
-      return logEntries_.get(index);
-    }
-
-    // optional int64 leaderCommit = 6;
-    public static final int LEADERCOMMIT_FIELD_NUMBER = 6;
-    private long leaderCommit_;
-    /**
-     * <code>optional int64 leaderCommit = 6;</code>
-     */
-    public boolean hasLeaderCommit() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>optional int64 leaderCommit = 6;</code>
-     */
-    public long getLeaderCommit() {
-      return leaderCommit_;
-    }
-
-    private void initFields() {
-      term_ = 0L;
-      leaderId_ = "";
-      prevLogIndex_ = 0L;
-      prevLogTerm_ = 0L;
-      logEntries_ = java.util.Collections.emptyList();
-      leaderCommit_ = 0L;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      for (int i = 0; i < getLogEntriesCount(); i++) {
-        if (!getLogEntries(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeInt64(1, term_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, getLeaderIdBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt64(3, prevLogIndex_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeInt64(4, prevLogTerm_);
-      }
-      for (int i = 0; i < logEntries_.size(); i++) {
-        output.writeMessage(5, logEntries_.get(i));
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeInt64(6, leaderCommit_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(1, term_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, getLeaderIdBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(3, prevLogIndex_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(4, prevLogTerm_);
-      }
-      for (int i = 0; i < logEntries_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(5, logEntries_.get(i));
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(6, leaderCommit_);
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code org.opendaylight.controller.cluster.raft.AppendEntries}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntriesOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.class, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.Builder.class);
-      }
-
-      // Construct using org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getLogEntriesFieldBuilder();
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        term_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        leaderId_ = "";
-        bitField0_ = (bitField0_ & ~0x00000002);
-        prevLogIndex_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        prevLogTerm_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        if (logEntriesBuilder_ == null) {
-          logEntries_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000010);
-        } else {
-          logEntriesBuilder_.clear();
-        }
-        leaderCommit_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000020);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_descriptor;
-      }
-
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries getDefaultInstanceForType() {
-        return org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.getDefaultInstance();
-      }
-
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries build() {
-        org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries buildPartial() {
-        org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries result = new org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.term_ = term_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.leaderId_ = leaderId_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.prevLogIndex_ = prevLogIndex_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.prevLogTerm_ = prevLogTerm_;
-        if (logEntriesBuilder_ == null) {
-          if (((bitField0_ & 0x00000010) == 0x00000010)) {
-            logEntries_ = java.util.Collections.unmodifiableList(logEntries_);
-            bitField0_ = (bitField0_ & ~0x00000010);
-          }
-          result.logEntries_ = logEntries_;
-        } else {
-          result.logEntries_ = logEntriesBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.leaderCommit_ = leaderCommit_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries) {
-          return mergeFrom((org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries other) {
-        if (other == org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.getDefaultInstance()) return this;
-        if (other.hasTerm()) {
-          setTerm(other.getTerm());
-        }
-        if (other.hasLeaderId()) {
-          bitField0_ |= 0x00000002;
-          leaderId_ = other.leaderId_;
-          onChanged();
-        }
-        if (other.hasPrevLogIndex()) {
-          setPrevLogIndex(other.getPrevLogIndex());
-        }
-        if (other.hasPrevLogTerm()) {
-          setPrevLogTerm(other.getPrevLogTerm());
-        }
-        if (logEntriesBuilder_ == null) {
-          if (!other.logEntries_.isEmpty()) {
-            if (logEntries_.isEmpty()) {
-              logEntries_ = other.logEntries_;
-              bitField0_ = (bitField0_ & ~0x00000010);
-            } else {
-              ensureLogEntriesIsMutable();
-              logEntries_.addAll(other.logEntries_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.logEntries_.isEmpty()) {
-            if (logEntriesBuilder_.isEmpty()) {
-              logEntriesBuilder_.dispose();
-              logEntriesBuilder_ = null;
-              logEntries_ = other.logEntries_;
-              bitField0_ = (bitField0_ & ~0x00000010);
-              logEntriesBuilder_ =
-                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
-                   getLogEntriesFieldBuilder() : null;
-            } else {
-              logEntriesBuilder_.addAllMessages(other.logEntries_);
-            }
-          }
-        }
-        if (other.hasLeaderCommit()) {
-          setLeaderCommit(other.getLeaderCommit());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        for (int i = 0; i < getLogEntriesCount(); i++) {
-          if (!getLogEntries(i).isInitialized()) {
-
-            return false;
-          }
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // optional int64 term = 1;
-      private long term_ ;
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      public boolean hasTerm() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      public long getTerm() {
-        return term_;
-      }
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      public Builder setTerm(long value) {
-        bitField0_ |= 0x00000001;
-        term_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      public Builder clearTerm() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        term_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional string leaderId = 2;
-      private java.lang.Object leaderId_ = "";
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      public boolean hasLeaderId() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      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)
-              .toStringUtf8();
-          leaderId_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      public com.google.protobuf.ByteString
-          getLeaderIdBytes() {
-        java.lang.Object ref = leaderId_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          leaderId_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      public Builder setLeaderId(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        leaderId_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      public Builder clearLeaderId() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        leaderId_ = getDefaultInstance().getLeaderId();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      public Builder setLeaderIdBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        leaderId_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 prevLogIndex = 3;
-      private long prevLogIndex_ ;
-      /**
-       * <code>optional int64 prevLogIndex = 3;</code>
-       */
-      public boolean hasPrevLogIndex() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>optional int64 prevLogIndex = 3;</code>
-       */
-      public long getPrevLogIndex() {
-        return prevLogIndex_;
-      }
-      /**
-       * <code>optional int64 prevLogIndex = 3;</code>
-       */
-      public Builder setPrevLogIndex(long value) {
-        bitField0_ |= 0x00000004;
-        prevLogIndex_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 prevLogIndex = 3;</code>
-       */
-      public Builder clearPrevLogIndex() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        prevLogIndex_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 prevLogTerm = 4;
-      private long prevLogTerm_ ;
-      /**
-       * <code>optional int64 prevLogTerm = 4;</code>
-       */
-      public boolean hasPrevLogTerm() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>optional int64 prevLogTerm = 4;</code>
-       */
-      public long getPrevLogTerm() {
-        return prevLogTerm_;
-      }
-      /**
-       * <code>optional int64 prevLogTerm = 4;</code>
-       */
-      public Builder setPrevLogTerm(long value) {
-        bitField0_ |= 0x00000008;
-        prevLogTerm_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 prevLogTerm = 4;</code>
-       */
-      public Builder clearPrevLogTerm() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        prevLogTerm_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;
-      private java.util.List<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry> logEntries_ =
-        java.util.Collections.emptyList();
-      private void ensureLogEntriesIsMutable() {
-        if (!((bitField0_ & 0x00000010) == 0x00000010)) {
-          logEntries_ = new java.util.ArrayList<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry>(logEntries_);
-          bitField0_ |= 0x00000010;
-         }
-      }
-
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder> logEntriesBuilder_;
-
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry> getLogEntriesList() {
-        if (logEntriesBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(logEntries_);
-        } else {
-          return logEntriesBuilder_.getMessageList();
-        }
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public int getLogEntriesCount() {
-        if (logEntriesBuilder_ == null) {
-          return logEntries_.size();
-        } else {
-          return logEntriesBuilder_.getCount();
-        }
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry getLogEntries(int index) {
-        if (logEntriesBuilder_ == null) {
-          return logEntries_.get(index);
-        } else {
-          return logEntriesBuilder_.getMessage(index);
-        }
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public Builder setLogEntries(
-          int index, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry value) {
-        if (logEntriesBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureLogEntriesIsMutable();
-          logEntries_.set(index, value);
-          onChanged();
-        } else {
-          logEntriesBuilder_.setMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public Builder setLogEntries(
-          int index, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder builderForValue) {
-        if (logEntriesBuilder_ == null) {
-          ensureLogEntriesIsMutable();
-          logEntries_.set(index, builderForValue.build());
-          onChanged();
-        } else {
-          logEntriesBuilder_.setMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public Builder addLogEntries(org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry value) {
-        if (logEntriesBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureLogEntriesIsMutable();
-          logEntries_.add(value);
-          onChanged();
-        } else {
-          logEntriesBuilder_.addMessage(value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public Builder addLogEntries(
-          int index, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry value) {
-        if (logEntriesBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureLogEntriesIsMutable();
-          logEntries_.add(index, value);
-          onChanged();
-        } else {
-          logEntriesBuilder_.addMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public Builder addLogEntries(
-          org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder builderForValue) {
-        if (logEntriesBuilder_ == null) {
-          ensureLogEntriesIsMutable();
-          logEntries_.add(builderForValue.build());
-          onChanged();
-        } else {
-          logEntriesBuilder_.addMessage(builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public Builder addLogEntries(
-          int index, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder builderForValue) {
-        if (logEntriesBuilder_ == null) {
-          ensureLogEntriesIsMutable();
-          logEntries_.add(index, builderForValue.build());
-          onChanged();
-        } else {
-          logEntriesBuilder_.addMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public Builder addAllLogEntries(
-          java.lang.Iterable<? extends org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry> values) {
-        if (logEntriesBuilder_ == null) {
-          ensureLogEntriesIsMutable();
-          super.addAll(values, logEntries_);
-          onChanged();
-        } else {
-          logEntriesBuilder_.addAllMessages(values);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public Builder clearLogEntries() {
-        if (logEntriesBuilder_ == null) {
-          logEntries_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000010);
-          onChanged();
-        } else {
-          logEntriesBuilder_.clear();
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public Builder removeLogEntries(int index) {
-        if (logEntriesBuilder_ == null) {
-          ensureLogEntriesIsMutable();
-          logEntries_.remove(index);
-          onChanged();
-        } else {
-          logEntriesBuilder_.remove(index);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder getLogEntriesBuilder(
-          int index) {
-        return getLogEntriesFieldBuilder().getBuilder(index);
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder getLogEntriesOrBuilder(
-          int index) {
-        if (logEntriesBuilder_ == null) {
-          return logEntries_.get(index);  } else {
-          return logEntriesBuilder_.getMessageOrBuilder(index);
-        }
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder>
-           getLogEntriesOrBuilderList() {
-        if (logEntriesBuilder_ != null) {
-          return logEntriesBuilder_.getMessageOrBuilderList();
-        } else {
-          return java.util.Collections.unmodifiableList(logEntries_);
-        }
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder addLogEntriesBuilder() {
-        return getLogEntriesFieldBuilder().addBuilder(
-            org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.getDefaultInstance());
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder addLogEntriesBuilder(
-          int index) {
-        return getLogEntriesFieldBuilder().addBuilder(
-            index, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.getDefaultInstance());
-      }
-      /**
-       * <code>repeated .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry logEntries = 5;</code>
-       */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder>
-           getLogEntriesBuilderList() {
-        return getLogEntriesFieldBuilder().getBuilderList();
-      }
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder>
-          getLogEntriesFieldBuilder() {
-        if (logEntriesBuilder_ == null) {
-          logEntriesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
-              org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Builder, org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages.AppendEntries.ReplicatedLogEntryOrBuilder>(
-                  logEntries_,
-                  ((bitField0_ & 0x00000010) == 0x00000010),
-                  getParentForChildren(),
-                  isClean());
-          logEntries_ = null;
-        }
-        return logEntriesBuilder_;
-      }
-
-      // optional int64 leaderCommit = 6;
-      private long leaderCommit_ ;
-      /**
-       * <code>optional int64 leaderCommit = 6;</code>
-       */
-      public boolean hasLeaderCommit() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
-      }
-      /**
-       * <code>optional int64 leaderCommit = 6;</code>
-       */
-      public long getLeaderCommit() {
-        return leaderCommit_;
-      }
-      /**
-       * <code>optional int64 leaderCommit = 6;</code>
-       */
-      public Builder setLeaderCommit(long value) {
-        bitField0_ |= 0x00000020;
-        leaderCommit_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 leaderCommit = 6;</code>
-       */
-      public Builder clearLeaderCommit() {
-        bitField0_ = (bitField0_ & ~0x00000020);
-        leaderCommit_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:org.opendaylight.controller.cluster.raft.AppendEntries)
-    }
-
-    static {
-      defaultInstance = new AppendEntries(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:org.opendaylight.controller.cluster.raft.AppendEntries)
-  }
-
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_fieldAccessorTable;
-
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
-  }
-  private static com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\033AppendEntriesMessages.proto\022(org.opend" +
-      "aylight.controller.cluster.raft\"\227\003\n\rAppe" +
-      "ndEntries\022\014\n\004term\030\001 \001(\003\022\020\n\010leaderId\030\002 \001(" +
-      "\t\022\024\n\014prevLogIndex\030\003 \001(\003\022\023\n\013prevLogTerm\030\004" +
-      " \001(\003\022^\n\nlogEntries\030\005 \003(\0132J.org.opendayli" +
-      "ght.controller.cluster.raft.AppendEntrie" +
-      "s.ReplicatedLogEntry\022\024\n\014leaderCommit\030\006 \001" +
-      "(\003\032\304\001\n\022ReplicatedLogEntry\022\014\n\004term\030\001 \001(\003\022" +
-      "\r\n\005index\030\002 \001(\003\022`\n\004data\030\003 \001(\0132R.org.opend" +
-      "aylight.controller.cluster.raft.AppendEn",
-      "tries.ReplicatedLogEntry.Payload\032/\n\007Payl" +
-      "oad\022\036\n\026clientPayloadClassName\030\001 \001(\t*\004\010\002\020" +
-      "eBV\n;org.opendaylight.controller.protobu" +
-      "ff.messages.cluster.raftB\025AppendEntriesM" +
-      "essagesH\001"
-    };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
-        public com.google.protobuf.ExtensionRegistry assignDescriptors(
-            com.google.protobuf.Descriptors.FileDescriptor root) {
-          descriptor = root;
-          internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_descriptor =
-            getDescriptor().getMessageTypes().get(0);
-          internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_descriptor,
-              new java.lang.String[] { "Term", "LeaderId", "PrevLogIndex", "PrevLogTerm", "LogEntries", "LeaderCommit", });
-          internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_descriptor =
-            internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_descriptor.getNestedTypes().get(0);
-          internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_descriptor,
-              new java.lang.String[] { "Term", "Index", "Data", });
-          internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_descriptor =
-            internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_descriptor.getNestedTypes().get(0);
-          internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_org_opendaylight_controller_cluster_raft_AppendEntries_ReplicatedLogEntry_Payload_descriptor,
-              new java.lang.String[] { "ClientPayloadClassName", });
-          return null;
-        }
-      };
-    com.google.protobuf.Descriptors.FileDescriptor
-      .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
-        }, assigner);
-  }
-
-  // @@protoc_insertion_point(outer_class_scope)
-}
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
deleted file mode 100644 (file)
index 03c381a..0000000
+++ /dev/null
@@ -1,1157 +0,0 @@
-// Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: InstallSnapshot.proto
-
-package org.opendaylight.controller.protobuff.messages.cluster.raft;
-
-@Deprecated
-public final class InstallSnapshotMessages {
-  private InstallSnapshotMessages() {}
-  public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
-  }
-  public interface InstallSnapshotOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // optional int64 term = 1;
-    /**
-     * <code>optional int64 term = 1;</code>
-     */
-    boolean hasTerm();
-    /**
-     * <code>optional int64 term = 1;</code>
-     */
-    long getTerm();
-
-    // optional string leaderId = 2;
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    boolean hasLeaderId();
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    java.lang.String getLeaderId();
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    com.google.protobuf.ByteString
-        getLeaderIdBytes();
-
-    // optional int64 lastIncludedIndex = 3;
-    /**
-     * <code>optional int64 lastIncludedIndex = 3;</code>
-     */
-    boolean hasLastIncludedIndex();
-    /**
-     * <code>optional int64 lastIncludedIndex = 3;</code>
-     */
-    long getLastIncludedIndex();
-
-    // optional int64 lastIncludedTerm = 4;
-    /**
-     * <code>optional int64 lastIncludedTerm = 4;</code>
-     */
-    boolean hasLastIncludedTerm();
-    /**
-     * <code>optional int64 lastIncludedTerm = 4;</code>
-     */
-    long getLastIncludedTerm();
-
-    // optional bytes data = 5;
-    /**
-     * <code>optional bytes data = 5;</code>
-     */
-    boolean hasData();
-    /**
-     * <code>optional bytes data = 5;</code>
-     */
-    com.google.protobuf.ByteString getData();
-
-    // optional int32 chunkIndex = 6;
-    /**
-     * <code>optional int32 chunkIndex = 6;</code>
-     */
-    boolean hasChunkIndex();
-    /**
-     * <code>optional int32 chunkIndex = 6;</code>
-     */
-    int getChunkIndex();
-
-    // optional int32 totalChunks = 7;
-    /**
-     * <code>optional int32 totalChunks = 7;</code>
-     */
-    boolean hasTotalChunks();
-    /**
-     * <code>optional int32 totalChunks = 7;</code>
-     */
-    int getTotalChunks();
-
-    // optional int32 lastChunkHashCode = 8;
-    /**
-     * <code>optional int32 lastChunkHashCode = 8;</code>
-     */
-    boolean hasLastChunkHashCode();
-    /**
-     * <code>optional int32 lastChunkHashCode = 8;</code>
-     */
-    int getLastChunkHashCode();
-  }
-  /**
-   * Protobuf type {@code org.opendaylight.controller.cluster.raft.InstallSnapshot}
-   */
-  public static final class InstallSnapshot extends
-      com.google.protobuf.GeneratedMessage
-      implements InstallSnapshotOrBuilder {
-    // Use InstallSnapshot.newBuilder() to construct.
-    private InstallSnapshot(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private InstallSnapshot(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final InstallSnapshot defaultInstance;
-    public static InstallSnapshot getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    @Override
-    public InstallSnapshot getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private InstallSnapshot(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 8: {
-              bitField0_ |= 0x00000001;
-              term_ = input.readInt64();
-              break;
-            }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              leaderId_ = input.readBytes();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              lastIncludedIndex_ = input.readInt64();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              lastIncludedTerm_ = input.readInt64();
-              break;
-            }
-            case 42: {
-              bitField0_ |= 0x00000010;
-              data_ = input.readBytes();
-              break;
-            }
-            case 48: {
-              bitField0_ |= 0x00000020;
-              chunkIndex_ = input.readInt32();
-              break;
-            }
-            case 56: {
-              bitField0_ |= 0x00000040;
-              totalChunks_ = input.readInt32();
-              break;
-            }
-            case 64: {
-              bitField0_ |= 0x00000080;
-              lastChunkHashCode_ = input.readInt32();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      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
-          .ensureFieldAccessorsInitialized(
-              org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot.class, org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<InstallSnapshot> PARSER =
-        new com.google.protobuf.AbstractParser<InstallSnapshot>() {
-      @Override
-    public InstallSnapshot parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new InstallSnapshot(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<InstallSnapshot> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // optional int64 term = 1;
-    public static final int TERM_FIELD_NUMBER = 1;
-    private long term_;
-    /**
-     * <code>optional int64 term = 1;</code>
-     */
-    @Override
-    public boolean hasTerm() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>optional int64 term = 1;</code>
-     */
-    @Override
-    public long getTerm() {
-      return term_;
-    }
-
-    // optional string leaderId = 2;
-    public static final int LEADERID_FIELD_NUMBER = 2;
-    private java.lang.Object leaderId_;
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    @Override
-    public boolean hasLeaderId() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    @Override
-    public java.lang.String getLeaderId() {
-      java.lang.Object ref = leaderId_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs =
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          leaderId_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string leaderId = 2;</code>
-     */
-    @Override
-    public com.google.protobuf.ByteString
-        getLeaderIdBytes() {
-      java.lang.Object ref = leaderId_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        leaderId_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional int64 lastIncludedIndex = 3;
-    public static final int LASTINCLUDEDINDEX_FIELD_NUMBER = 3;
-    private long lastIncludedIndex_;
-    /**
-     * <code>optional int64 lastIncludedIndex = 3;</code>
-     */
-    @Override
-    public boolean hasLastIncludedIndex() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional int64 lastIncludedIndex = 3;</code>
-     */
-    @Override
-    public long getLastIncludedIndex() {
-      return lastIncludedIndex_;
-    }
-
-    // optional int64 lastIncludedTerm = 4;
-    public static final int LASTINCLUDEDTERM_FIELD_NUMBER = 4;
-    private long lastIncludedTerm_;
-    /**
-     * <code>optional int64 lastIncludedTerm = 4;</code>
-     */
-    @Override
-    public boolean hasLastIncludedTerm() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>optional int64 lastIncludedTerm = 4;</code>
-     */
-    @Override
-    public long getLastIncludedTerm() {
-      return lastIncludedTerm_;
-    }
-
-    // optional bytes data = 5;
-    public static final int DATA_FIELD_NUMBER = 5;
-    private com.google.protobuf.ByteString data_;
-    /**
-     * <code>optional bytes data = 5;</code>
-     */
-    @Override
-    public boolean hasData() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>optional bytes data = 5;</code>
-     */
-    @Override
-    public com.google.protobuf.ByteString getData() {
-      return data_;
-    }
-
-    // optional int32 chunkIndex = 6;
-    public static final int CHUNKINDEX_FIELD_NUMBER = 6;
-    private int chunkIndex_;
-    /**
-     * <code>optional int32 chunkIndex = 6;</code>
-     */
-    @Override
-    public boolean hasChunkIndex() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
-    }
-    /**
-     * <code>optional int32 chunkIndex = 6;</code>
-     */
-    @Override
-    public int getChunkIndex() {
-      return chunkIndex_;
-    }
-
-    // optional int32 totalChunks = 7;
-    public static final int TOTALCHUNKS_FIELD_NUMBER = 7;
-    private int totalChunks_;
-    /**
-     * <code>optional int32 totalChunks = 7;</code>
-     */
-    @Override
-    public boolean hasTotalChunks() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
-    }
-    /**
-     * <code>optional int32 totalChunks = 7;</code>
-     */
-    @Override
-    public int getTotalChunks() {
-      return totalChunks_;
-    }
-
-    // optional int32 lastChunkHashCode = 8;
-    public static final int LASTCHUNKHASHCODE_FIELD_NUMBER = 8;
-    private int lastChunkHashCode_;
-    /**
-     * <code>optional int32 lastChunkHashCode = 8;</code>
-     */
-    @Override
-    public boolean hasLastChunkHashCode() {
-      return ((bitField0_ & 0x00000080) == 0x00000080);
-    }
-    /**
-     * <code>optional int32 lastChunkHashCode = 8;</code>
-     */
-    @Override
-    public int getLastChunkHashCode() {
-      return lastChunkHashCode_;
-    }
-
-    private void initFields() {
-      term_ = 0L;
-      leaderId_ = "";
-      lastIncludedIndex_ = 0L;
-      lastIncludedTerm_ = 0L;
-      data_ = com.google.protobuf.ByteString.EMPTY;
-      chunkIndex_ = 0;
-      totalChunks_ = 0;
-      lastChunkHashCode_ = 0;
-    }
-    private byte memoizedIsInitialized = -1;
-    @Override
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) {
-        return isInitialized == 1;
-    }
-
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    @Override
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeInt64(1, term_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, getLeaderIdBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt64(3, lastIncludedIndex_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeInt64(4, lastIncludedTerm_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeBytes(5, data_);
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeInt32(6, chunkIndex_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeInt32(7, totalChunks_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        output.writeInt32(8, lastChunkHashCode_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    @Override
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) {
-        return size;
-    }
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(1, term_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, getLeaderIdBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(3, lastIncludedIndex_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(4, lastIncludedTerm_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(5, data_);
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(6, chunkIndex_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(7, totalChunks_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(8, lastChunkHashCode_);
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    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
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code org.opendaylight.controller.cluster.raft.InstallSnapshot}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshotOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        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
-            .ensureFieldAccessorsInitialized(
-                org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot.class, org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot.Builder.class);
-      }
-
-      // Construct using org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      @Override
-    public Builder clear() {
-        super.clear();
-        term_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        leaderId_ = "";
-        bitField0_ = (bitField0_ & ~0x00000002);
-        lastIncludedIndex_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        lastIncludedTerm_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        data_ = com.google.protobuf.ByteString.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        chunkIndex_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000020);
-        totalChunks_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        lastChunkHashCode_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000080);
-        return this;
-      }
-
-      @Override
-    public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      @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;
-      }
-
-      @Override
-    public org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot getDefaultInstanceForType() {
-        return org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot.getDefaultInstance();
-      }
-
-      @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);
-        }
-        return result;
-      }
-
-      @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;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.term_ = term_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.leaderId_ = leaderId_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.lastIncludedIndex_ = lastIncludedIndex_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.lastIncludedTerm_ = lastIncludedTerm_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.data_ = data_;
-        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
-          to_bitField0_ |= 0x00000020;
-        }
-        result.chunkIndex_ = chunkIndex_;
-        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
-          to_bitField0_ |= 0x00000040;
-        }
-        result.totalChunks_ = totalChunks_;
-        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
-          to_bitField0_ |= 0x00000080;
-        }
-        result.lastChunkHashCode_ = lastChunkHashCode_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      @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 {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      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.hasTerm()) {
-          setTerm(other.getTerm());
-        }
-        if (other.hasLeaderId()) {
-          bitField0_ |= 0x00000002;
-          leaderId_ = other.leaderId_;
-          onChanged();
-        }
-        if (other.hasLastIncludedIndex()) {
-          setLastIncludedIndex(other.getLastIncludedIndex());
-        }
-        if (other.hasLastIncludedTerm()) {
-          setLastIncludedTerm(other.getLastIncludedTerm());
-        }
-        if (other.hasData()) {
-          setData(other.getData());
-        }
-        if (other.hasChunkIndex()) {
-          setChunkIndex(other.getChunkIndex());
-        }
-        if (other.hasTotalChunks()) {
-          setTotalChunks(other.getTotalChunks());
-        }
-        if (other.hasLastChunkHashCode()) {
-          setLastChunkHashCode(other.getLastChunkHashCode());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      @Override
-    public final boolean isInitialized() {
-        return true;
-      }
-
-      @Override
-    public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages.InstallSnapshot) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // optional int64 term = 1;
-      private long term_ ;
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      @Override
-    public boolean hasTerm() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      @Override
-    public long getTerm() {
-        return term_;
-      }
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      public Builder setTerm(long value) {
-        bitField0_ |= 0x00000001;
-        term_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 term = 1;</code>
-       */
-      public Builder clearTerm() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        term_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional string leaderId = 2;
-      private java.lang.Object leaderId_ = "";
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      @Override
-    public boolean hasLeaderId() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      @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)
-              .toStringUtf8();
-          leaderId_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      @Override
-    public com.google.protobuf.ByteString
-          getLeaderIdBytes() {
-        java.lang.Object ref = leaderId_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          leaderId_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      public Builder setLeaderId(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        leaderId_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      public Builder clearLeaderId() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        leaderId_ = getDefaultInstance().getLeaderId();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string leaderId = 2;</code>
-       */
-      public Builder setLeaderIdBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        leaderId_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 lastIncludedIndex = 3;
-      private long lastIncludedIndex_ ;
-      /**
-       * <code>optional int64 lastIncludedIndex = 3;</code>
-       */
-      @Override
-    public boolean hasLastIncludedIndex() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>optional int64 lastIncludedIndex = 3;</code>
-       */
-      @Override
-    public long getLastIncludedIndex() {
-        return lastIncludedIndex_;
-      }
-      /**
-       * <code>optional int64 lastIncludedIndex = 3;</code>
-       */
-      public Builder setLastIncludedIndex(long value) {
-        bitField0_ |= 0x00000004;
-        lastIncludedIndex_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 lastIncludedIndex = 3;</code>
-       */
-      public Builder clearLastIncludedIndex() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        lastIncludedIndex_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 lastIncludedTerm = 4;
-      private long lastIncludedTerm_ ;
-      /**
-       * <code>optional int64 lastIncludedTerm = 4;</code>
-       */
-      @Override
-    public boolean hasLastIncludedTerm() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>optional int64 lastIncludedTerm = 4;</code>
-       */
-      @Override
-    public long getLastIncludedTerm() {
-        return lastIncludedTerm_;
-      }
-      /**
-       * <code>optional int64 lastIncludedTerm = 4;</code>
-       */
-      public Builder setLastIncludedTerm(long value) {
-        bitField0_ |= 0x00000008;
-        lastIncludedTerm_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 lastIncludedTerm = 4;</code>
-       */
-      public Builder clearLastIncludedTerm() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        lastIncludedTerm_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional bytes data = 5;
-      private com.google.protobuf.ByteString data_ = com.google.protobuf.ByteString.EMPTY;
-      /**
-       * <code>optional bytes data = 5;</code>
-       */
-      @Override
-    public boolean hasData() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <code>optional bytes data = 5;</code>
-       */
-      @Override
-    public com.google.protobuf.ByteString getData() {
-        return data_;
-      }
-      /**
-       * <code>optional bytes data = 5;</code>
-       */
-      public Builder setData(com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000010;
-        data_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bytes data = 5;</code>
-       */
-      public Builder clearData() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        data_ = getDefaultInstance().getData();
-        onChanged();
-        return this;
-      }
-
-      // optional int32 chunkIndex = 6;
-      private int chunkIndex_ ;
-      /**
-       * <code>optional int32 chunkIndex = 6;</code>
-       */
-      @Override
-    public boolean hasChunkIndex() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
-      }
-      /**
-       * <code>optional int32 chunkIndex = 6;</code>
-       */
-      @Override
-    public int getChunkIndex() {
-        return chunkIndex_;
-      }
-      /**
-       * <code>optional int32 chunkIndex = 6;</code>
-       */
-      public Builder setChunkIndex(int value) {
-        bitField0_ |= 0x00000020;
-        chunkIndex_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int32 chunkIndex = 6;</code>
-       */
-      public Builder clearChunkIndex() {
-        bitField0_ = (bitField0_ & ~0x00000020);
-        chunkIndex_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // optional int32 totalChunks = 7;
-      private int totalChunks_ ;
-      /**
-       * <code>optional int32 totalChunks = 7;</code>
-       */
-      @Override
-    public boolean hasTotalChunks() {
-        return ((bitField0_ & 0x00000040) == 0x00000040);
-      }
-      /**
-       * <code>optional int32 totalChunks = 7;</code>
-       */
-      @Override
-    public int getTotalChunks() {
-        return totalChunks_;
-      }
-      /**
-       * <code>optional int32 totalChunks = 7;</code>
-       */
-      public Builder setTotalChunks(int value) {
-        bitField0_ |= 0x00000040;
-        totalChunks_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int32 totalChunks = 7;</code>
-       */
-      public Builder clearTotalChunks() {
-        bitField0_ = (bitField0_ & ~0x00000040);
-        totalChunks_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // optional int32 lastChunkHashCode = 8;
-      private int lastChunkHashCode_ ;
-      /**
-       * <code>optional int32 lastChunkHashCode = 8;</code>
-       */
-      @Override
-    public boolean hasLastChunkHashCode() {
-        return ((bitField0_ & 0x00000080) == 0x00000080);
-      }
-      /**
-       * <code>optional int32 lastChunkHashCode = 8;</code>
-       */
-      @Override
-    public int getLastChunkHashCode() {
-        return lastChunkHashCode_;
-      }
-      /**
-       * <code>optional int32 lastChunkHashCode = 8;</code>
-       */
-      public Builder setLastChunkHashCode(int value) {
-        bitField0_ |= 0x00000080;
-        lastChunkHashCode_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int32 lastChunkHashCode = 8;</code>
-       */
-      public Builder clearLastChunkHashCode() {
-        bitField0_ = (bitField0_ & ~0x00000080);
-        lastChunkHashCode_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:org.opendaylight.controller.cluster.raft.InstallSnapshot)
-    }
-
-    static {
-      defaultInstance = new InstallSnapshot(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:org.opendaylight.controller.cluster.raft.InstallSnapshot)
-  }
-
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_fieldAccessorTable;
-
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
-  }
-  private static com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\025InstallSnapshot.proto\022(org.opendayligh" +
-      "t.controller.cluster.raft\"\270\001\n\017InstallSna" +
-      "pshot\022\014\n\004term\030\001 \001(\003\022\020\n\010leaderId\030\002 \001(\t\022\031\n" +
-      "\021lastIncludedIndex\030\003 \001(\003\022\030\n\020lastIncluded" +
-      "Term\030\004 \001(\003\022\014\n\004data\030\005 \001(\014\022\022\n\nchunkIndex\030\006" +
-      " \001(\005\022\023\n\013totalChunks\030\007 \001(\005\022\031\n\021lastChunkHa" +
-      "shCode\030\010 \001(\005BX\n;org.opendaylight.control" +
-      "ler.protobuff.messages.cluster.raftB\027Ins" +
-      "tallSnapshotMessagesH\001"
-    };
-    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;
-          internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_descriptor =
-            getDescriptor().getMessageTypes().get(0);
-          internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_org_opendaylight_controller_cluster_raft_InstallSnapshot_descriptor,
-              new java.lang.String[] { "Term", "LeaderId", "LastIncludedIndex", "LastIncludedTerm", "Data", "ChunkIndex", "TotalChunks", "LastChunkHashCode", });
-          return null;
-        }
-      };
-    com.google.protobuf.Descriptors.FileDescriptor
-      .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
-        }, assigner);
-  }
-
-  // @@protoc_insertion_point(outer_class_scope)
-}
diff --git a/opendaylight/md-sal/sal-clustering-commons/src/main/resources/AppendEntriesMessages.proto b/opendaylight/md-sal/sal-clustering-commons/src/main/resources/AppendEntriesMessages.proto
deleted file mode 100644 (file)
index 4df56ff..0000000
+++ /dev/null
@@ -1,30 +0,0 @@
-package org.opendaylight.controller.cluster.raft;
-
-option java_package = "org.opendaylight.controller.protobuff.messages.cluster.raft";
-option java_outer_classname = "AppendEntriesMessages";
-option optimize_for = SPEED;
-
-message AppendEntries {
-    optional int64 term = 1;
-    optional string leaderId = 2;
-    optional int64 prevLogIndex = 3;
-    optional int64 prevLogTerm = 4;
-
-    message ReplicatedLogEntry {
-        optional int64 term = 1;
-        optional int64 index = 2;
-
-        message Payload {
-            optional string clientPayloadClassName = 1;
-            extensions 2 to 100;
-        }
-
-        optional Payload data = 3;
-    }
-
-    repeated ReplicatedLogEntry logEntries = 5;
-    optional int64 leaderCommit = 6;
-
-}
-
-
diff --git a/opendaylight/md-sal/sal-clustering-commons/src/main/resources/InstallSnapshot.proto b/opendaylight/md-sal/sal-clustering-commons/src/main/resources/InstallSnapshot.proto
deleted file mode 100644 (file)
index adb58ae..0000000
+++ /dev/null
@@ -1,16 +0,0 @@
-package org.opendaylight.controller.cluster.raft;
-
-option java_package = "org.opendaylight.controller.protobuff.messages.cluster.raft";
-option java_outer_classname = "InstallSnapshotMessages";
-option optimize_for = SPEED;
-
-message InstallSnapshot {
-    optional int64 term = 1;
-    optional string leaderId = 2;
-    optional int64 lastIncludedIndex = 3;
-    optional int64 lastIncludedTerm = 4;
-    optional bytes data = 5;
-    optional int32 chunkIndex = 6;
-    optional int32 totalChunks = 7;
-    optional int32 lastChunkHashCode = 8;
-}
index ad7836347d27578f6801dfec643834cab4a76f7a..2cd5b97ceb45c3b9c2755337ce09f06932fc93ca 100644 (file)
@@ -45,9 +45,6 @@ public class DummyShard extends UntypedActor{
             sender().tell(new RequestVoteReply(req.getTerm(), true), self());
         } else if(o instanceof AppendEntries) {
             handleAppendEntries((AppendEntries)o);
-        } else if(InstallSnapshot.isSerializedType(o)) {
-            InstallSnapshot req = InstallSnapshot.fromSerializable(o);
-            handleInstallSnapshot(req);
         } else if(o instanceof InstallSnapshot){
             handleInstallSnapshot((InstallSnapshot) o);
         } else {