From: Ed Warnicke Date: Fri, 1 Aug 2014 01:26:27 +0000 (+0000) Subject: Merge "Add replication capability to Shard" X-Git-Tag: release/helium~385 X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=commitdiff_plain;h=07ba9a998f0b3c3045ed8e31afda5c96de141b3b;hp=8ddcccfc1045eec79f6a22dc68250d95fc23a017 Merge "Add replication capability to Shard" --- diff --git a/opendaylight/md-sal/pom.xml b/opendaylight/md-sal/pom.xml index 5e96e1cc0d..3f02765aff 100644 --- a/opendaylight/md-sal/pom.xml +++ b/opendaylight/md-sal/pom.xml @@ -66,7 +66,7 @@ sal-protocolbuffer-encoding - + sal-distributed-datastore diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java index 15239795a8..b8e9653bc5 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java @@ -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. + *

+ * 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 + *

+ * 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); diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContext.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContext.java index 7150ec0e6e..ae9431a43a 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContext.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContext.java @@ -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 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 + *

+ * + * + * @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. + * + *

+ * Throws an IllegalStateException if the peer is unknown + * + * @param peerId + * @param peerAddress + */ + void setPeerAddress(String peerId, String peerAddress); } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContextImpl.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContextImpl.java index a0f13280c2..833c8a9e8a 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContextImpl.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorContextImpl.java @@ -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); + } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplySnapshot.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplySnapshot.java similarity index 89% rename from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplySnapshot.java rename to opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplySnapshot.java index f9c7c133ee..9739fb2f1b 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplySnapshot.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplySnapshot.java @@ -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 java.io.Serializable; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplyState.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplyState.java similarity index 94% rename from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplyState.java rename to opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplyState.java index f58d74829a..b904335be3 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ApplyState.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ApplyState.java @@ -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; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/CommitEntry.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/CommitEntry.java similarity index 86% rename from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/CommitEntry.java rename to opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/CommitEntry.java index ba321c6c0e..07e376fca3 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/CommitEntry.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/CommitEntry.java @@ -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 java.io.Serializable; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ElectionTimeout.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ElectionTimeout.java similarity index 85% rename from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ElectionTimeout.java rename to opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ElectionTimeout.java index 1c58f4bd31..a844849f15 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/ElectionTimeout.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/ElectionTimeout.java @@ -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 java.io.Serializable; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/PersistEntry.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/PersistEntry.java similarity index 87% rename from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/PersistEntry.java rename to opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/PersistEntry.java index fe0dcc550b..6a62817e90 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/PersistEntry.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/PersistEntry.java @@ -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 java.io.Serializable; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/Replicate.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/Replicate.java similarity index 94% rename from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/Replicate.java rename to opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/Replicate.java index e9cdcf0b86..9bc737a0e3 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/Replicate.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/Replicate.java @@ -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; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SaveSnapshot.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SaveSnapshot.java similarity index 88% rename from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SaveSnapshot.java rename to opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SaveSnapshot.java index ce0ad1f501..861f5ee715 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SaveSnapshot.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SaveSnapshot.java @@ -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 java.io.Serializable; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendHeartBeat.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendHeartBeat.java similarity index 88% rename from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendHeartBeat.java rename to opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendHeartBeat.java index 1555766cdb..3c8c845f5d 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendHeartBeat.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendHeartBeat.java @@ -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 java.io.Serializable; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendInstallSnapshot.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendInstallSnapshot.java similarity index 85% rename from opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendInstallSnapshot.java rename to opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendInstallSnapshot.java index b85e58b4f8..6c3313f316 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/internal/messages/SendInstallSnapshot.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/base/messages/SendInstallSnapshot.java @@ -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 java.io.Serializable; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java index bc87370b15..f7281bb8e3 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehavior.java @@ -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; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java index ecd4901246..c125bd32b6 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java @@ -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 peerToActor = new HashMap<>(); - private int voteCount; private final int votesRequired; + private final Set peers; + public Candidate(RaftActorContext context) { super(context); - Collection 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() + ); + } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java index 68b444b2ba..db62dfc2ac 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java @@ -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() diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Leader.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Leader.java index 26beed2f7a..53e47c2f84 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Leader.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Leader.java @@ -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 followerToLog = new HashMap(); - private final Map followerToActor = new HashMap<>(); + private final Set 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 entries = Collections.emptyList(); + List 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; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java index 45ac9a9ebf..94366efd5e 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntries.java @@ -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 Object toSerializable(){ diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java index 7e6628abe1..b923baa716 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/AppendEntriesReply.java @@ -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(); + } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java index 5053560b41..888854fa71 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshot.java @@ -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; diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java index 02a3252776..85b89b70ae 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/InstallSnapshotReply.java @@ -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 diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RaftRPC.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RaftRPC.java index a770e54f58..10d99988d8 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RaftRPC.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RaftRPC.java @@ -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(); } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java index 310968de95..6ef2a06285 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVote.java @@ -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(); + } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java index a658ab7d81..df80b4e5e2 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/messages/RequestVoteReply.java @@ -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; diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java index 406c164cdb..2e200cba27 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java @@ -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 peerAddresses) { this.peerAddresses = peerAddresses; } diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/CandidateTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/CandidateTest.java index 8bcee58afe..c763683705 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/CandidateTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/CandidateTest.java @@ -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; diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java index 126e7d10c2..c015d950c4 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java @@ -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; diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java index bc9d4a2767..d33b33925b 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java @@ -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; diff --git a/opendaylight/md-sal/sal-distributed-datastore/pom.xml b/opendaylight/md-sal/sal-distributed-datastore/pom.xml index 16b97b7855..94c895dab3 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/pom.xml +++ b/opendaylight/md-sal/sal-distributed-datastore/pom.xml @@ -124,6 +124,12 @@ 1.1-SNAPSHOT + + org.opendaylight.controller + sal-akka-raft + 1.1-SNAPSHOT + + junit @@ -160,6 +166,7 @@ !*snappy;!org.jboss.*;* 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 index 0000000000..955e4bbf22 --- /dev/null +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/CompositeModificationPayload.java @@ -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 encode() { + Preconditions.checkState(modification!=null); + Map 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; + } +} diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Configuration.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Configuration.java index a67f58c760..34239070a3 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Configuration.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Configuration.java @@ -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 getMemberShardNames(String memberName); + + /** + * Given a module namespace return the name of a module + * @param nameSpace + * @return + */ Optional getModuleNameFromNameSpace(String nameSpace); + + /** + * Get a mapping of the module names to it's corresponding ShardStrategy + * @return + */ Map getModuleNameToShardStrategyMap(); + + /** + * Given a module name find all the shardNames corresponding to it + * @param moduleName + * @return + */ List getShardNamesFromModuleName(String moduleName); + + /** + * Given a shardName find all the members on which it belongs + * + * @param shardName + * @return + */ List 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 index 0000000000..eb28159025 --- /dev/null +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpCohort.java @@ -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); + } + + } +} + diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java index d6ad553cf3..999d0f8baf 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java @@ -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 *

