Add replication capability to Shard 23/9523/5
authorMoiz Raja <moraja@cisco.com>
Thu, 31 Jul 2014 09:02:55 +0000 (02:02 -0700)
committerMoiz Raja <moraja@cisco.com>
Thu, 31 Jul 2014 14:20:28 +0000 (07:20 -0700)
This commit integrates the distributed data store with our Raft implementation.

Shard now extends RaftActor which provides it the replication capabilities required.

Other notable changes are,
- The FindPrimary algorithm has been changed to find the first replica for a shard.
  The shard then forwards requests to create a transaction or transaction chain to
  the leader
- Changed the package name for Raft internal messages from "internal" to "base" to be
  more BND tool friendly
- Fix some issues with Serialization of Raft messages
- Create a NoOpTransaction when no Primary can be found. The commit for this transaction
  will always fail. The NoOpTransaction returns absent for reads in all cases.
- Add PeerAddressResolution capability to Raft. What this basically does is given a static
  configuration where a shard has 'n' peers, you can pass the names of those peers
  to the shard and resolve their addresses at a later time. This allows the Shard to ensure
  consensus even in a situation where it is the first one to come up but it's peers are still
  not running

Change-Id: I3087deb5eb4418cd629a707ba14f43858db1f463
Signed-off-by: Moiz Raja <moraja@cisco.com>
59 files changed:
opendaylight/md-sal/pom.xml
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContext.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContextImpl.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplySnapshot.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplySnapshot.java with 89% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplyState.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplyState.java with 94% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/CommitEntry.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/CommitEntry.java with 86% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ElectionTimeout.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ElectionTimeout.java with 85% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/PersistEntry.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/PersistEntry.java with 87% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/Replicate.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/Replicate.java with 94% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SaveSnapshot.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SaveSnapshot.java with 88% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendHeartBeat.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendHeartBeat.java with 88% similarity]
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendInstallSnapshot.java [moved from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendInstallSnapshot.java with 85% similarity]
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/behaviors/Leader.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.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/main/java/org/opendaylight/controller/cluster/raft/messages/RaftRPC.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java
opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/CandidateTest.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java
opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java
opendaylight/md-sal/sal-distributed-datastore/pom.xml
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/CompositeModificationPayload.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Configuration.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpCohort.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/PrimaryNotFoundException.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/TimeoutException.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/messages/PeerAddressResolved.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractActorTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreIntegrationTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/resources/module-shards.conf
opendaylight/md-sal/sal-distributed-datastore/src/test/resources/modules.conf
opendaylight/md-sal/sal-protocolbuffer-encoding/pom.xml
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/mdsal/CompositeModificationPayload.java [new file with mode: 0644]
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/cohort3pc/ThreePhaseCommitCohortMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/NormalizedNodeMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/SimpleNormalizedNodeMessage.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/datachange/notification/DataChangeListenerMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/persistent/PersistentMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/registration/ListenerRegistrationMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/shard/ShardManagerMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionChainMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionMessages.java
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/CompositeModificationPayload.proto [new file with mode: 0644]
opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/Persistent.proto

index cadd8ca87d784095c9a9a6bc5f08b396f399691d..14d4f24d1c590cbfdffe3b3b5219b0b87e965f56 100644 (file)
@@ -66,7 +66,7 @@
     <!--sal-protocolbuffer-encoding-->
     <module>sal-protocolbuffer-encoding</module>
 
-    <!--sal-distributed-datastore-->
+    <!-- sal-distributed-datastore -->
     <module>sal-distributed-datastore</module>
 
     <!-- Yang Test Models for MD-SAL -->
index 15239795a8abc2aab0019439e2a64bbab2701afc..b8e9653bc5db92ffa4cb15afd391c6a9ec05cd16 100644 (file)
@@ -19,6 +19,9 @@ import akka.persistence.SaveSnapshotSuccess;
 import akka.persistence.SnapshotOffer;
 import akka.persistence.SnapshotSelectionCriteria;
 import akka.persistence.UntypedPersistentActor;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
 import org.opendaylight.controller.cluster.raft.behaviors.Candidate;
 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
 import org.opendaylight.controller.cluster.raft.behaviors.Leader;
@@ -26,10 +29,7 @@ import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
 import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 
 import java.io.Serializable;
@@ -250,6 +250,24 @@ public abstract class RaftActor extends UntypedPersistentActor {
         return currentBehavior.state();
     }
 
+    /**
+     * setPeerAddress sets the address of a known peer at a later time.
+     * <p>
+     * This is to account for situations where a we know that a peer
+     * exists but we do not know an address up-front. This may also be used in
+     * situations where a known peer starts off in a different location and we
+     * need to change it's address
+     * <p>
+     * Note that if the peerId does not match the list of peers passed to
+     * this actor during construction an IllegalStateException will be thrown.
+     *
+     * @param peerId
+     * @param peerAddress
+     */
+    protected void setPeerAddress(String peerId, String peerAddress){
+        context.setPeerAddress(peerId, peerAddress);
+    }
+
 
 
     /**
@@ -451,7 +469,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
             final String identifier,
             final ReplicatedLogEntry replicatedLogEntry) {
             context.getLogger().debug(
-                "Append log entry and persist " + replicatedLogEntry);
+                "Append log entry and persist {} ", replicatedLogEntry);
             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
             journal.add(replicatedLogEntry);
 
index 7150ec0e6e8fd889f383a4aaace4b250a44061bc..ae9431a43acf576f2769672b49d3ce234014e3c0 100644 (file)
@@ -96,21 +96,27 @@ public interface RaftActorContext {
     ActorSystem getActorSystem();
 
     /**
+     * Get the logger to be used for logging messages
      *
      * @return
      */
     LoggingAdapter getLogger();
 
     /**
-     * Get a mapping of peer id's their addresses
+     * Get a mapping of peerId's to their addresses
+     *
      * @return
+     *
      */
     Map<String, String> getPeerAddresses();
 
     /**
+     * Get the address of the peer as a String. This is the same format in
+     * which a consumer would provide the address
      *
      * @param peerId
-     * @return
+     * @return The address of the peer or null if the address has not yet been
+     *         resolved
      */
     String getPeerAddress(String peerId);
 
@@ -126,4 +132,27 @@ public interface RaftActorContext {
      * @param name
      */
     public void removePeer(String name);
+
+    /**
+     * Given a peerId return the corresponding actor
+     * <p>
+     *
+     *
+     * @param peerId
+     * @return The actorSelection corresponding to the peer or null if the
+     *         address has not yet been resolved
+     */
+    ActorSelection getPeerActorSelection(String peerId);
+
+    /**
+     * Set Peer Address can be called at a later time to change the address of
+     * a known peer.
+     *
+     * <p>
+     * Throws an IllegalStateException if the peer is unknown
+     *
+     * @param peerId
+     * @param peerAddress
+     */
+    void setPeerAddress(String peerId, String peerAddress);
 }
index a0f13280c28f68cb3225499924597578302489c2..833c8a9e8ad1ca7b8fe170aadffc25b3c12b6b24 100644 (file)
@@ -17,6 +17,8 @@ import akka.event.LoggingAdapter;
 
 import java.util.Map;
 