*/ -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 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 peerAddresses) { return Props.create(new Creator() { @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 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> 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; + } } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java index 0363b3ceb3..5fbce4cd98 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java @@ -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 memberNameToAddress = new HashMap<>(); - private final Map localShards = new HashMap<>(); + private final Map 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 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 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 getPeerAddresses(String shardName){ + + Map peerAddresses = new HashMap<>(); + + List 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 peerAddresses; + + private ShardInformation(String shardName, ActorRef actor, + Map 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 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()); + + } + } + } } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java index 5a0049aa6d..c85d32012f 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java @@ -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 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>> call = new Callable() { - - @Override public Optional> 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>) Optional.of(reply.getNormalizedNode()); - } - - return Optional.absent(); - } - }; - - ListenableFutureTask>> - 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 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>> 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>> readData(final YangInstanceIdentifier path) { + + Callable>> call = new Callable() { + + @Override public Optional> 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>) Optional.of(reply.getNormalizedNode()); + } + + return Optional.absent(); + } + }; + + ListenableFutureTask>> + 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>> readData( + YangInstanceIdentifier path) { + LOG.error("readData called path = {}", path); + return Futures.immediateFuture( + Optional.>absent()); + } + + @Override public void writeData(YangInstanceIdentifier path, + NormalizedNode data) { + LOG.error("writeData called path = {}", path); + } } + + } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/PrimaryNotFoundException.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/PrimaryNotFoundException.java index 5a131ade33..7b07053fd9 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/PrimaryNotFoundException.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/PrimaryNotFoundException.java @@ -9,4 +9,7 @@ package org.opendaylight.controller.cluster.datastore.exceptions; public class PrimaryNotFoundException extends RuntimeException { + public PrimaryNotFoundException(String message){ + super(message); + } } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/TimeoutException.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/TimeoutException.java index 4780aaccfb..472cd38734 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/TimeoutException.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/exceptions/TimeoutException.java @@ -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 index 0000000000..8c2543e486 --- /dev/null +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/messages/PeerAddressResolved.java @@ -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; + } +} diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java index c7ee7d8c2c..ac0893da5a 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java @@ -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 graphs = CacheBuilder.newBuilder() - .expireAfterAccess(2, TimeUnit.SECONDS) - .build( - new CacheLoader() { - 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(); diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractActorTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractActorTest.java index 214b3e9d3d..e23a76b0b2 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractActorTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractActorTest.java @@ -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() { diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java index b62a4b36d5..11ad559744 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java @@ -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"); } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreIntegrationTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreIntegrationTest.java index 116e5e75b5..b5e3d24ef6 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreIntegrationTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DistributedDataStoreIntegrationTest.java @@ -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(); diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java index f20cd8c3d7..7d57ea8284 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java @@ -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 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> noOpDataChangeListener() { return new AsyncDataChangeListener>() { diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java index 4d7c61a197..7884eeccda 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java @@ -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 = diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/resources/module-shards.conf b/opendaylight/md-sal/sal-distributed-datastore/src/test/resources/module-shards.conf index 9f35f2b9f2..b3de998479 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/resources/module-shards.conf +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/resources/module-shards.conf @@ -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" ] } ] diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/resources/modules.conf b/opendaylight/md-sal/sal-distributed-datastore/src/test/resources/modules.conf index 52f249a7ff..22854cb11a 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/resources/modules.conf +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/resources/modules.conf @@ -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" } ] diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/pom.xml b/opendaylight/md-sal/sal-protocolbuffer-encoding/pom.xml index e3fac63a83..28e629a92c 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/pom.xml +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/pom.xml @@ -114,6 +114,13 @@ org.opendaylight.yangtools yang-binding + + + org.opendaylight.controller + sal-akka-raft + 1.1-SNAPSHOT + + com.google.guava guava 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 index 0000000000..87b246bd7e --- /dev/null +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/mdsal/CompositeModificationPayload.java @@ -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; + /** + * extend .org.opendaylight.controller.cluster.raft.AppendEntries.ReplicatedLogEntry.Payload { ... } + */ + 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) +} diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/cohort3pc/ThreePhaseCommitCohortMessages.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/cohort3pc/ThreePhaseCommitCohortMessages.java index 22a93c0e10..ac0701a6d8 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/cohort3pc/ThreePhaseCommitCohortMessages.java +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/cohort3pc/ThreePhaseCommitCohortMessages.java @@ -652,7 +652,7 @@ public final class ThreePhaseCommitCohortMessages { public final boolean isInitialized() { if (!hasCanCommit()) { - + return false; } return true; diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/NormalizedNodeMessages.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/NormalizedNodeMessages.java index 35c2940be3..81e5b462cc 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/NormalizedNodeMessages.java +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/NormalizedNodeMessages.java @@ -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 { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4; */ - java.util.List + java.util.List getAttributesList(); /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4; @@ -1354,7 +1354,7 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4; */ - java.util.List + java.util.List getAttributesOrBuilderList(); /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4; @@ -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 { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4; */ - public java.util.List + public java.util.List 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 { * optional .org.opendaylight.controller.mdsal.QName nodeType = 3; */ 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 { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4; */ - public java.util.List + public java.util.List getAttributesOrBuilderList() { if (attributesBuilder_ != null) { return attributesBuilder_.getMessageOrBuilderList(); @@ -2509,12 +2509,12 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 4; */ - public java.util.List + public java.util.List 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 { /** * repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1; */ - java.util.List + java.util.List getArgumentsList(); /** * repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1; @@ -2559,7 +2559,7 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1; */ - java.util.List + java.util.List getArgumentsOrBuilderList(); /** * repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1; @@ -2680,7 +2680,7 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1; */ - public java.util.List + public java.util.List 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 { /** * repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1; */ - public java.util.List + public java.util.List getArgumentsOrBuilderList() { if (argumentsBuilder_ != null) { return argumentsBuilder_.getMessageOrBuilderList(); @@ -3195,12 +3195,12 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.PathArgument arguments = 1; */ - public java.util.List + public java.util.List 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 { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3; */ - java.util.List + java.util.List getAttributesList(); /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3; @@ -3275,7 +3275,7 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3; */ - java.util.List + java.util.List getAttributesOrBuilderList(); /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3; @@ -3287,7 +3287,7 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.Node child = 4; */ - java.util.List + java.util.List getChildList(); /** * repeated .org.opendaylight.controller.mdsal.Node child = 4; @@ -3300,7 +3300,7 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.Node child = 4; */ - java.util.List + java.util.List getChildOrBuilderList(); /** * repeated .org.opendaylight.controller.mdsal.Node child = 4; @@ -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 { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3; */ - public java.util.List + public java.util.List getAttributesOrBuilderList() { return attributes_; } @@ -3663,7 +3663,7 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.Node child = 4; */ - public java.util.List + public java.util.List 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 { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3; */ - public java.util.List + public java.util.List getAttributesOrBuilderList() { if (attributesBuilder_ != null) { return attributesBuilder_.getMessageOrBuilderList(); @@ -4673,12 +4673,12 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.Attribute attributes = 3; */ - public java.util.List + public java.util.List 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 { /** * repeated .org.opendaylight.controller.mdsal.Node child = 4; */ - public java.util.List + public java.util.List getChildOrBuilderList() { if (childBuilder_ != null) { return childBuilder_.getMessageOrBuilderList(); @@ -4913,12 +4913,12 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.Node child = 4; */ - public java.util.List + public java.util.List 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 { * optional .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierValue = 8; */ 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 { * optional .org.opendaylight.controller.mdsal.Node normalizedNode = 2; */ 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 { * required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPath = 1; */ 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 { * optional .org.opendaylight.controller.mdsal.Node normalizedNode = 2; */ 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 { /** * repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1; */ - java.util.List + java.util.List getMapEntriesList(); /** * repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1; @@ -6731,7 +6731,7 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1; */ - java.util.List + java.util.List getMapEntriesOrBuilderList(); /** * repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1; @@ -6852,7 +6852,7 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1; */ - public java.util.List + public java.util.List 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 { /** * repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1; */ - public java.util.List + public java.util.List getMapEntriesOrBuilderList() { if (mapEntriesBuilder_ != null) { return mapEntriesBuilder_.getMessageOrBuilderList(); @@ -7367,12 +7367,12 @@ public final class NormalizedNodeMessages { /** * repeated .org.opendaylight.controller.mdsal.NodeMapEntry mapEntries = 1; */ - public java.util.List + public java.util.List 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< diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/SimpleNormalizedNodeMessage.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/SimpleNormalizedNodeMessage.java index 67ab472d0c..29e54571d3 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/SimpleNormalizedNodeMessage.java +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/common/SimpleNormalizedNodeMessage.java @@ -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; diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/datachange/notification/DataChangeListenerMessages.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/datachange/notification/DataChangeListenerMessages.java index 384b389f92..2018834768 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/datachange/notification/DataChangeListenerMessages.java +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/datachange/notification/DataChangeListenerMessages.java @@ -85,7 +85,7 @@ public final class DataChangeListenerMessages { /** * repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6; */ - java.util.List + java.util.List getRemovedPathsList(); /** * repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6; @@ -98,7 +98,7 @@ public final class DataChangeListenerMessages { /** * repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6; */ - java.util.List + java.util.List getRemovedPathsOrBuilderList(); /** * repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6; @@ -395,7 +395,7 @@ public final class DataChangeListenerMessages { /** * repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6; */ - public java.util.List + public java.util.List 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 { * optional .org.opendaylight.controller.mdsal.Node originalSubTree = 1; */ 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 { * optional .org.opendaylight.controller.mdsal.Node updatedSubTree = 2; */ 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 { * optional .org.opendaylight.controller.mdsal.NodeMap originalData = 3; */ 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 { * optional .org.opendaylight.controller.mdsal.NodeMap updatedData = 4; */ 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 { * optional .org.opendaylight.controller.mdsal.NodeMap createdData = 5; */ 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 { /** * repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6; */ - public java.util.List + public java.util.List getRemovedPathsOrBuilderList() { if (removedPathsBuilder_ != null) { return removedPathsBuilder_.getMessageOrBuilderList(); @@ -1687,12 +1687,12 @@ public final class DataChangeListenerMessages { /** * repeated .org.opendaylight.controller.mdsal.InstanceIdentifier removedPaths = 6; */ - public java.util.List + public java.util.List 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< diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/persistent/PersistentMessages.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/persistent/PersistentMessages.java index 6c1e2722f6..eaa90012db 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/persistent/PersistentMessages.java +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/persistent/PersistentMessages.java @@ -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 { * required .org.opendaylight.controller.mdsal.InstanceIdentifier path = 2; */ 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 { * optional .org.opendaylight.controller.mdsal.Node data = 3; */ 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 { /** * repeated .org.opendaylight.controller.mdsal.Modification modification = 1; */ - java.util.List + java.util.List getModificationList(); /** * repeated .org.opendaylight.controller.mdsal.Modification modification = 1; @@ -919,7 +919,7 @@ public final class PersistentMessages { /** * repeated .org.opendaylight.controller.mdsal.Modification modification = 1; */ - java.util.List + java.util.List getModificationOrBuilderList(); /** * repeated .org.opendaylight.controller.mdsal.Modification modification = 1; @@ -1040,7 +1040,7 @@ public final class PersistentMessages { /** * repeated .org.opendaylight.controller.mdsal.Modification modification = 1; */ - public java.util.List + public java.util.List 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 { /** * repeated .org.opendaylight.controller.mdsal.Modification modification = 1; */ - public java.util.List + public java.util.List getModificationOrBuilderList() { if (modificationBuilder_ != null) { return modificationBuilder_.getMessageOrBuilderList(); @@ -1555,12 +1555,12 @@ public final class PersistentMessages { /** * repeated .org.opendaylight.controller.mdsal.Modification modification = 1; */ - public java.util.List + public java.util.List 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); } diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/registration/ListenerRegistrationMessages.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/registration/ListenerRegistrationMessages.java index 77cbd4da46..e06dd0d429 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/registration/ListenerRegistrationMessages.java +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/registration/ListenerRegistrationMessages.java @@ -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 { * required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPath = 1; */ 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; diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/shard/ShardManagerMessages.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/shard/ShardManagerMessages.java index 2324dfc2a2..7c2a47e1b0 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/shard/ShardManagerMessages.java +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/shard/ShardManagerMessages.java @@ -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; diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionChainMessages.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionChainMessages.java index 63dd5e7081..3f354ba40e 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionChainMessages.java +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionChainMessages.java @@ -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; diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionMessages.java b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionMessages.java index 7ce3b586b4..ee2c70423e 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionMessages.java +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/java/org/opendaylight/controller/protobuff/messages/transaction/ShardTransactionMessages.java @@ -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 { * required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1; */ 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 { * required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1; */ 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 { * optional .org.opendaylight.controller.mdsal.Node normalizedNode = 1; */ 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 { * required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1; */ 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 { * required .org.opendaylight.controller.mdsal.Node normalizedNode = 2; */ 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 { * required .org.opendaylight.controller.mdsal.InstanceIdentifier instanceIdentifierPathArguments = 1; */ 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 { * required .org.opendaylight.controller.mdsal.Node normalizedNode = 2; */ 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 index 0000000000..b571cd25c5 --- /dev/null +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/CompositeModificationPayload.proto @@ -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; +} + + diff --git a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/Persistent.proto b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/Persistent.proto index b3f28197ed..8e834494cb 100644 --- a/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/Persistent.proto +++ b/opendaylight/md-sal/sal-protocolbuffer-encoding/src/main/resources/Persistent.proto @@ -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; } +