+import static com.google.common.base.Preconditions.checkState;
+
 public class RaftActorContextImpl implements RaftActorContext{
 
     private final ActorRef actor;
@@ -37,7 +39,6 @@ public class RaftActorContextImpl implements RaftActorContext{
 
     private final LoggingAdapter LOG;
 
-
     public RaftActorContextImpl(ActorRef actor, UntypedActorContext context,
         String id,
         ElectionTerm termInformation, long commitIndex,
@@ -110,12 +111,25 @@ public class RaftActorContextImpl implements RaftActorContext{
     }
 
     @Override public void addToPeers(String name, String address) {
-        LOG.debug("Kamal--> addToPeer for:"+name);
         peerAddresses.put(name, address);
     }
 
     @Override public void removePeer(String name) {
-        LOG.debug("Kamal--> removePeer for:"+name);
         peerAddresses.remove(name);
     }
+
+    @Override public ActorSelection getPeerActorSelection(String peerId) {
+        String peerAddress = getPeerAddress(peerId);
+        if(peerAddress != null){
+            return actorSelection(peerAddress);
+        }
+        return null;
+    }
+
+    @Override public void setPeerAddress(String peerId, String peerAddress) {
+        LOG.info("Peer address for peer {} set to {}", peerId, peerAddress);
+        checkState(peerAddresses.containsKey(peerId), peerId + " is unknown");
+
+        peerAddresses.put(peerId, peerAddress);
+    }
 }
@@ -6,7 +6,7 @@
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 
-package org.opendaylight.controller.cluster.raft.internal.messages;
+package org.opendaylight.controller.cluster.raft.base.messages;
 
 import akka.actor.ActorRef;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
@@ -6,7 +6,7 @@
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 
-package org.opendaylight.controller.cluster.raft.internal.messages;
+package org.opendaylight.controller.cluster.raft.base.messages;
 
 import akka.actor.ActorRef;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
index bc87370b159cbd52ce829f1c65df564073e5558d..f7281bb8e3967c3ca2be583adb35210cbd07e87b 100644 (file)
@@ -15,8 +15,8 @@ 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.internal.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
index ecd49012461a7b1ee76f4678c7451a4fa1edbfb1..c125bd32b60a5c5d714ea13e7008417f904d88dc 100644 (file)
@@ -12,16 +12,14 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
-import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
 
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
+import java.util.Set;
 
 /**
  * The behavior of a RaftActor when it is in the CandidateState
@@ -43,24 +41,20 @@ import java.util.Map;
  */
 public class Candidate extends AbstractRaftActorBehavior {
 
-    private final Map<String, ActorSelection> peerToActor = new HashMap<>();
-
     private int voteCount;
 
     private final int votesRequired;
 
+    private final Set<String> peers;
+
     public Candidate(RaftActorContext context) {
         super(context);
 
-        Collection<String> peerPaths = context.getPeerAddresses().values();
+        peers = context.getPeerAddresses().keySet();
 
-        for (String peerPath : peerPaths) {
-            peerToActor.put(peerPath,
-                context.actorSelection(peerPath));
-        }
+        context.getLogger().debug("Election:Candidate has following peers:"+ peers);
 
-        context.getLogger().debug("Election:Candidate has following peers:"+peerToActor.keySet());
-        if(peerPaths.size() > 0) {
+        if(peers.size() > 0) {
             // Votes are required from a majority of the peers including self.
             // The votesRequired field therefore stores a calculated value
             // of the number of votes required for this candidate to win an
@@ -73,7 +67,7 @@ public class Candidate extends AbstractRaftActorBehavior {
             // 0 peers = 1 votesRequired (0 + 1) / 2 + 1 = 1
             // 2 peers = 2 votesRequired (2 + 1) / 2 + 1 = 2
             // 4 peers = 3 votesRequired (4 + 1) / 2 + 1 = 3
-            int noOfPeers = peerPaths.size();
+            int noOfPeers = peers.size();
             int self = 1;
             votesRequired = (noOfPeers + self) / 2 + 1;
         } else {
@@ -87,6 +81,8 @@ public class Candidate extends AbstractRaftActorBehavior {
     @Override protected RaftState handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries) {
 
+        context.getLogger().info("Candidate: Received {}", appendEntries.toString());
+
         return state();
     }
 
@@ -115,10 +111,16 @@ public class Candidate extends AbstractRaftActorBehavior {
     }
 
     @Override
-    public RaftState handleMessage(ActorRef sender, Object message) {
+    public RaftState handleMessage(ActorRef sender, Object originalMessage) {
+
+        Object message = fromSerializableMessage(originalMessage);
 
         if (message instanceof RaftRPC) {
+
             RaftRPC rpc = (RaftRPC) message;
+
+            context.getLogger().debug("RaftRPC message received {} my term is {}", rpc.toString(), context.getTermInformation().getCurrentTerm());
+
             // If RPC request or response contains term T > currentTerm:
             // set currentTerm = T, convert to follower (§5.1)
             // This applies to all RPC messages and responses
@@ -141,6 +143,7 @@ public class Candidate extends AbstractRaftActorBehavior {
             scheduleElection(electionDuration());
             return state();
         }
+
         return super.handleMessage(sender, message);
     }
 
@@ -153,21 +156,25 @@ public class Candidate extends AbstractRaftActorBehavior {
 
         // Increment the election term and vote for self
         long currentTerm = context.getTermInformation().getCurrentTerm();
-        context.getTermInformation().updateAndPersist(currentTerm + 1, context.getId());
+        context.getTermInformation().updateAndPersist(currentTerm + 1,
+            context.getId());
 
-        context.getLogger().debug("Starting new term " + (currentTerm+1));
+        context.getLogger().debug("Starting new term " + (currentTerm + 1));
 
         // Request for a vote
         // TODO: Retry request for vote if replies do not arrive in a reasonable
         // amount of time TBD
-        for (ActorSelection peerActor : peerToActor.values()) {
-            peerActor.tell(new RequestVote(
-                    context.getTermInformation().getCurrentTerm(),
-                    context.getId(),
-                    context.getReplicatedLog().lastIndex(),
-                    context.getReplicatedLog().lastTerm()),
-                context.getActor()
-            );
+        for (String peerId : peers) {
+            ActorSelection peerActor = context.getPeerActorSelection(peerId);
+            if(peerActor != null) {
+                peerActor.tell(new RequestVote(
+                        context.getTermInformation().getCurrentTerm(),
+                        context.getId(),
+                        context.getReplicatedLog().lastIndex(),
+                        context.getReplicatedLog().lastTerm()),
+                    context.getActor()
+                );
+            }
         }
 
 
index 68b444b2bac7d20ae0ab7a307ede546191392938..db62dfc2ac1cd4f10c59849c0666b9291ca946a3 100644 (file)
@@ -12,8 +12,8 @@ import akka.actor.ActorRef;
 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.internal.messages.ApplySnapshot;
-import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
@@ -40,6 +40,11 @@ public class Follower extends AbstractRaftActorBehavior {
     @Override protected RaftState handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries) {
 
+        if(appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
+            context.getLogger()
+                .info("Follower: Received {}", appendEntries.toString());
+        }
+
         // TODO : Refactor this method into a bunch of smaller methods
         // to make it easier to read. Before refactoring ensure tests
         // cover the code properly
@@ -162,8 +167,9 @@ public class Follower extends AbstractRaftActorBehavior {
             for (int i = addEntriesFrom;
                  i < appendEntries.getEntries().size(); i++) {
 
-                context.getLogger().debug(
-                    "Append entry to log " + appendEntries.getEntries().get(i).getData()
+                context.getLogger().info(
+                    "Append entry to log " + appendEntries.getEntries().get(
+                        i).getData()
                         .toString()
                 );
                 context.getReplicatedLog()
index 26beed2f7a69b5191407f230c839edd1e4178ade..53e47c2f842f17ac0b2766811745a7f1ec6eabde 100644 (file)
@@ -19,10 +19,10 @@ import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
 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.internal.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
-import org.opendaylight.controller.cluster.raft.internal.messages.SendHeartBeat;
-import org.opendaylight.controller.cluster.raft.internal.messages.SendInstallSnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
+import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
+import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
@@ -36,6 +36,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -67,7 +68,7 @@ public class Leader extends AbstractRaftActorBehavior {
     private final Map<String, FollowerLogInformation> followerToLog =
         new HashMap();
 
-    private final Map<String, ActorSelection> followerToActor = new HashMap<>();
+    private final Set<String> followers;
 
     private Cancellable heartbeatSchedule = null;
     private Cancellable appendEntriesSchedule = null;
@@ -84,22 +85,21 @@ public class Leader extends AbstractRaftActorBehavior {
             context.setCommitIndex(lastIndex());
         }
 
-        for (String followerId : context.getPeerAddresses().keySet()) {
+        followers = context.getPeerAddresses().keySet();
+
+        for (String followerId : followers) {
             FollowerLogInformation followerLogInformation =
                 new FollowerLogInformationImpl(followerId,
                     new AtomicLong(lastIndex()),
                     new AtomicLong(-1));
 
-            followerToActor.put(followerId,
-                context.actorSelection(context.getPeerAddress(followerId)));
-
             followerToLog.put(followerId, followerLogInformation);
         }
 
-        context.getLogger().debug("Election:Leader has following peers:"+followerToActor.keySet());
+        context.getLogger().debug("Election:Leader has following peers:"+ followers);
 
-        if (followerToActor.size() > 0) {
-            minReplicationCount = (followerToActor.size() + 1) / 2 + 1;
+        if (followers.size() > 0) {
+            minReplicationCount = (followers.size() + 1) / 2 + 1;
         } else {
             minReplicationCount = 0;
         }
@@ -121,16 +121,29 @@ public class Leader extends AbstractRaftActorBehavior {
     @Override protected RaftState handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries) {
 
+        context.getLogger().info("Leader: Received {}", appendEntries.toString());
+
         return state();
     }
 
     @Override protected RaftState handleAppendEntriesReply(ActorRef sender,
         AppendEntriesReply appendEntriesReply) {
 
+        if(! appendEntriesReply.isSuccess()) {
+            context.getLogger()
+                .info("Leader: Received {}", appendEntriesReply.toString());
+        }
+
         // Update the FollowerLogInformation
         String followerId = appendEntriesReply.getFollowerId();
         FollowerLogInformation followerLogInformation =
             followerToLog.get(followerId);
+
+        if(followerLogInformation == null){
+            context.getLogger().error("Unknown follower {}", followerId);
+            return state();
+        }
+
         if (appendEntriesReply.isSuccess()) {
             followerLogInformation
                 .setMatchIndex(appendEntriesReply.getLogLastIndex());
@@ -251,7 +264,7 @@ public class Leader extends AbstractRaftActorBehavior {
 
         context.getLogger().debug("Replicate message " + logIndex);
 
-        if (followerToActor.size() == 0) {
+        if (followers.size() == 0) {
             context.setCommitIndex(
                 replicate.getReplicatedLogEntry().getIndex());
 
@@ -277,32 +290,37 @@ public class Leader extends AbstractRaftActorBehavior {
 
     private void sendAppendEntries() {
         // Send an AppendEntries to all followers
-        for (String followerId : followerToActor.keySet()) {
+        for (String followerId : followers) {
             ActorSelection followerActor =
-                followerToActor.get(followerId);
+                context.getPeerActorSelection(followerId);
 
-            FollowerLogInformation followerLogInformation =
-                followerToLog.get(followerId);
+            if (followerActor != null) {
+                FollowerLogInformation followerLogInformation =
+                    followerToLog.get(followerId);
 
-            long nextIndex = followerLogInformation.getNextIndex().get();
+                long nextIndex = followerLogInformation.getNextIndex().get();
 
-            List<ReplicatedLogEntry> entries = Collections.emptyList();
+                List<ReplicatedLogEntry> entries = Collections.emptyList();
 
-            if(context.getReplicatedLog().isPresent(nextIndex)){
-                // TODO: Instead of sending all entries from nextIndex
-                // only send a fixed number of entries to each follower
-                // This is to avoid the situation where there are a lot of
-                // entries to install for a fresh follower or to a follower
-                // that has fallen too far behind with the log but yet is not
-                // eligible to receive a snapshot
-                entries =
-                    context.getReplicatedLog().getFrom(nextIndex);
-            }
+                if (context.getReplicatedLog().isPresent(nextIndex)) {
+                    // TODO: Instead of sending all entries from nextIndex
+                    // only send a fixed number of entries to each follower
+                    // This is to avoid the situation where there are a lot of
+                    // entries to install for a fresh follower or to a follower
+                    // that has fallen too far behind with the log but yet is not
+                    // eligible to receive a snapshot
+                    entries =
+                        context.getReplicatedLog().getFrom(nextIndex);
+                }
 
-            followerActor.tell(
-                new AppendEntries(currentTerm(), context.getId(), prevLogIndex(nextIndex),
-                    prevLogTerm(nextIndex), entries, context.getCommitIndex()).toSerializable(),
-                actor());
+                followerActor.tell(
+                    new AppendEntries(currentTerm(), context.getId(),
+                        prevLogIndex(nextIndex),
+                        prevLogTerm(nextIndex), entries,
+                        context.getCommitIndex()).toSerializable(),
+                    actor()
+                );
+            }
         }
     }
 
@@ -312,30 +330,33 @@ public class Leader extends AbstractRaftActorBehavior {
      * snapshots at every heartbeat.
      */
     private void installSnapshotIfNeeded(){
-        for (String followerId : followerToActor.keySet()) {
+        for (String followerId : followers) {
             ActorSelection followerActor =
-                followerToActor.get(followerId);
-
-            FollowerLogInformation followerLogInformation =
-                followerToLog.get(followerId);
-
-            long nextIndex = followerLogInformation.getNextIndex().get();
-
-            if(!context.getReplicatedLog().isPresent(nextIndex) && context.getReplicatedLog().isInSnapshot(nextIndex)){
-                followerActor.tell(
-                    new InstallSnapshot(currentTerm(), context.getId(),
-                        context.getReplicatedLog().getSnapshotIndex(),
-                        context.getReplicatedLog().getSnapshotTerm(),
-                        context.getReplicatedLog().getSnapshot()
-                    ),
-                    actor()
-                );
+                context.getPeerActorSelection(followerId);
+
+            if(followerActor != null) {
+                FollowerLogInformation followerLogInformation =
+                    followerToLog.get(followerId);
+
+                long nextIndex = followerLogInformation.getNextIndex().get();
+
+                if (!context.getReplicatedLog().isPresent(nextIndex) && context
+                    .getReplicatedLog().isInSnapshot(nextIndex)) {
+                    followerActor.tell(
+                        new InstallSnapshot(currentTerm(), context.getId(),
+                            context.getReplicatedLog().getSnapshotIndex(),
+                            context.getReplicatedLog().getSnapshotTerm(),
+                            context.getReplicatedLog().getSnapshot()
+                        ),
+                        actor()
+                    );
+                }
             }
         }
     }
 
     private RaftState sendHeartBeat() {
-        if (followerToActor.size() > 0) {
+        if (followers.size() > 0) {
             sendAppendEntries();
         }
         return state();
@@ -354,7 +375,7 @@ public class Leader extends AbstractRaftActorBehavior {
     }
 
     private void scheduleHeartBeat(FiniteDuration interval) {
-        if(followerToActor.keySet().size() == 0){
+        if(followers.size() == 0){
             // Optimization - do not bother scheduling a heartbeat as there are
             // no followers
             return;
@@ -376,7 +397,7 @@ public class Leader extends AbstractRaftActorBehavior {
 
 
     private void scheduleInstallSnapshotCheck(FiniteDuration interval) {
-        if(followerToActor.keySet().size() == 0){
+        if(followers.size() == 0){
             // Optimization - do not bother scheduling a heartbeat as there are
             // no followers
             return;
index 45ac9a9ebfbecbb2fb13881dd3f365f8ac9f7e8e..94366efd5e897657744df37b0a475a847273e1b4 100644 (file)
@@ -14,7 +14,6 @@ import org.opendaylight.controller.cluster.raft.ReplicatedLogImplEntry;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -24,7 +23,7 @@ import java.util.Map;
  * Invoked by leader to replicate log entries (§5.3); also used as
  * heartbeat (§5.2).
  */
-public class AppendEntries extends AbstractRaftRPC implements Serializable {
+public class AppendEntries extends AbstractRaftRPC {
 
     public static final Class SERIALIZABLE_CLASS = AppendEntriesMessages.AppendEntries.class;
 
@@ -77,13 +76,16 @@ public class AppendEntries extends AbstractRaftRPC implements Serializable {
     }
 
     @Override public String toString() {
-        return "AppendEntries{" +
-            "leaderId='" + leaderId + '\'' +
-            ", prevLogIndex=" + prevLogIndex +
-            ", prevLogTerm=" + prevLogTerm +
-            ", entries=" + entries +
-            ", leaderCommit=" + leaderCommit +
-            '}';
+        final StringBuilder sb =
+            new StringBuilder("AppendEntries{");
+        sb.append("term=").append(getTerm());
+        sb.append("leaderId='").append(leaderId).append('\'');
+        sb.append(", prevLogIndex=").append(prevLogIndex);
+        sb.append(", prevLogTerm=").append(prevLogTerm);
+        sb.append(", entries=").append(entries);
+        sb.append(", leaderCommit=").append(leaderCommit);
+        sb.append('}');
+        return sb.toString();
     }
 
     public <T extends Object> Object toSerializable(){
index 7e6628abe1464800ef2449501a165a89c19cbcb3..b923baa7168df3d34ae7064c722222b66bbf9c51 100644 (file)
@@ -8,12 +8,10 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
 /**
  * Reply for the AppendEntriesRpc message
  */
-public class AppendEntriesReply extends AbstractRaftRPC implements Serializable {
+public class AppendEntriesReply extends AbstractRaftRPC {
 
     // true if follower contained entry matching
     // prevLogIndex and prevLogTerm
@@ -58,4 +56,16 @@ public class AppendEntriesReply extends AbstractRaftRPC implements Serializable
     public String getFollowerId() {
         return followerId;
     }
+
+    @Override public String toString() {
+        final StringBuilder sb =
+            new StringBuilder("AppendEntriesReply{");
+        sb.append("term=").append(term);
+        sb.append(", success=").append(success);
+        sb.append(", logLastIndex=").append(logLastIndex);
+        sb.append(", logLastTerm=").append(logLastTerm);
+        sb.append(", followerId='").append(followerId).append('\'');
+        sb.append('}');
+        return sb.toString();
+    }
 }
index 5053560b411c014e9dd756c48fd84e9a17ab4cab..888854fa71eaf4f745e11456c7b09349a7d8a443 100644 (file)
@@ -8,9 +8,7 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
-public class InstallSnapshot extends AbstractRaftRPC implements Serializable {
+public class InstallSnapshot extends AbstractRaftRPC {
 
     private final String leaderId;
     private final long lastIncludedIndex;
index 02a3252776167580b3bfa444ee765223ad6ded3b..85b89b70ae2b84c8ab26fcb7e16ba041fbd2040a 100644 (file)
@@ -8,9 +8,7 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
-public class InstallSnapshotReply extends AbstractRaftRPC implements Serializable {
+public class InstallSnapshotReply extends AbstractRaftRPC {
 
     // The followerId - this will be used to figure out which follower is
     // responding
index a770e54f580a971b679b923ef5d44e37d9fc98dd..10d99988d8bf6ac58a16a784a4fcc00f6c1d8ece 100644 (file)
@@ -8,6 +8,8 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-public interface RaftRPC {
+import java.io.Serializable;
+
+public interface RaftRPC extends Serializable {
     public long getTerm();
 }
index 310968de950bf2f5e4ff31bcbcc3d7718ad618a8..6ef2a06285fedb85997f79b3763d367d46ceb663 100644 (file)
@@ -8,12 +8,10 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
 /**
  * Invoked by candidates to gather votes (§5.2).
  */
-public class RequestVote extends AbstractRaftRPC implements Serializable{
+public class RequestVote extends AbstractRaftRPC {
 
     // candidate requesting vote
     private String candidateId;
@@ -63,4 +61,15 @@ public class RequestVote extends AbstractRaftRPC implements Serializable{
     public void setLastLogTerm(long lastLogTerm) {
         this.lastLogTerm = lastLogTerm;
     }
+
+    @Override public String toString() {
+        final StringBuilder sb =
+            new StringBuilder("RequestVote{");
+        sb.append("term='").append(getTerm()).append('\'');
+        sb.append("candidateId='").append(candidateId).append('\'');
+        sb.append(", lastLogIndex=").append(lastLogIndex);
+        sb.append(", lastLogTerm=").append(lastLogTerm);
+        sb.append('}');
+        return sb.toString();
+    }
 }
index a658ab7d810685c53b90ed6955ce5df5273b1b10..df80b4e5e24e34f0d9d4a3017d7dbfdea1b181ea 100644 (file)
@@ -8,9 +8,7 @@
 
 package org.opendaylight.controller.cluster.raft.messages;
 
-import java.io.Serializable;
-
-public class RequestVoteReply extends AbstractRaftRPC implements Serializable {
+public class RequestVoteReply extends AbstractRaftRPC {
 
     // true means candidate received vot
     private final boolean voteGranted;
index 406c164cdbd66acbc7ce452ad6ee4640ec4521b3..2e200cba275d9d4265f56da69ee36e61acd0db76 100644 (file)
@@ -18,6 +18,7 @@ import com.google.protobuf.GeneratedMessage;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages;
 import org.opendaylight.controller.cluster.raft.protobuff.messages.MockPayloadMessages;
+import com.google.common.base.Preconditions;
 
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -159,6 +160,19 @@ public class MockRaftActorContext implements RaftActorContext {
         peerAddresses.remove(name);
     }
 
+    @Override public ActorSelection getPeerActorSelection(String peerId) {
+        String peerAddress = getPeerAddress(peerId);
+        if(peerAddress != null){
+            return actorSelection(peerAddress);
+        }
+        return null;
+    }
+
+    @Override public void setPeerAddress(String peerId, String peerAddress) {
+        Preconditions.checkState(peerAddresses.containsKey(peerId));
+        peerAddresses.put(peerId, peerAddress);
+    }
+
     public void setPeerAddresses(Map<String, String> peerAddresses) {
         this.peerAddresses = peerAddresses;
     }
index 8bcee58afe11b40c03bf8fa36eeb0b338a1ae3ff..c76368370506af15ab4b7b567672514514e41690 100644 (file)
@@ -9,7 +9,7 @@ import org.junit.Test;
 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.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
index 126e7d10c25a1981ab3e4ba30021e68cf07df214..c015d950c42055799bdd5826f0cca1aaf398f357 100644 (file)
@@ -9,7 +9,7 @@ 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.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
index bc9d4a2767b130e1afde72c40641ae90af9fd31a..d33b33925b5e7aba358152ff6067e4842fec3b71 100644 (file)
@@ -8,9 +8,9 @@ import org.junit.Test;
 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.internal.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
-import org.opendaylight.controller.cluster.raft.internal.messages.SendHeartBeat;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
+import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.utils.DoNothingActor;
 
index 16b97b7855eb4755b63ebe6bac07784e460d20c5..94c895dab385a92e3f547099ac83010cb7e517ec 100644 (file)
         <version>1.1-SNAPSHOT</version>
     </dependency>
 
+    <dependency>
+        <groupId>org.opendaylight.controller</groupId>
+        <artifactId>sal-akka-raft</artifactId>
+        <version>1.1-SNAPSHOT</version>
+    </dependency>
+
     <!-- Test Dependencies -->
     <dependency>
       <groupId>junit</groupId>
             <Import-Package>!*snappy;!org.jboss.*;*</Import-Package>
             <Embed-Dependency>
                 sal-protocolbuffer-encoding;
+                sal-akka-raft;
                 !sal*;
                 !*config-api*;
                 !*testkit*;
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/CompositeModificationPayload.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/CompositeModificationPayload.java
new file mode 100644 (file)
index 0000000..955e4bb
--- /dev/null
@@ -0,0 +1,53 @@
+/*
+ * 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.datastore;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.GeneratedMessage;
+import org.opendaylight.controller.cluster.example.protobuff.messages.KeyValueMessages;
+import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
+import org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages;
+import org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+public class CompositeModificationPayload extends Payload implements
+    Serializable {
+
+    private final PersistentMessages.CompositeModification modification;
+
+    public CompositeModificationPayload(){
+        modification = null;
+    }
+    public CompositeModificationPayload(Object modification){
+        this.modification = (PersistentMessages.CompositeModification) modification;
+    }
+
+    @Override public Map<GeneratedMessage.GeneratedExtension, PersistentMessages.CompositeModification> encode() {
+        Preconditions.checkState(modification!=null);
+        Map<GeneratedMessage.GeneratedExtension, PersistentMessages.CompositeModification> map = new HashMap<>();
+        map.put(org.opendaylight.controller.mdsal.CompositeModificationPayload.modification, this.modification);
+        return map;
+    }
+
+    @Override public Payload decode(
+        AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload payload) {
+        PersistentMessages.CompositeModification modification = payload
+            .getExtension(
+                org.opendaylight.controller.mdsal.CompositeModificationPayload.modification);
+        payload.getExtension(KeyValueMessages.value);
+        return new CompositeModificationPayload(modification);
+    }
+
+    public Object getModification(){
+        return this.modification;
+    }
+}
index a67f58c760bbdaeda30f2f6b9c32512537ee52c0..34239070a3dfafe15c8ba7e71c630eaf56136e3b 100644 (file)
@@ -15,9 +15,41 @@ import java.util.List;
 import java.util.Map;
 
 public interface Configuration {
+
+    /**
+     * Given a memberName find all the shards that belong on that member and
+     * return the names of those shards
+     *
+     * @param memberName
+     * @return
+     */
     List<String> getMemberShardNames(String memberName);
+
+    /**
+     * Given a module namespace return the name of a module
+     * @param nameSpace
+     * @return
+     */
     Optional<String> getModuleNameFromNameSpace(String nameSpace);
+
+    /**
+     * Get a mapping of the module names to it's corresponding ShardStrategy
+     * @return
+     */
     Map<String, ShardStrategy> getModuleNameToShardStrategyMap();
+
+    /**
+     * Given a module name find all the shardNames corresponding to it
+     * @param moduleName
+     * @return
+     */
     List<String> getShardNamesFromModuleName(String moduleName);
+
+    /**
+     * Given a shardName find all the members on which it belongs
+     *
+     * @param shardName
+     * @return
+     */
     List<String> getMembersFromShardName(String shardName);
 }
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpCohort.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpCohort.java
new file mode 100644 (file)
index 0000000..eb28159
--- /dev/null
@@ -0,0 +1,40 @@
+/*
+ * 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.datastore;
+
+import akka.actor.UntypedActor;
+import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.PreCommitTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.PreCommitTransactionReply;
+
+public class NoOpCohort extends UntypedActor {
+
+    @Override public void onReceive(Object message) throws Exception {
+        if (message.getClass().equals(CanCommitTransaction.SERIALIZABLE_CLASS)) {
+            getSender().tell(new CanCommitTransactionReply(false).toSerializable(), getSelf());
+        } else if (message.getClass().equals(PreCommitTransaction.SERIALIZABLE_CLASS)) {
+            getSender().tell(
+                new PreCommitTransactionReply().toSerializable(),
+                getSelf());
+        } else if (message.getClass().equals(CommitTransaction.SERIALIZABLE_CLASS)) {
+            getSender().tell(new CommitTransactionReply().toSerializable(), getSelf());
+        } else if (message.getClass().equals(AbortTransaction.SERIALIZABLE_CLASS)) {
+            getSender().tell(new AbortTransactionReply().toSerializable(), getSelf());
+        } else {
+            throw new Exception ("Not recognized message received,message="+message);
+        }
+
+    }
+}
+
index d6ad553cf3a237002645811dad1b299c3f2fd309..999d0f8bafca9639baa70ea7c893656f87704ae7 100644 (file)
@@ -14,9 +14,7 @@ import akka.actor.Props;
 import akka.event.Logging;
 import akka.event.LoggingAdapter;
 import akka.japi.Creator;
-import akka.persistence.Persistent;
-import akka.persistence.RecoveryCompleted;
-import akka.persistence.UntypedProcessor;
+import akka.serialization.Serialization;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
@@ -28,12 +26,13 @@ import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionC
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChainReply;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.ForwardedCommitTransaction;
-import org.opendaylight.controller.cluster.datastore.messages.NonPersistent;
+import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
 import org.opendaylight.controller.cluster.datastore.modification.Modification;
 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
+import org.opendaylight.controller.cluster.raft.RaftActor;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
@@ -54,7 +53,7 @@ import java.util.concurrent.Executors;
  * Our Shard uses InMemoryDataStore as it's internal representation and delegates all requests it
  * </p>
  */
-public class Shard extends UntypedProcessor {
+public class Shard extends RaftActor {
 
     public static final String DEFAULT_NAME = "default";
 
@@ -73,11 +72,16 @@ public class Shard extends UntypedProcessor {
     // property persistent
     private final boolean persistent;
 
+    private final String name;
+
     private SchemaContext schemaContext;
 
     private final ShardStats shardMBean;
 
-    private Shard(String name) {
+    private Shard(String name, Map<String, String> peerAddresses) {
+        super(name, peerAddresses);
+
+        this.name = name;
 
         String setting = System.getProperty("shard.persistent");
 
@@ -91,78 +95,88 @@ public class Shard extends UntypedProcessor {
 
     }
 
-    public static Props props(final String name) {
+    public static Props props(final String name, final Map<String, String> peerAddresses) {
         return Props.create(new Creator<Shard>() {
 
             @Override
             public Shard create() throws Exception {
-                return new Shard(name);
+                return new Shard(name, peerAddresses);
             }
 
         });
     }
 
 
-    @Override
-    public void onReceive(Object message) throws Exception {
-        LOG.debug("Received message " + message.getClass().toString());
-
-        if(!recoveryFinished()){
-            // FIXME : Properly handle recovery
-            return;
-        }
+    @Override public void onReceiveCommand(Object message){
+        LOG.debug("Received message {} from {}", message.getClass().toString(), getSender());
 
         if (message.getClass().equals(CreateTransactionChain.SERIALIZABLE_CLASS)) {
-            createTransactionChain();
+            if(isLeader()) {
+                createTransactionChain();
+            } else if(getLeader() != null){
+                getLeader().forward(message, getContext());
+            }
         } else if (message.getClass().equals(RegisterChangeListener.SERIALIZABLE_CLASS)) {
             registerChangeListener(RegisterChangeListener.fromSerializable(getContext().system(), message));
         } else if (message instanceof UpdateSchemaContext) {
             updateSchemaContext((UpdateSchemaContext) message);
         } else if (message instanceof ForwardedCommitTransaction) {
             handleForwardedCommit((ForwardedCommitTransaction) message);
-        } else if (message instanceof Persistent) {
-            commit(((Persistent)message).payload());
         } else if (message.getClass().equals(CreateTransaction.SERIALIZABLE_CLASS)) {
-            createTransaction(CreateTransaction.fromSerializable(message));
-        } else if(message instanceof NonPersistent){
-            commit(((NonPersistent)message).payload());
-        }else if (message instanceof RecoveryCompleted) {
-            //FIXME: PROPERLY HANDLE RECOVERY COMPLETED
-
-        }else {
-          throw new Exception("Not recognized message found message=" + message);
+            if(isLeader()) {
+                createTransaction(CreateTransaction.fromSerializable(message));
+            } else if(getLeader() != null){
+                getLeader().forward(message, getContext());
+            }
+        } else if (message instanceof PeerAddressResolved){
+            PeerAddressResolved resolved = (PeerAddressResolved) message;
+            setPeerAddress(resolved.getPeerId(), resolved.getPeerAddress());
+        } else {
+          super.onReceiveCommand(message);
         }
     }
 
     private void createTransaction(CreateTransaction createTransaction) {
         DOMStoreReadWriteTransaction transaction =
             store.newReadWriteTransaction();
+        String transactionId = "shard-" + createTransaction.getTransactionId();
+        LOG.info("Creating transaction : {} " , transactionId);
         ActorRef transactionActor = getContext().actorOf(
-            ShardTransaction.props(transaction, getSelf(), schemaContext), "shard-" + createTransaction.getTransactionId());
+            ShardTransaction.props(transaction, getSelf(), schemaContext), transactionId);
+
         getSender()
-            .tell(new CreateTransactionReply(transactionActor.path().toString(), createTransaction.getTransactionId()).toSerializable(),
+            .tell(new CreateTransactionReply(Serialization.serializedActorPath(transactionActor), createTransaction.getTransactionId()).toSerializable(),
                 getSelf());
     }
 
-    private void commit(Object serialized) {
+    private void commit(final ActorRef sender, Object serialized) {
         Modification modification = MutableCompositeModification.fromSerializable(serialized, schemaContext);
         DOMStoreThreePhaseCommitCohort cohort =
             modificationToCohort.remove(serialized);
         if (cohort == null) {
             LOG.error(
                 "Could not find cohort for modification : " + modification);
+            LOG.info("Writing modification using a new transaction");
+            modification.apply(store.newReadWriteTransaction());
             return;
         }
+
         final ListenableFuture<Void> future = cohort.commit();
         shardMBean.incrementCommittedTransactionCount();
-        final ActorRef sender = getSender();
         final ActorRef self = getSelf();
         future.addListener(new Runnable() {
             @Override
             public void run() {
                 try {
                     future.get();
-                    sender.tell(new CommitTransactionReply().toSerializable(), self);
+
+                    if(sender != null) {
+                        sender
+                            .tell(new CommitTransactionReply().toSerializable(),
+                                self);
+                    } else {
+                        LOG.error("sender is null ???");
+                    }
                 } catch (InterruptedException | ExecutionException e) {
                     // FIXME : Handle this properly
                     LOG.error(e, "An exception happened when committing");
@@ -176,12 +190,11 @@ public class Shard extends UntypedProcessor {
 
         modificationToCohort
             .put(serializedModification , message.getCohort());
+
         if(persistent) {
-            getSelf().forward(Persistent.create(serializedModification),
-                getContext());
+            this.persistData(getSender(), "identifier", new CompositeModificationPayload(serializedModification));
         } else {
-            getSelf().forward(NonPersistent.create(serializedModification),
-                getContext());
+            this.commit(getSender(), serializedModification);
         }
     }
 
@@ -197,7 +210,8 @@ public class Shard extends UntypedProcessor {
 
 
         ActorSelection dataChangeListenerPath = getContext()
-            .system().actorSelection(registerChangeListener.getDataChangeListenerPath());
+            .system().actorSelection(
+                registerChangeListener.getDataChangeListenerPath());
 
         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>
             listener = new DataChangeListenerProxy(schemaContext,dataChangeListenerPath);
@@ -220,9 +234,36 @@ public class Shard extends UntypedProcessor {
     private void createTransactionChain() {
         DOMStoreTransactionChain chain = store.createTransactionChain();
         ActorRef transactionChain =
-            getContext().actorOf(ShardTransactionChain.props(chain, schemaContext));
+            getContext().actorOf(
+                ShardTransactionChain.props(chain, schemaContext));
         getSender()
-            .tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(),
+            .tell(new CreateTransactionChainReply(transactionChain.path())
+                .toSerializable(),
                 getSelf());
     }
+
+    @Override protected void applyState(ActorRef clientActor, String identifier,
+        Object data) {
+
+        if(data instanceof CompositeModificationPayload){
+            Object modification =
+                ((CompositeModificationPayload) data).getModification();
+            commit(clientActor, modification);
+        } else {
+            LOG.error("Unknown state received {}", data);
+        }
+
+    }
+
+    @Override protected Object createSnapshot() {
+        throw new UnsupportedOperationException("createSnapshot");
+    }
+
+    @Override protected void applySnapshot(Object snapshot) {
+        throw new UnsupportedOperationException("applySnapshot");
+    }
+
+    @Override public String persistenceId() {
+        return this.name;
+    }
 }
index 0363b3ceb370772ca617bb50abcddef95ed7da5e..5fbce4cd98900be65053939ca7a51d4f7a929a98 100644 (file)
@@ -19,6 +19,7 @@ import akka.japi.Creator;
 import akka.japi.Function;
 import com.google.common.base.Preconditions;
 import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
+import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryFound;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryNotFound;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
@@ -56,7 +57,7 @@ public class ShardManager extends AbstractUntypedActor {
     // Stores a mapping between a member name and the address of the member
     private final Map<String, Address> memberNameToAddress = new HashMap<>();
 
-    private final Map<String, ActorPath> localShards = new HashMap<>();
+    private final Map<String, ShardInformation> localShards = new HashMap<>();
 
 
     private final String type;
@@ -125,14 +126,20 @@ public class ShardManager extends AbstractUntypedActor {
     }
 
     private void memberUp(ClusterEvent.MemberUp message) {
-        memberNameToAddress.put(message.member().roles().head(), message.member().address());
+        String memberName = message.member().roles().head();
+
+        memberNameToAddress.put(memberName , message.member().address());
+
+        for(ShardInformation info : localShards.values()){
+            String shardName = info.getShardName();
+            info.updatePeerAddress(getShardActorName(memberName, shardName),
+                getShardActorPath(shardName, memberName));
+        }
     }
 
     private void updateSchemaContext(Object message) {
-        for(ActorPath path : localShards.values()){
-            getContext().system().actorSelection(path)
-                .forward(message,
-                    getContext());
+        for(ShardInformation info : localShards.values()){
+            info.getActor().tell(message,getSelf());
         }
     }
 
@@ -142,35 +149,50 @@ public class ShardManager extends AbstractUntypedActor {
         List<String> members =
             configuration.getMembersFromShardName(shardName);
 
-        for(String memberName : members) {
-            if (memberName.equals(cluster.getCurrentMemberName())) {
-                // This is a local shard
-                ActorPath shardPath = localShards.get(shardName);
-                if (shardPath == null) {
-                    getSender()
-                        .tell(new PrimaryNotFound(shardName).toSerializable(), getSelf());
-                    return;
-                }
-                getSender().tell(new PrimaryFound(shardPath.toString()).toSerializable(),
-                    getSelf());
+        // First see if the there is a local replica for the shard
+        ShardInformation info = localShards.get(shardName);
+        if(info != null) {
+            ActorPath shardPath = info.getActorPath();
+            if (shardPath != null) {
+                getSender()
+                    .tell(
+                        new PrimaryFound(shardPath.toString()).toSerializable(),
+                        getSelf());
                 return;
-            } else {
-                Address address = memberNameToAddress.get(memberName);
-                if(address != null){
-                    String path =
-                        address.toString() + "/user/shardmanager-" + this.type + "/" + getShardActorName(
-                            memberName, shardName);
-                    getSender().tell(new PrimaryFound(path).toSerializable(), getSelf());
-                    return;
-                }
+            }
+        }
 
+        if(cluster.getCurrentMemberName() != null) {
+            members.remove(cluster.getCurrentMemberName());
+        }
 
+        // There is no way for us to figure out the primary (for now) so assume
+        // that one of the remote nodes is a primary
+        for(String memberName : members) {
+            Address address = memberNameToAddress.get(memberName);
+            if(address != null){
+                String path =
+                    getShardActorPath(shardName, memberName);
+                getSender().tell(new PrimaryFound(path).toSerializable(), getSelf());
+                return;
             }
         }
-
         getSender().tell(new PrimaryNotFound(shardName).toSerializable(), getSelf());
     }
 
+    private String
+
+
+    getShardActorPath(String shardName, String memberName) {
+        Address address = memberNameToAddress.get(memberName);
+        if(address != null) {
+            return address.toString() + "/user/shardmanager-" + this.type + "/"
+                + getShardActorName(
+                memberName, shardName);
+        }
+        return null;
+    }
+
     private String getShardActorName(String memberName, String shardName){
         return memberName + "-shard-" + shardName + "-" + this.type;
     }
@@ -183,14 +205,35 @@ public class ShardManager extends AbstractUntypedActor {
 
         for(String shardName : memberShardNames){
             String shardActorName = getShardActorName(memberName, shardName);
+            Map<String, String> peerAddresses = getPeerAddresses(shardName);
             ActorRef actor = getContext()
-                .actorOf(Shard.props(shardActorName), shardActorName);
-            ActorPath path = actor.path();
-            localShards.put(shardName, path);
+                .actorOf(Shard.props(shardActorName, peerAddresses),
+                    shardActorName);
+            localShards.put(shardName, new ShardInformation(shardName, actor, peerAddresses));
         }
 
     }
 
+    private Map<String, String> getPeerAddresses(String shardName){
+
+        Map<String, String> peerAddresses = new HashMap<>();
+
+        List<String> members =
+            this.configuration.getMembersFromShardName(shardName);
+
+        String currentMemberName = this.cluster.getCurrentMemberName();
+
+        for(String memberName : members){
+            if(!currentMemberName.equals(memberName)){
+                String shardActorName = getShardActorName(memberName, shardName);
+                String path =
+                    getShardActorPath(shardName, currentMemberName);
+                peerAddresses.put(shardActorName, path);
+            }
+        }
+        return peerAddresses;
+    }
+
 
     @Override
     public SupervisorStrategy supervisorStrategy() {
@@ -204,4 +247,49 @@ public class ShardManager extends AbstractUntypedActor {
         );
 
     }
+
+    private class ShardInformation {
+        private final String shardName;
+        private final ActorRef actor;
+        private final ActorPath actorPath;
+        private final Map<String, String> peerAddresses;
+
+        private ShardInformation(String shardName, ActorRef actor,
+            Map<String, String> peerAddresses) {
+            this.shardName = shardName;
+            this.actor = actor;
+            this.actorPath = actor.path();
+            this.peerAddresses = peerAddresses;
+        }
+
+        public String getShardName() {
+            return shardName;
+        }
+
+        public ActorRef getActor(){
+            return actor;
+        }
+
+        public ActorPath getActorPath() {
+            return actorPath;
+        }
+
+        public Map<String, String> getPeerAddresses() {
+            return peerAddresses;
+        }
+
+        public void updatePeerAddress(String peerId, String peerAddress){
+            LOG.info("updatePeerAddress for peer {} with address {}", peerId, peerAddress);
+            if(peerAddresses.containsKey(peerId)){
+                peerAddresses.put(peerId, peerAddress);
+
+                LOG.info("Sending PeerAddressResolved for peer {} with address {} to {}", peerId, peerAddress, actor.path());
+
+                actor
+                    .tell(new PeerAddressResolved(peerId, peerAddress),
+                        getSelf());
+
+            }
+        }
+    }
 }
index 5a0049aa6d05acd5f0e9114ff93e5dcc4ca53dad..c85d32012fed9ee036a96b2e3663c061eabd0385 100644 (file)
@@ -9,10 +9,14 @@
 package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorPath;
+import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
+import akka.actor.Props;
 import com.google.common.base.Optional;
+import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListenableFutureTask;
+import org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
@@ -30,6 +34,8 @@ import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCoh
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -60,6 +66,10 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     private static final AtomicLong counter = new AtomicLong();
 
+    private static final Logger
+        LOG = LoggerFactory.getLogger(TransactionProxy.class);
+
+
     private final TransactionType transactionType;
     private final ActorContext actorContext;
     private final Map<String, TransactionContext> remoteTransactionPaths = new HashMap<>();
@@ -72,7 +82,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         TransactionType transactionType,
         ExecutorService executor,
         SchemaContext schemaContext
-        ) {
+    ) {
 
         this.identifier = actorContext.getCurrentMemberName() + "-txn-" + counter.getAndIncrement();
         this.transactionType = transactionType;
@@ -88,33 +98,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         createTransactionIfMissing(actorContext, path);
 
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-
-        Callable<Optional<NormalizedNode<?,?>>> call = new Callable() {
-
-            @Override public Optional<NormalizedNode<?,?>> call() throws Exception {
-                Object response = actorContext
-                    .executeRemoteOperation(remoteTransaction, new ReadData(path).toSerializable(),
-                        ActorContext.ASK_DURATION);
-                if(response.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)){
-                    ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext,path, response);
-                    if(reply.getNormalizedNode() == null){
-                        return Optional.absent();
-                    }
-                    //FIXME : A cast should not be required here ???
-                    return (Optional<NormalizedNode<?, ?>>) Optional.of(reply.getNormalizedNode());
-                }
-
-                return Optional.absent();
-            }
-        };
-
-        ListenableFutureTask<Optional<NormalizedNode<?, ?>>>
-            future = ListenableFutureTask.create(call);
-
-        executor.submit(future);
-
-        return future;
+        return transactionContext(path).readData(path);
     }
 
     @Override
@@ -122,8 +106,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         createTransactionIfMissing(actorContext, path);
 
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-        remoteTransaction.tell(new WriteData(path, data, schemaContext).toSerializable(), null);
+        transactionContext(path).writeData(path, data);
     }
 
     @Override
@@ -131,8 +114,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         createTransactionIfMissing(actorContext, path);
 
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-        remoteTransaction.tell(new MergeData(path, data, schemaContext).toSerializable(), null);
+        transactionContext(path).mergeData(path, data);
     }
 
     @Override
@@ -140,8 +122,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         createTransactionIfMissing(actorContext, path);
 
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-        remoteTransaction.tell(new DeleteData(path).toSerializable(), null);
+        transactionContext(path).deleteData(path);
     }
 
     @Override
@@ -149,10 +130,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         List<ActorPath> cohortPaths = new ArrayList<>();
 
         for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            Object result = actorContext.executeRemoteOperation(transactionContext.getActor(),
-                new ReadyTransaction().toSerializable(),
-                ActorContext.ASK_DURATION
-            );
+            Object result = transactionContext.readyTransaction();
 
             if(result.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)){
                 ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(actorContext.getActorSystem(),result);
@@ -173,14 +151,13 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     @Override
     public void close() {
         for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            transactionContext.getActor().tell(
-                new CloseTransaction().toSerializable(), null);
+            transactionContext.closeTransaction();
         }
     }
 
-    private ActorSelection remoteTransactionFromIdentifier(YangInstanceIdentifier path){
+    private TransactionContext transactionContext(YangInstanceIdentifier path){
         String shardName = shardNameFromIdentifier(path);
-        return remoteTransactionPaths.get(shardName).getActor();
+        return remoteTransactionPaths.get(shardName);
     }
 
     private String shardNameFromIdentifier(YangInstanceIdentifier path){
@@ -198,48 +175,186 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             return;
         }
 
-        Object response = actorContext.executeShardOperation(shardName, new CreateTransaction(identifier).toSerializable(), ActorContext.ASK_DURATION);
-        if(response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)){
-            CreateTransactionReply reply = CreateTransactionReply.fromSerializable(response);
-            String transactionPath = actorContext.getRemoteActorPath(shardName, reply.getTransactionPath());
+        try {
+            Object response = actorContext.executeShardOperation(shardName,
+                new CreateTransaction(identifier).toSerializable(),
+                ActorContext.ASK_DURATION);
+            if (response.getClass()
+                .equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
+                CreateTransactionReply reply =
+                    CreateTransactionReply.fromSerializable(response);
+
+                String transactionPath = reply.getTransactionPath();
 
-            ActorSelection transactionActor = actorContext.actorSelection(transactionPath);
-            transactionContext = new TransactionContext(shardName, transactionPath, transactionActor);
+                LOG.info("Received transaction path = {}"  , transactionPath );
 
-            remoteTransactionPaths.put(shardName, transactionContext);
+                ActorSelection transactionActor =
+                    actorContext.actorSelection(transactionPath);
+                transactionContext =
+                    new TransactionContextImpl(shardName, transactionPath,
+                        transactionActor);
+
+                remoteTransactionPaths.put(shardName, transactionContext);
+            }
+        } catch(TimeoutException e){
+            remoteTransactionPaths.put(shardName, new NoOpTransactionContext(shardName));
         }
     }
 
+    private interface TransactionContext {
+        String getShardName();
+
+        String getResolvedCohortPath(String cohortPath);
+
+        public void closeTransaction();
+
+        public Object readyTransaction();
 
-    private class TransactionContext {
+        void deleteData(YangInstanceIdentifier path);
+
+        void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+
+        ListenableFuture<Optional<NormalizedNode<?, ?>>> readData(final YangInstanceIdentifier path);
+
+        void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+    }
+
+
+    private class TransactionContextImpl implements TransactionContext{
         private final String shardName;
         private final String actorPath;
         private final ActorSelection  actor;
 
 
-        private TransactionContext(String shardName, String actorPath,
+        private TransactionContextImpl(String shardName, String actorPath,
             ActorSelection actor) {
             this.shardName = shardName;
             this.actorPath = actorPath;
             this.actor = actor;
         }
 
-
-        public String getShardName() {
+        @Override public String getShardName() {
             return shardName;
         }
 
-        public String getActorPath() {
-            return actorPath;
-        }
-
-        public ActorSelection getActor() {
+        private ActorSelection getActor() {
             return actor;
         }
 
-        public String getResolvedCohortPath(String cohortPath){
+        @Override public String getResolvedCohortPath(String cohortPath){
             return actorContext.resolvePath(actorPath, cohortPath);
         }
+
+        @Override public void closeTransaction() {
+            getActor().tell(
+                new CloseTransaction().toSerializable(), null);
+        }
+
+        @Override public Object readyTransaction() {
+            return actorContext.executeRemoteOperation(getActor(),
+                new ReadyTransaction().toSerializable(),
+                ActorContext.ASK_DURATION
+            );
+
+        }
+
+        @Override public void deleteData(YangInstanceIdentifier path) {
+            getActor().tell(new DeleteData(path).toSerializable(), null);
+        }
+
+        @Override public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data){
+            getActor().tell(new MergeData(path, data, schemaContext).toSerializable(), null);
+        }
+
+        @Override public ListenableFuture<Optional<NormalizedNode<?, ?>>> readData(final YangInstanceIdentifier path) {
+
+            Callable<Optional<NormalizedNode<?,?>>> call = new Callable() {
+
+                @Override public Optional<NormalizedNode<?,?>> call() throws Exception {
+                    Object response = actorContext
+                        .executeRemoteOperation(getActor(), new ReadData(path).toSerializable(),
+                            ActorContext.ASK_DURATION);
+                    if(response.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)){
+                        ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext,path, response);
+                        if(reply.getNormalizedNode() == null){
+                            return Optional.absent();
+                        }
+                        //FIXME : A cast should not be required here ???
+                        return (Optional<NormalizedNode<?, ?>>) Optional.of(reply.getNormalizedNode());
+                    }
+
+                    return Optional.absent();
+                }
+            };
+
+            ListenableFutureTask<Optional<NormalizedNode<?, ?>>>
+                future = ListenableFutureTask.create(call);
+
+            executor.submit(future);
+
+            return future;
+        }
+
+        @Override public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+            getActor().tell(new WriteData(path, data, schemaContext).toSerializable(), null);
+        }
+
+    }
+
+    private class NoOpTransactionContext implements TransactionContext {
+
+        private final Logger
+            LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
+
+        private final String shardName;
+
+        private ActorRef cohort;
+
+        public NoOpTransactionContext(String shardName){
+            this.shardName = shardName;
+        }
+        @Override public String getShardName() {
+            return  shardName;
+
+        }
+
+        @Override public String getResolvedCohortPath(String cohortPath) {
+            return cohort.path().toString();
+        }
+
+        @Override public void closeTransaction() {
+            LOG.error("closeTransaction called");
+        }
+
+        @Override public Object readyTransaction() {
+            LOG.error("readyTransaction called");
+            cohort = actorContext.getActorSystem().actorOf(Props.create(NoOpCohort.class));
+            return new ReadyTransactionReply(cohort.path()).toSerializable();
+        }
+
+        @Override public void deleteData(YangInstanceIdentifier path) {
+            LOG.error("deleteData called path = {}", path);
+        }
+
+        @Override public void mergeData(YangInstanceIdentifier path,
+            NormalizedNode<?, ?> data) {
+            LOG.error("mergeData called path = {}", path);
+        }
+
+        @Override
+        public ListenableFuture<Optional<NormalizedNode<?, ?>>> readData(
+            YangInstanceIdentifier path) {
+            LOG.error("readData called path = {}", path);
+            return Futures.immediateFuture(
+                Optional.<NormalizedNode<?, ?>>absent());
+        }
+
+        @Override public void writeData(YangInstanceIdentifier path,
+            NormalizedNode<?, ?> data) {
+            LOG.error("writeData called path = {}", path);
+        }
     }
 
+
+
 }
index 4780aaccfb960b0254dde011a543666a373282dc..472cd387345a5ffd61747af83629dcb86a8100f3 100644 (file)
@@ -9,7 +9,7 @@
 package org.opendaylight.controller.cluster.datastore.exceptions;
 
 public class TimeoutException extends RuntimeException {
-    public TimeoutException(Exception e){
-        super(e);
+    public TimeoutException(String message, Exception e){
+        super(message, e);
     }
 }
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/messages/PeerAddressResolved.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/messages/PeerAddressResolved.java
new file mode 100644 (file)
index 0000000..8c2543e
--- /dev/null
@@ -0,0 +1,27 @@
+/*
+ * 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.datastore.messages;
+
+public class PeerAddressResolved {
+    private final String peerId;
+    private final String peerAddress;
+
+    public PeerAddressResolved(String peerId, String peerAddress) {
+        this.peerId = peerId;
+        this.peerAddress = peerAddress;
+    }
+
+    public String getPeerId() {
+        return peerId;
+    }
+
+    public String getPeerAddress() {
+        return peerAddress;
+    }
+}
index c7ee7d8c2cca9ced8f17cf9f4fbc2eecd5cd3f45..ac0893da5ad157c230fad2aab7401b27d0054751 100644 (file)
@@ -14,9 +14,6 @@ import akka.actor.ActorSelection;
 import akka.actor.ActorSystem;
 import akka.actor.PoisonPill;
 import akka.util.Timeout;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
 import org.opendaylight.controller.cluster.datastore.ClusterWrapper;
 import org.opendaylight.controller.cluster.datastore.Configuration;
 import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
@@ -105,7 +102,7 @@ public class ActorContext {
 
             return found.getPrimaryPath();
         }
-        throw new PrimaryNotFoundException();
+        throw new PrimaryNotFoundException("Could not find primary for shardName " + shardName);
     }
 
 
@@ -125,7 +122,7 @@ public class ActorContext {
         try {
             return Await.result(future, AWAIT_DURATION);
         } catch (Exception e) {
-            throw new TimeoutException(e);
+            throw new TimeoutException("Sending message " + message.getClass().toString() + " to actor " + actor.toString() + " failed" , e);
         }
     }
 
@@ -148,7 +145,7 @@ public class ActorContext {
         try {
             return Await.result(future, AWAIT_DURATION);
         } catch (Exception e) {
-            throw new TimeoutException(e);
+            throw new TimeoutException("Sending message " + message.getClass().toString() + " to actor " + actor.toString() + " failed" , e);
         }
     }
 
@@ -178,22 +175,15 @@ public class ActorContext {
         actorSystem.shutdown();
     }
 
-    public String getRemoteActorPath(final String shardName,
-        final String localPathOfRemoteActor) {
-        final String path = findPrimaryPath(shardName);
-
-        LoadingCache<String, String> graphs = CacheBuilder.newBuilder()
-            .expireAfterAccess(2, TimeUnit.SECONDS)
-            .build(
-                new CacheLoader<String, String>() {
-                    public String load(String key) {
-                        return resolvePath(path, localPathOfRemoteActor);
-                    }
-                }
-            );
-        return graphs.getUnchecked(localPathOfRemoteActor);
-    }
-
+    /**
+     * @deprecated Need to stop using this method. There are ways to send a
+     * remote ActorRef as a string which should be used instead of this hack
+     *
+     * @param primaryPath
+     * @param localPathOfRemoteActor
+     * @return
+     */
+    @Deprecated
     public String resolvePath(final String primaryPath,
         final String localPathOfRemoteActor) {
         StringBuilder builder = new StringBuilder();
index 214b3e9d3d989d73d0cd618db937a0fa6bcfdcb6..e23a76b0b223ee2b8ee0d5398da878cd520caf65 100644 (file)
@@ -10,22 +10,38 @@ package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorSystem;
 import akka.testkit.JavaTestKit;
+import org.apache.commons.io.FileUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
+import java.io.File;
+import java.io.IOException;
+
 public abstract class AbstractActorTest {
     private static ActorSystem system;
 
     @BeforeClass
-    public static void setUpClass() {
+    public static void setUpClass() throws IOException {
+        File journal = new File("journal");
+
+        if(journal.exists()) {
+            FileUtils.deleteDirectory(journal);
+        }
+
         System.setProperty("shard.persistent", "false");
         system = ActorSystem.create("test");
     }
 
     @AfterClass
-    public static void tearDownClass() {
+    public static void tearDownClass() throws IOException {
         JavaTestKit.shutdownActorSystem(system);
         system = null;
+
+        File journal = new File("journal");
+
+        if(journal.exists()) {
+            FileUtils.deleteDirectory(journal);
+        }
     }
 
     protected ActorSystem getSystem() {
index b62a4b36d5016a00b2b86ae54d1623a3ea76aba5..11ad559744a3923aebfbb07472771b98a4439ab0 100644 (file)
@@ -33,6 +33,8 @@ import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.util.Collections;
+
 public class BasicIntegrationTest extends AbstractActorTest {
 
     @Test
@@ -47,16 +49,25 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
 
         new JavaTestKit(getSystem()) {{
-            final Props props = Shard.props("config");
+            final Props props = Shard.props("config", Collections.EMPTY_MAP);
             final ActorRef shard = getSystem().actorOf(props);
 
             new Within(duration("5 seconds")) {
                 protected void run() {
 
+
                     shard.tell(
                         new UpdateSchemaContext(TestModel.createTestContext()),
                         getRef());
 
+
+                    // Wait for Shard to become a Leader
+                    try {
+                        Thread.sleep(1000);
+                    } catch (InterruptedException e) {
+                        e.printStackTrace();
+                    }
+                    // 1. Create a TransactionChain
                     shard.tell(new CreateTransactionChain().toSerializable(), getRef());
 
                     final ActorSelection transactionChain =
@@ -76,6 +87,9 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertNotNull(transactionChain);
 
+                    System.out.println("Successfully created transaction chain");
+
+                    // 2. Create a Transaction on the TransactionChain
                     transactionChain.tell(new CreateTransaction("txn-1").toSerializable(), getRef());
 
                     final ActorSelection transaction =
@@ -94,9 +108,9 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertNotNull(transaction);
 
-                    // Add a watch on the transaction actor so that we are notified when it dies
-                    final ActorRef transactionActorRef = watchActor(transaction);
+                    System.out.println("Successfully created transaction");
 
+                    // 3. Write some data
                     transaction.tell(new WriteData(TestModel.TEST_PATH,
                         ImmutableNodes.containerNode(TestModel.TEST_QNAME), TestModel.createTestContext()).toSerializable(),
                         getRef());
@@ -113,6 +127,10 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertTrue(writeDone);
 
+                    System.out.println("Successfully wrote data");
+
+                    // 4. Ready the transaction for commit
+
                     transaction.tell(new ReadyTransaction().toSerializable(), getRef());
 
                     final ActorSelection cohort =
@@ -132,8 +150,9 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertNotNull(cohort);
 
-                    // Add a watch on the transaction actor so that we are notified when it dies
-                    final ActorRef cohorActorRef = watchActor(cohort);
+                    System.out.println("Successfully readied the transaction");
+
+                    // 5. PreCommit the transaction
 
                     cohort.tell(new PreCommitTransaction().toSerializable(), getRef());
 
@@ -150,10 +169,14 @@ public class BasicIntegrationTest extends AbstractActorTest {
 
                     Assert.assertTrue(preCommitDone);
 
+                    System.out.println("Successfully pre-committed the transaction");
+
+                    // 6. Commit the transaction
                     cohort.tell(new CommitTransaction().toSerializable(), getRef());
 
                     // FIXME : Add assertions that the commit worked and that the cohort and transaction actors were terminated
 
+                    System.out.println("TODO : Check Successfully committed the transaction");
                 }
 
 
index 116e5e75b50d261e3d9673378ee643cfe2f2e31f..b5e3d24ef6d536d6afe2c9a6abc111bbb4b8688b 100644 (file)
@@ -9,7 +9,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
 import org.opendaylight.controller.cluster.datastore.utils.MockClusterWrapper;
-import org.opendaylight.controller.cluster.datastore.utils.MockConfiguration;
 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
 import org.opendaylight.controller.md.cluster.datastore.model.PeopleModel;
 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
@@ -46,12 +45,15 @@ public class DistributedDataStoreIntegrationTest{
 
     @Test
     public void integrationTest() throws Exception {
-        ShardStrategyFactory.setConfiguration(new MockConfiguration());
+        Configuration configuration = new ConfigurationImpl("module-shards.conf", "modules.conf");
+        ShardStrategyFactory.setConfiguration(configuration);
         DistributedDataStore distributedDataStore =
-            new DistributedDataStore(getSystem(), "config", new MockClusterWrapper(), new MockConfiguration());
+            new DistributedDataStore(getSystem(), "config", new MockClusterWrapper(), configuration);
 
         distributedDataStore.onGlobalContextUpdated(TestModel.createTestContext());
 
+        Thread.sleep(1000);
+
         DOMStoreReadWriteTransaction transaction =
             distributedDataStore.newReadWriteTransaction();
 
@@ -95,6 +97,8 @@ public class DistributedDataStoreIntegrationTest{
 
         distributedDataStore.onGlobalContextUpdated(SchemaContextHelper.full());
 
+        Thread.sleep(1000);
+
         DOMStoreReadWriteTransaction transaction =
             distributedDataStore.newReadWriteTransaction();
 
index f20cd8c3d7ef2a18bbc375ce3fa770975c26bc9c..7d57ea8284e90dc3f941b7b82385e5db28ed75f4 100644 (file)
@@ -7,6 +7,7 @@ import org.junit.Test;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChain;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChainReply;
+import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
@@ -19,6 +20,10 @@ import org.opendaylight.controller.protobuff.messages.transaction.ShardTransacti
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -26,10 +31,18 @@ public class ShardTest extends AbstractActorTest {
     @Test
     public void testOnReceiveCreateTransactionChain() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final Props props = Shard.props("config");
+            final Props props = Shard.props("config", Collections.EMPTY_MAP);
             final ActorRef subject =
                 getSystem().actorOf(props, "testCreateTransactionChain");
 
+
+            // Wait for Shard to become a Leader
+            try {
+                Thread.sleep(1000);
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+
             new Within(duration("1 seconds")) {
                 protected void run() {
 
@@ -64,7 +77,7 @@ public class ShardTest extends AbstractActorTest {
     @Test
     public void testOnReceiveRegisterListener() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final Props props = Shard.props("config");
+            final Props props = Shard.props("config", Collections.EMPTY_MAP);
             final ActorRef subject =
                 getSystem().actorOf(props, "testRegisterChangeListener");
 
@@ -107,10 +120,19 @@ public class ShardTest extends AbstractActorTest {
     @Test
     public void testCreateTransaction(){
         new JavaTestKit(getSystem()) {{
-            final Props props = Shard.props("config");
+            final Props props = Shard.props("config", Collections.EMPTY_MAP);
             final ActorRef subject =
                 getSystem().actorOf(props, "testCreateTransaction");
 
+
+            // Wait for Shard to become a Leader
+            try {
+                Thread.sleep(1000);
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+
+
             new Within(duration("1 seconds")) {
                 protected void run() {
 
@@ -135,9 +157,8 @@ public class ShardTest extends AbstractActorTest {
                         }
                     }.get(); // this extracts the received message
 
-                    assertEquals("Unexpected transaction path " + out,
-                        "akka://test/user/testCreateTransaction/shard-txn-1",
-                        out);
+                    assertTrue("Unexpected transaction path " + out,
+                        out.contains("akka://test/user/testCreateTransaction/shard-txn-1"));
                     expectNoMsg();
                 }
 
@@ -146,7 +167,29 @@ public class ShardTest extends AbstractActorTest {
         }};
     }
 
+    @Test
+    public void testPeerAddressResolved(){
+        new JavaTestKit(getSystem()) {{
+            Map<String, String> peerAddresses = new HashMap<>();
+            peerAddresses.put("member-2", null);
+            final Props props = Shard.props("config", peerAddresses);
+            final ActorRef subject =
+                getSystem().actorOf(props, "testPeerAddressResolved");
+
+            new Within(duration("1 seconds")) {
+                protected void run() {
 
+                    subject.tell(
+                        new PeerAddressResolved("member-2", "akka://foobar"),
+                        getRef());
+
+                    expectNoMsg();
+                }
+
+
+            };
+        }};
+    }
 
     private AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> noOpDataChangeListener() {
         return new AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>() {
index 4d7c61a197e08420e7cec90753aa1ef427f4ce4d..7884eeccdae7874f8d19c9090bf08d755cc991be 100644 (file)
@@ -30,6 +30,8 @@ import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 
+import java.util.Collections;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -49,7 +51,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveReadData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, testSchemaContext);
             final ActorRef subject = getSystem().actorOf(props, "testReadData");
@@ -89,7 +91,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveReadDataWhenDataNotFound() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, testSchemaContext);
             final ActorRef subject = getSystem().actorOf(props, "testReadDataWhenDataNotFound");
@@ -163,7 +165,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveWriteData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, TestModel.createTestContext());
             final ActorRef subject =
@@ -201,7 +203,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveMergeData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, testSchemaContext);
             final ActorRef subject =
@@ -240,7 +242,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveDeleteData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, TestModel.createTestContext());
             final ActorRef subject =
@@ -277,7 +279,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveReadyTransaction() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, TestModel.createTestContext());
             final ActorRef subject =
@@ -313,7 +315,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveCloseTransaction() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props("config"));
+            final ActorRef shard = getSystem().actorOf(Shard.props("config", Collections.EMPTY_MAP));
             final Props props =
                 ShardTransaction.props(store.newReadWriteTransaction(), shard, TestModel.createTestContext());
             final ActorRef subject =
index 9f35f2b9f24f4547c5989e7b7bd6c8458ff23d23..b3de998479cf37d84c211e033884230386096b72 100644 (file)
@@ -18,9 +18,7 @@ module-shards = [
             {
                 name="people-1"
                 replicas = [
-                    "member-1",
-                    "member-2",
-                    "member-3"
+                    "member-1"
                 ]
             }
         ]
@@ -31,9 +29,18 @@ module-shards = [
             {
                 name="cars-1"
                 replicas = [
-                    "member-4",
-                    "member-1",
-                    "member-5"
+                    "member-1"
+                ]
+            }
+        ]
+    },
+    {
+        name = "test"
+        shards = [
+            {
+                name="test-1"
+                replicas = [
+                    "member-1"
                 ]
             }
         ]
index 52f249a7ff57fa1a3ef3a0b8ca12faac09c06d6e..22854cb11a968bd66f1b969d95a5cc07da7941de 100644 (file)
@@ -8,6 +8,11 @@ modules = [
         name = "cars"
         namespace = "urn:opendaylight:params:xml:ns:yang:controller:md:sal:dom:store:test:cars"
         shard-strategy = "module"
+    },
+    {
+     name = "test"
+     namespace = "urn:opendaylight:params:xml:ns:yang:controller:md:sal:dom:store:test"
+     shard-strategy = "module"
     }
 
 ]
index e3fac63a8335b9e2fed6ef6844dc4ea3125a6cd9..28e629a92c92fa7b165a390e12e2fd89a15e2ba5 100644 (file)
           <groupId>org.opendaylight.yangtools</groupId>
           <artifactId>yang-binding</artifactId>
       </dependency>
+
+      <dependency>
+          <groupId>org.opendaylight.controller</groupId>
+          <artifactId>sal-akka-raft</artifactId>
+          <version>1.1-SNAPSHOT</version>
+      </dependency>
+
       <dependency>
           <groupId>com.google.guava</groupId>
           <artifactId>guava</artifactId>
diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/mdsal/CompositeModificationPayload.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/mdsal/CompositeModificationPayload.java
new file mode 100644 (file)
index 0000000..87b246b
--- /dev/null
@@ -0,0 +1,60 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: CompositeModificationPayload.proto
+
+package org.opendaylight.controller.mdsal;
+
+public final class CompositeModificationPayload {
+  private CompositeModificationPayload() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+    registry.add(org.opendaylight.controller.mdsal.CompositeModificationPayload.modification);
+  }
+  public static final int MODIFICATION_FIELD_NUMBER = 2;
+  /**
+   * <code>extend .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload { ... }</code>
+   */
+  public static final
+    com.google.protobuf.GeneratedMessage.GeneratedExtension<
+      org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages.AppendEntries.ReplicatedLogEntry.Payload,
+      org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.CompositeModification> modification = com.google.protobuf.GeneratedMessage
+          .newFileScopedGeneratedExtension(
+        org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.CompositeModification.class,
+        org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.CompositeModification.getDefaultInstance());
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\"CompositeModificationPayload.proto\022!or" +
+      "g.opendaylight.controller.mdsal\032\033AppendE" +
+      "ntriesMessages.proto\032\014Common.proto\032\020Pers" +
+      "istent.proto:\242\001\n\014modification\022R.org.open" +
+      "daylight.controller.cluster.raft.AppendE" +
+      "ntries.ReplicatedLogEntry.Payload\030\002 \001(\0132" +
+      "8.org.opendaylight.controller.mdsal.Comp" +
+      "ositeModification"
+    };
+    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;
+          modification.internalInit(descriptor.getExtensions().get(0));
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+          org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages.getDescriptor(),
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.getDescriptor(),
+          org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.getDescriptor(),
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}
index 35c2940be3e05aa9e03d8c922ecffde2cd0e823a..81e5b462cc56a6d27b95f1dd050f330b4dfd1520 100644 (file)
@@ -179,7 +179,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -195,7 +195,7 @@ public final class NormalizedNodeMessages {
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
@@ -222,7 +222,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -238,7 +238,7 @@ public final class NormalizedNodeMessages {
         getValueBytes() {
       java.lang.Object ref = value_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         value_ = b;
@@ -265,7 +265,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -281,7 +281,7 @@ public final class NormalizedNodeMessages {
         getTypeBytes() {
       java.lang.Object ref = type_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         type_ = b;
@@ -541,7 +541,7 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasName()) {
-
+          
           return false;
         }
         return true;
@@ -595,7 +595,7 @@ public final class NormalizedNodeMessages {
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
@@ -669,7 +669,7 @@ public final class NormalizedNodeMessages {
           getValueBytes() {
         java.lang.Object ref = value_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           value_ = b;
@@ -743,7 +743,7 @@ public final class NormalizedNodeMessages {
           getTypeBytes() {
         java.lang.Object ref = type_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           type_ = b;
@@ -930,7 +930,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -946,7 +946,7 @@ public final class NormalizedNodeMessages {
         getValueBytes() {
       java.lang.Object ref = value_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         value_ = b;
@@ -1168,7 +1168,7 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasValue()) {
-
+          
           return false;
         }
         return true;
@@ -1222,7 +1222,7 @@ public final class NormalizedNodeMessages {
           getValueBytes() {
         java.lang.Object ref = value_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           value_ = b;
@@ -1341,7 +1341,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute> 
         getAttributesList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
@@ -1354,7 +1354,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
         getAttributesOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
@@ -1504,7 +1504,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1520,7 +1520,7 @@ public final class NormalizedNodeMessages {
         getValueBytes() {
       java.lang.Object ref = value_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         value_ = b;
@@ -1555,7 +1555,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1575,7 +1575,7 @@ public final class NormalizedNodeMessages {
         getTypeBytes() {
       java.lang.Object ref = type_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         type_ = b;
@@ -1619,7 +1619,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
         getAttributesOrBuilderList() {
       return attributes_;
     }
@@ -1948,7 +1948,7 @@ public final class NormalizedNodeMessages {
               attributesBuilder_ = null;
               attributes_ = other.attributes_;
               bitField0_ = (bitField0_ & ~0x00000008);
-              attributesBuilder_ =
+              attributesBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getAttributesFieldBuilder() : null;
             } else {
@@ -1962,18 +1962,18 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasValue()) {
-
+          
           return false;
         }
         if (hasNodeType()) {
           if (!getNodeType().isInitialized()) {
-
+            
             return false;
           }
         }
         for (int i = 0; i < getAttributesCount(); i++) {
           if (!getAttributes(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -2028,7 +2028,7 @@ public final class NormalizedNodeMessages {
           getValueBytes() {
         java.lang.Object ref = value_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           value_ = b;
@@ -2114,7 +2114,7 @@ public final class NormalizedNodeMessages {
           getTypeBytes() {
         java.lang.Object ref = type_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           type_ = b;
@@ -2275,7 +2275,7 @@ public final class NormalizedNodeMessages {
        * <code>optional .org.opendaylight.controller.mdsal.QName nodeType = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QName, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QName.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QNameOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QName, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QName.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.QNameOrBuilder> 
           getNodeTypeFieldBuilder() {
         if (nodeTypeBuilder_ == null) {
           nodeTypeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -2483,7 +2483,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
            getAttributesOrBuilderList() {
         if (attributesBuilder_ != null) {
           return attributesBuilder_.getMessageOrBuilderList();
@@ -2509,12 +2509,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder> 
            getAttributesBuilderList() {
         return getAttributesFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
           getAttributesFieldBuilder() {
         if (attributesBuilder_ == null) {
           attributesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -2546,7 +2546,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument> 
         getArgumentsList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
@@ -2559,7 +2559,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder> 
         getArgumentsOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
@@ -2680,7 +2680,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder> 
         getArgumentsOrBuilderList() {
       return arguments_;
     }
@@ -2933,7 +2933,7 @@ public final class NormalizedNodeMessages {
               argumentsBuilder_ = null;
               arguments_ = other.arguments_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              argumentsBuilder_ =
+              argumentsBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getArgumentsFieldBuilder() : null;
             } else {
@@ -2948,7 +2948,7 @@ public final class NormalizedNodeMessages {
       public final boolean isInitialized() {
         for (int i = 0; i < getArgumentsCount(); i++) {
           if (!getArguments(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -3169,7 +3169,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder> 
            getArgumentsOrBuilderList() {
         if (argumentsBuilder_ != null) {
           return argumentsBuilder_.getMessageOrBuilderList();
@@ -3195,12 +3195,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument.Builder> 
            getArgumentsBuilderList() {
         return getArgumentsFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgument.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.PathArgumentOrBuilder> 
           getArgumentsFieldBuilder() {
         if (argumentsBuilder_ == null) {
           argumentsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -3262,7 +3262,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute> 
         getAttributesList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
@@ -3275,7 +3275,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
         getAttributesOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
@@ -3287,7 +3287,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node> 
         getChildList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
@@ -3300,7 +3300,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
         getChildOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
@@ -3546,7 +3546,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -3562,7 +3562,7 @@ public final class NormalizedNodeMessages {
         getPathBytes() {
       java.lang.Object ref = path_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         path_ = b;
@@ -3589,7 +3589,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -3605,7 +3605,7 @@ public final class NormalizedNodeMessages {
         getTypeBytes() {
       java.lang.Object ref = type_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         type_ = b;
@@ -3627,7 +3627,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
         getAttributesOrBuilderList() {
       return attributes_;
     }
@@ -3663,7 +3663,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
         getChildOrBuilderList() {
       return child_;
     }
@@ -3704,7 +3704,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -3720,7 +3720,7 @@ public final class NormalizedNodeMessages {
         getValueBytes() {
       java.lang.Object ref = value_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         value_ = b;
@@ -3747,7 +3747,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -3763,7 +3763,7 @@ public final class NormalizedNodeMessages {
         getValueTypeBytes() {
       java.lang.Object ref = valueType_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         valueType_ = b;
@@ -4202,7 +4202,7 @@ public final class NormalizedNodeMessages {
               attributesBuilder_ = null;
               attributes_ = other.attributes_;
               bitField0_ = (bitField0_ & ~0x00000004);
-              attributesBuilder_ =
+              attributesBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getAttributesFieldBuilder() : null;
             } else {
@@ -4228,7 +4228,7 @@ public final class NormalizedNodeMessages {
               childBuilder_ = null;
               child_ = other.child_;
               bitField0_ = (bitField0_ & ~0x00000008);
-              childBuilder_ =
+              childBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getChildFieldBuilder() : null;
             } else {
@@ -4266,19 +4266,19 @@ public final class NormalizedNodeMessages {
       public final boolean isInitialized() {
         for (int i = 0; i < getAttributesCount(); i++) {
           if (!getAttributes(i).isInitialized()) {
-
+            
             return false;
           }
         }
         for (int i = 0; i < getChildCount(); i++) {
           if (!getChild(i).isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasInstanceIdentifierValue()) {
           if (!getInstanceIdentifierValue().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -4333,7 +4333,7 @@ public final class NormalizedNodeMessages {
           getPathBytes() {
         java.lang.Object ref = path_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           path_ = b;
@@ -4407,7 +4407,7 @@ public final class NormalizedNodeMessages {
           getTypeBytes() {
         java.lang.Object ref = type_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           type_ = b;
@@ -4647,7 +4647,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
            getAttributesOrBuilderList() {
         if (attributesBuilder_ != null) {
           return attributesBuilder_.getMessageOrBuilderList();
@@ -4673,12 +4673,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder> 
            getAttributesBuilderList() {
         return getAttributesFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Attribute.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.AttributeOrBuilder> 
           getAttributesFieldBuilder() {
         if (attributesBuilder_ == null) {
           attributesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -4887,7 +4887,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
            getChildOrBuilderList() {
         if (childBuilder_ != null) {
           return childBuilder_.getMessageOrBuilderList();
@@ -4913,12 +4913,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Node child = 4;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder> 
            getChildBuilderList() {
         return getChildFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getChildFieldBuilder() {
         if (childBuilder_ == null) {
           childBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -4961,7 +4961,7 @@ public final class NormalizedNodeMessages {
           getValueBytes() {
         java.lang.Object ref = value_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           value_ = b;
@@ -5035,7 +5035,7 @@ public final class NormalizedNodeMessages {
           getValueTypeBytes() {
         java.lang.Object ref = valueType_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           valueType_ = b;
@@ -5277,7 +5277,7 @@ public final class NormalizedNodeMessages {
        * <code>optional .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierValue = 8;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierValueFieldBuilder() {
         if (instanceIdentifierValueBuilder_ == null) {
           instanceIdentifierValueBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -5459,7 +5459,7 @@ public final class NormalizedNodeMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -5475,7 +5475,7 @@ public final class NormalizedNodeMessages {
         getParentPathBytes() {
       java.lang.Object ref = parentPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         parentPath_ = b;
@@ -5751,12 +5751,12 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasParentPath()) {
-
+          
           return false;
         }
         if (hasNormalizedNode()) {
           if (!getNormalizedNode().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -5811,7 +5811,7 @@ public final class NormalizedNodeMessages {
           getParentPathBytes() {
         java.lang.Object ref = parentPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           parentPath_ = b;
@@ -5960,7 +5960,7 @@ public final class NormalizedNodeMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node normalizedNode = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -6431,16 +6431,16 @@ public final class NormalizedNodeMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPath()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPath().isInitialized()) {
-
+          
           return false;
         }
         if (hasNormalizedNode()) {
           if (!getNormalizedNode().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -6570,7 +6570,7 @@ public final class NormalizedNodeMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPath = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathFieldBuilder() {
         if (instanceIdentifierPathBuilder_ == null) {
           instanceIdentifierPathBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -6687,7 +6687,7 @@ public final class NormalizedNodeMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node normalizedNode = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -6718,7 +6718,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry> 
         getMapEntriesList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
@@ -6731,7 +6731,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder> 
         getMapEntriesOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
@@ -6852,7 +6852,7 @@ public final class NormalizedNodeMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder> 
         getMapEntriesOrBuilderList() {
       return mapEntries_;
     }
@@ -7105,7 +7105,7 @@ public final class NormalizedNodeMessages {
               mapEntriesBuilder_ = null;
               mapEntries_ = other.mapEntries_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              mapEntriesBuilder_ =
+              mapEntriesBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getMapEntriesFieldBuilder() : null;
             } else {
@@ -7120,7 +7120,7 @@ public final class NormalizedNodeMessages {
       public final boolean isInitialized() {
         for (int i = 0; i < getMapEntriesCount(); i++) {
           if (!getMapEntries(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -7341,7 +7341,7 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder> 
            getMapEntriesOrBuilderList() {
         if (mapEntriesBuilder_ != null) {
           return mapEntriesBuilder_.getMessageOrBuilderList();
@@ -7367,12 +7367,12 @@ public final class NormalizedNodeMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry.Builder> 
            getMapEntriesBuilderList() {
         return getMapEntriesFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntry.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapEntryOrBuilder> 
           getMapEntriesFieldBuilder() {
         if (mapEntriesBuilder_ == null) {
           mapEntriesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
index 67ab472d0ce27973757a8c22b1e104bd72550fe2..29e54571d30b6ddbb9392c6f3edbed83f09c5db8 100644 (file)
@@ -159,7 +159,7 @@ public final class SimpleNormalizedNodeMessage {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -175,7 +175,7 @@ public final class SimpleNormalizedNodeMessage {
         getNodeIdentifierBytes() {
       java.lang.Object ref = nodeIdentifier_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         nodeIdentifier_ = b;
@@ -202,7 +202,7 @@ public final class SimpleNormalizedNodeMessage {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -218,7 +218,7 @@ public final class SimpleNormalizedNodeMessage {
         getXmlStringBytes() {
       java.lang.Object ref = xmlString_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         xmlString_ = b;
@@ -463,11 +463,11 @@ public final class SimpleNormalizedNodeMessage {
 
       public final boolean isInitialized() {
         if (!hasNodeIdentifier()) {
-
+          
           return false;
         }
         if (!hasXmlString()) {
-
+          
           return false;
         }
         return true;
@@ -521,7 +521,7 @@ public final class SimpleNormalizedNodeMessage {
           getNodeIdentifierBytes() {
         java.lang.Object ref = nodeIdentifier_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           nodeIdentifier_ = b;
@@ -595,7 +595,7 @@ public final class SimpleNormalizedNodeMessage {
           getXmlStringBytes() {
         java.lang.Object ref = xmlString_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           xmlString_ = b;
index 384b389f9251b9a233cfbe817262eb10a4829443..201883476838f3c592f63e47c08fcde5a661b29a 100644 (file)
@@ -85,7 +85,7 @@ public final class DataChangeListenerMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier>
+    java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier> 
         getRemovedPathsList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
@@ -98,7 +98,7 @@ public final class DataChangeListenerMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
         getRemovedPathsOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
@@ -395,7 +395,7 @@ public final class DataChangeListenerMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
         getRemovedPathsOrBuilderList() {
       return removedPaths_;
     }
@@ -810,7 +810,7 @@ public final class DataChangeListenerMessages {
               removedPathsBuilder_ = null;
               removedPaths_ = other.removedPaths_;
               bitField0_ = (bitField0_ & ~0x00000020);
-              removedPathsBuilder_ =
+              removedPathsBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getRemovedPathsFieldBuilder() : null;
             } else {
@@ -825,37 +825,37 @@ public final class DataChangeListenerMessages {
       public final boolean isInitialized() {
         if (hasOriginalSubTree()) {
           if (!getOriginalSubTree().isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasUpdatedSubTree()) {
           if (!getUpdatedSubTree().isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasOriginalData()) {
           if (!getOriginalData().isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasUpdatedData()) {
           if (!getUpdatedData().isInitialized()) {
-
+            
             return false;
           }
         }
         if (hasCreatedData()) {
           if (!getCreatedData().isInitialized()) {
-
+            
             return false;
           }
         }
         for (int i = 0; i < getRemovedPathsCount(); i++) {
           if (!getRemovedPaths(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -985,7 +985,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node originalSubTree = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getOriginalSubTreeFieldBuilder() {
         if (originalSubTreeBuilder_ == null) {
           originalSubTreeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1102,7 +1102,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node updatedSubTree = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getUpdatedSubTreeFieldBuilder() {
         if (updatedSubTreeBuilder_ == null) {
           updatedSubTreeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1219,7 +1219,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.NodeMap originalData = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder> 
           getOriginalDataFieldBuilder() {
         if (originalDataBuilder_ == null) {
           originalDataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1336,7 +1336,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.NodeMap updatedData = 4;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder> 
           getUpdatedDataFieldBuilder() {
         if (updatedDataBuilder_ == null) {
           updatedDataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1453,7 +1453,7 @@ public final class DataChangeListenerMessages {
        * <code>optional .org.opendaylight.controller.mdsal.NodeMap createdData = 5;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMap.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeMapOrBuilder> 
           getCreatedDataFieldBuilder() {
         if (createdDataBuilder_ == null) {
           createdDataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1661,7 +1661,7 @@ public final class DataChangeListenerMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
            getRemovedPathsOrBuilderList() {
         if (removedPathsBuilder_ != null) {
           return removedPathsBuilder_.getMessageOrBuilderList();
@@ -1687,12 +1687,12 @@ public final class DataChangeListenerMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder> 
            getRemovedPathsBuilderList() {
         return getRemovedPathsFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getRemovedPathsFieldBuilder() {
         if (removedPathsBuilder_ == null) {
           removedPathsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
index 6c1e2722f65cfb69ba10a73683456d67ea012a78..eaa90012dbdc58425adc8fbfa2ea4e467f0c4379 100644 (file)
@@ -193,7 +193,7 @@ public final class PersistentMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -209,7 +209,7 @@ public final class PersistentMessages {
         getTypeBytes() {
       java.lang.Object ref = type_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         type_ = b;
@@ -541,20 +541,20 @@ public final class PersistentMessages {
 
       public final boolean isInitialized() {
         if (!hasType()) {
-
+          
           return false;
         }
         if (!hasPath()) {
-
+          
           return false;
         }
         if (!getPath().isInitialized()) {
-
+          
           return false;
         }
         if (hasData()) {
           if (!getData().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -609,7 +609,7 @@ public final class PersistentMessages {
           getTypeBytes() {
         java.lang.Object ref = type_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           type_ = b;
@@ -758,7 +758,7 @@ public final class PersistentMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier path = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getPathFieldBuilder() {
         if (pathBuilder_ == null) {
           pathBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -875,7 +875,7 @@ public final class PersistentMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node data = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getDataFieldBuilder() {
         if (dataBuilder_ == null) {
           dataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -906,7 +906,7 @@ public final class PersistentMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
      */
-    java.util.List<org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification>
+    java.util.List<org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification> 
         getModificationList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
@@ -919,7 +919,7 @@ public final class PersistentMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
      */
-    java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder>
+    java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder> 
         getModificationOrBuilderList();
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
@@ -1040,7 +1040,7 @@ public final class PersistentMessages {
     /**
      * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
      */
-    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder>
+    public java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder> 
         getModificationOrBuilderList() {
       return modification_;
     }
@@ -1293,7 +1293,7 @@ public final class PersistentMessages {
               modificationBuilder_ = null;
               modification_ = other.modification_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              modificationBuilder_ =
+              modificationBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getModificationFieldBuilder() : null;
             } else {
@@ -1308,7 +1308,7 @@ public final class PersistentMessages {
       public final boolean isInitialized() {
         for (int i = 0; i < getModificationCount(); i++) {
           if (!getModification(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -1529,7 +1529,7 @@ public final class PersistentMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
        */
-      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder>
+      public java.util.List<? extends org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder> 
            getModificationOrBuilderList() {
         if (modificationBuilder_ != null) {
           return modificationBuilder_.getMessageOrBuilderList();
@@ -1555,12 +1555,12 @@ public final class PersistentMessages {
       /**
        * <code>repeated .org.opendaylight.controller.mdsal.Modification modification = 1;</code>
        */
-      public java.util.List<org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification.Builder>
+      public java.util.List<org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification.Builder> 
            getModificationBuilderList() {
         return getModificationFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification, org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification.Builder, org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder>
+          org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification, org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.Modification.Builder, org.opendaylight.controller.protobuff.messages.persistent.PersistentMessages.ModificationOrBuilder> 
           getModificationFieldBuilder() {
         if (modificationBuilder_ == null) {
           modificationBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
@@ -1605,15 +1605,16 @@ public final class PersistentMessages {
   static {
     java.lang.String[] descriptorData = {
       "\n\020Persistent.proto\022!org.opendaylight.con" +
-      "troller.mdsal\032\014Common.proto\"\230\001\n\014Modifica" +
-      "tion\022\014\n\004type\030\001 \002(\t\022C\n\004path\030\002 \002(\01325.org.o" +
-      "pendaylight.controller.mdsal.InstanceIde" +
-      "ntifier\0225\n\004data\030\003 \001(\0132\'.org.opendaylight" +
-      ".controller.mdsal.Node\"^\n\025CompositeModif" +
-      "ication\022E\n\014modification\030\001 \003(\0132/.org.open" +
-      "daylight.controller.mdsal.ModificationBO" +
-      "\n9org.opendaylight.controller.protobuff." +
-      "messages.persistentB\022PersistentMessages"
+      "troller.mdsal\032\014Common.proto\032\033AppendEntri" +
+      "esMessages.proto\"\230\001\n\014Modification\022\014\n\004typ" +
+      "e\030\001 \002(\t\022C\n\004path\030\002 \002(\01325.org.opendaylight" +
+      ".controller.mdsal.InstanceIdentifier\0225\n\004" +
+      "data\030\003 \001(\0132\'.org.opendaylight.controller" +
+      ".mdsal.Node\"^\n\025CompositeModification\022E\n\014" +
+      "modification\030\001 \003(\0132/.org.opendaylight.co" +
+      "ntroller.mdsal.ModificationBO\n9org.opend" +
+      "aylight.controller.protobuff.messages.pe",
+      "rsistentB\022PersistentMessages"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -1639,6 +1640,7 @@ public final class PersistentMessages {
       .internalBuildGeneratedFileFrom(descriptorData,
         new com.google.protobuf.Descriptors.FileDescriptor[] {
           org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.getDescriptor(),
+          org.opendaylight.controller.cluster.raft.protobuff.messages.AppendEntriesMessages.getDescriptor(),
         }, assigner);
   }
 
index 77cbd4da46c811e43a3a2692a40f007ab51e5097..e06dd0d42982d779b8aa40931338b88c528d6069 100644 (file)
@@ -837,7 +837,7 @@ public final class ListenerRegistrationMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -853,7 +853,7 @@ public final class ListenerRegistrationMessages {
         getDataChangeListenerActorPathBytes() {
       java.lang.Object ref = dataChangeListenerActorPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         dataChangeListenerActorPath_ = b;
@@ -1146,19 +1146,19 @@ public final class ListenerRegistrationMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPath()) {
-
+          
           return false;
         }
         if (!hasDataChangeListenerActorPath()) {
-
+          
           return false;
         }
         if (!hasDataChangeScope()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPath().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -1287,7 +1287,7 @@ public final class ListenerRegistrationMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPath = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathFieldBuilder() {
         if (instanceIdentifierPathBuilder_ == null) {
           instanceIdentifierPathBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -1329,7 +1329,7 @@ public final class ListenerRegistrationMessages {
           getDataChangeListenerActorPathBytes() {
         java.lang.Object ref = dataChangeListenerActorPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           dataChangeListenerActorPath_ = b;
@@ -1556,7 +1556,7 @@ public final class ListenerRegistrationMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1572,7 +1572,7 @@ public final class ListenerRegistrationMessages {
         getListenerRegistrationPathBytes() {
       java.lang.Object ref = listenerRegistrationPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         listenerRegistrationPath_ = b;
@@ -1801,7 +1801,7 @@ public final class ListenerRegistrationMessages {
 
       public final boolean isInitialized() {
         if (!hasListenerRegistrationPath()) {
-
+          
           return false;
         }
         return true;
@@ -1855,7 +1855,7 @@ public final class ListenerRegistrationMessages {
           getListenerRegistrationPathBytes() {
         java.lang.Object ref = listenerRegistrationPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           listenerRegistrationPath_ = b;
index 2324dfc2a238b1e197a04c360ee1c1b0db66ebf4..7c2a47e1b0daa740f96acf355b4281f9331ec7f2 100644 (file)
@@ -139,7 +139,7 @@ public final class ShardManagerMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -155,7 +155,7 @@ public final class ShardManagerMessages {
         getShardNameBytes() {
       java.lang.Object ref = shardName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         shardName_ = b;
@@ -377,7 +377,7 @@ public final class ShardManagerMessages {
 
       public final boolean isInitialized() {
         if (!hasShardName()) {
-
+          
           return false;
         }
         return true;
@@ -431,7 +431,7 @@ public final class ShardManagerMessages {
           getShardNameBytes() {
         java.lang.Object ref = shardName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           shardName_ = b;
@@ -618,7 +618,7 @@ public final class ShardManagerMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -634,7 +634,7 @@ public final class ShardManagerMessages {
         getPrimaryPathBytes() {
       java.lang.Object ref = primaryPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         primaryPath_ = b;
@@ -856,7 +856,7 @@ public final class ShardManagerMessages {
 
       public final boolean isInitialized() {
         if (!hasPrimaryPath()) {
-
+          
           return false;
         }
         return true;
@@ -910,7 +910,7 @@ public final class ShardManagerMessages {
           getPrimaryPathBytes() {
         java.lang.Object ref = primaryPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           primaryPath_ = b;
@@ -1097,7 +1097,7 @@ public final class ShardManagerMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1113,7 +1113,7 @@ public final class ShardManagerMessages {
         getShardNameBytes() {
       java.lang.Object ref = shardName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         shardName_ = b;
@@ -1335,7 +1335,7 @@ public final class ShardManagerMessages {
 
       public final boolean isInitialized() {
         if (!hasShardName()) {
-
+          
           return false;
         }
         return true;
@@ -1389,7 +1389,7 @@ public final class ShardManagerMessages {
           getShardNameBytes() {
         java.lang.Object ref = shardName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           shardName_ = b;
index 63dd5e7081603aed89168980a5f6ad6bb27de28e..3f354ba40eeea6751b1410c47a5bcf7b39db86a6 100644 (file)
@@ -1066,7 +1066,7 @@ public final class ShardTransactionChainMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1082,7 +1082,7 @@ public final class ShardTransactionChainMessages {
         getTransactionChainPathBytes() {
       java.lang.Object ref = transactionChainPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         transactionChainPath_ = b;
@@ -1304,7 +1304,7 @@ public final class ShardTransactionChainMessages {
 
       public final boolean isInitialized() {
         if (!hasTransactionChainPath()) {
-
+          
           return false;
         }
         return true;
@@ -1358,7 +1358,7 @@ public final class ShardTransactionChainMessages {
           getTransactionChainPathBytes() {
         java.lang.Object ref = transactionChainPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           transactionChainPath_ = b;
index 7ce3b586b43a6e5e470297b13d65bb5a8e199c80..ee2c70423e14fcaa8775d285f22cd2a8bb02f324 100644 (file)
@@ -757,7 +757,7 @@ public final class ShardTransactionMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -773,7 +773,7 @@ public final class ShardTransactionMessages {
         getTransactionIdBytes() {
       java.lang.Object ref = transactionId_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         transactionId_ = b;
@@ -995,7 +995,7 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasTransactionId()) {
-
+          
           return false;
         }
         return true;
@@ -1049,7 +1049,7 @@ public final class ShardTransactionMessages {
           getTransactionIdBytes() {
         java.lang.Object ref = transactionId_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           transactionId_ = b;
@@ -1256,7 +1256,7 @@ public final class ShardTransactionMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1272,7 +1272,7 @@ public final class ShardTransactionMessages {
         getTransactionActorPathBytes() {
       java.lang.Object ref = transactionActorPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         transactionActorPath_ = b;
@@ -1299,7 +1299,7 @@ public final class ShardTransactionMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -1315,7 +1315,7 @@ public final class ShardTransactionMessages {
         getTransactionIdBytes() {
       java.lang.Object ref = transactionId_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         transactionId_ = b;
@@ -1560,11 +1560,11 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasTransactionActorPath()) {
-
+          
           return false;
         }
         if (!hasTransactionId()) {
-
+          
           return false;
         }
         return true;
@@ -1618,7 +1618,7 @@ public final class ShardTransactionMessages {
           getTransactionActorPathBytes() {
         java.lang.Object ref = transactionActorPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           transactionActorPath_ = b;
@@ -1692,7 +1692,7 @@ public final class ShardTransactionMessages {
           getTransactionIdBytes() {
         java.lang.Object ref = transactionId_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           transactionId_ = b;
@@ -2188,7 +2188,7 @@ public final class ShardTransactionMessages {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs =
+        com.google.protobuf.ByteString bs = 
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -2204,7 +2204,7 @@ public final class ShardTransactionMessages {
         getActorPathBytes() {
       java.lang.Object ref = actorPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b =
+        com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         actorPath_ = b;
@@ -2426,7 +2426,7 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasActorPath()) {
-
+          
           return false;
         }
         return true;
@@ -2480,7 +2480,7 @@ public final class ShardTransactionMessages {
           getActorPathBytes() {
         java.lang.Object ref = actorPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b =
+          com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           actorPath_ = b;
@@ -2902,11 +2902,11 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPathArguments()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPathArguments().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -3035,7 +3035,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathArgumentsFieldBuilder() {
         if (instanceIdentifierPathArgumentsBuilder_ == null) {
           instanceIdentifierPathArgumentsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -3734,11 +3734,11 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPathArguments()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPathArguments().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -3867,7 +3867,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathArgumentsFieldBuilder() {
         if (instanceIdentifierPathArgumentsBuilder_ == null) {
           instanceIdentifierPathArgumentsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -4256,7 +4256,7 @@ public final class ShardTransactionMessages {
       public final boolean isInitialized() {
         if (hasNormalizedNode()) {
           if (!getNormalizedNode().isInitialized()) {
-
+            
             return false;
           }
         }
@@ -4386,7 +4386,7 @@ public final class ShardTransactionMessages {
        * <code>optional .org.opendaylight.controller.mdsal.Node normalizedNode = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -4859,19 +4859,19 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPathArguments()) {
-
+          
           return false;
         }
         if (!hasNormalizedNode()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPathArguments().isInitialized()) {
-
+          
           return false;
         }
         if (!getNormalizedNode().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -5000,7 +5000,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathArgumentsFieldBuilder() {
         if (instanceIdentifierPathArgumentsBuilder_ == null) {
           instanceIdentifierPathArgumentsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -5117,7 +5117,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.Node normalizedNode = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -5899,19 +5899,19 @@ public final class ShardTransactionMessages {
 
       public final boolean isInitialized() {
         if (!hasInstanceIdentifierPathArguments()) {
-
+          
           return false;
         }
         if (!hasNormalizedNode()) {
-
+          
           return false;
         }
         if (!getInstanceIdentifierPathArguments().isInitialized()) {
-
+          
           return false;
         }
         if (!getNormalizedNode().isInitialized()) {
-
+          
           return false;
         }
         return true;
@@ -6040,7 +6040,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifier.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.InstanceIdentifierOrBuilder> 
           getInstanceIdentifierPathArgumentsFieldBuilder() {
         if (instanceIdentifierPathArgumentsBuilder_ == null) {
           instanceIdentifierPathArgumentsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
@@ -6157,7 +6157,7 @@ public final class ShardTransactionMessages {
        * <code>required .org.opendaylight.controller.mdsal.Node normalizedNode = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder>
+          org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.Node.Builder, org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages.NodeOrBuilder> 
           getNormalizedNodeFieldBuilder() {
         if (normalizedNodeBuilder_ == null) {
           normalizedNodeBuilder_ = new com.google.protobuf.SingleFieldBuilder<
diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/CompositeModificationPayload.proto b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/CompositeModificationPayload.proto
new file mode 100644 (file)
index 0000000..b571cd2
--- /dev/null
@@ -0,0 +1,11 @@
+package org.opendaylight.controller.mdsal;
+
+import "AppendEntriesMessages.proto";
+import "Common.proto";
+import "Persistent.proto";
+
+extend org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload {
+    optional CompositeModification modification=2;
+}
+
+
index b3f28197ed3e24459d721071c476dae1b8d52bb3..8e834494cbe590aabdb2c68a49389f5d05c040aa 100644 (file)
@@ -1,6 +1,7 @@
 package org.opendaylight.controller.mdsal;
 
 import "Common.proto";
+import "AppendEntriesMessages.proto";
 
 option java_package = "org.opendaylight.controller.protobuff.messages.persistent";
 option java_outer_classname = "PersistentMessages";
@@ -16,3 +17,4 @@ message Modification {
 message CompositeModification {
     repeated Modification modification=1;
 }
+